From 94011d91ba29e92f3dcd43fd77814af869fc4800 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 9 Jun 2021 02:03:36 +0000 Subject: [PATCH 001/919] initial implement --- src/CMakeLists.txt | 2 + src/Storages/FileLog/Buffer_fwd.h | 10 + .../FileLog/FileLogBlockInputStream.cpp | 173 ++++++++++ .../FileLog/FileLogBlockInputStream.h | 49 +++ .../FileLog/ReadBufferFromFileLog.cpp | 156 +++++++++ src/Storages/FileLog/ReadBufferFromFileLog.h | 95 +++++ src/Storages/FileLog/StorageFileLog.cpp | 326 ++++++++++++++++++ src/Storages/FileLog/StorageFileLog.h | 92 +++++ src/Storages/registerStorages.cpp | 6 +- 9 files changed, 908 insertions(+), 1 deletion(-) create mode 100644 src/Storages/FileLog/Buffer_fwd.h create mode 100644 src/Storages/FileLog/FileLogBlockInputStream.cpp create mode 100644 src/Storages/FileLog/FileLogBlockInputStream.h create mode 100644 src/Storages/FileLog/ReadBufferFromFileLog.cpp create mode 100644 src/Storages/FileLog/ReadBufferFromFileLog.h create mode 100644 src/Storages/FileLog/StorageFileLog.cpp create mode 100644 src/Storages/FileLog/StorageFileLog.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 272bea4f6d7..b82c72d19f4 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -104,6 +104,8 @@ if (USE_HDFS) add_headers_and_sources(dbms Disks/HDFS) endif() +add_headers_and_sources(dbms Storages/FileLog) + list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) list (APPEND clickhouse_common_io_headers ${CONFIG_VERSION} ${CONFIG_COMMON}) diff --git a/src/Storages/FileLog/Buffer_fwd.h b/src/Storages/FileLog/Buffer_fwd.h new file mode 100644 index 00000000000..ec644aa7d36 --- /dev/null +++ b/src/Storages/FileLog/Buffer_fwd.h @@ -0,0 +1,10 @@ +#pragma once + +#include + +namespace DB +{ +class ReadBufferFromFileLog; + +using ReadBufferFromFileLogPtr = std::shared_ptr; +} diff --git a/src/Storages/FileLog/FileLogBlockInputStream.cpp b/src/Storages/FileLog/FileLogBlockInputStream.cpp new file mode 100644 index 00000000000..538fbb6049a --- /dev/null +++ b/src/Storages/FileLog/FileLogBlockInputStream.cpp @@ -0,0 +1,173 @@ +#include + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +// with default poll timeout (500ms) it will give about 5 sec delay for doing 10 retries +// when selecting from empty topic +const auto MAX_FAILED_POLL_ATTEMPTS = 10; + +FileLogBlockInputStream::FileLogBlockInputStream( + StorageFileLog & storage_, + const StorageMetadataPtr & metadata_snapshot_, + const std::shared_ptr & context_, + const Names & columns, + Poco::Logger * log_, + size_t max_block_size_) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , context(context_) + , column_names(columns) + , log(log_) + , max_block_size(max_block_size_) + , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) + , virtual_header( + metadata_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames(), storage.getVirtuals(), storage.getStorageID())) +{ +} + +Block FileLogBlockInputStream::getHeader() const +{ + return metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID()); +} + +void FileLogBlockInputStream::readPrefixImpl() +{ + buffer = storage.getBuffer(); + + if (!buffer) + return; + + buffer->open(); +} + +Block FileLogBlockInputStream::readImpl() +{ + if (!buffer || finished) + return Block(); + + finished = true; + // now it's one-time usage InputStream + // one block of the needed size (or with desired flush timeout) is formed in one internal iteration + // otherwise external iteration will reuse that and logic will became even more fuzzy + MutableColumns result_columns = non_virtual_header.cloneEmptyColumns(); + // MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); + + auto input_format = FormatFactory::instance().getInputFormat( + storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); + + InputPort port(input_format->getPort().getHeader(), input_format.get()); + connect(input_format->getPort(), port); + port.setNeeded(); + + std::optional exception_message; + auto read_file_log = [&] { + size_t new_rows = 0; + while (true) + { + auto status = input_format->prepare(); + + switch (status) + { + case IProcessor::Status::Ready: + input_format->work(); + break; + + case IProcessor::Status::Finished: + input_format->resetParser(); + return new_rows; + + case IProcessor::Status::PortFull: + { + auto chunk = port.pull(); + + // that was returning bad value before https://github.com/ClickHouse/ClickHouse/pull/8005 + // if will be backported should go together with #8005 + auto chunk_rows = chunk.getNumRows(); + new_rows += chunk_rows; + + auto columns = chunk.detachColumns(); + for (size_t i = 0, s = columns.size(); i < s; ++i) + { + result_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); + } + break; + } + case IProcessor::Status::NeedData: + case IProcessor::Status::Async: + case IProcessor::Status::ExpandPipeline: + throw Exception("Source processor returned status " + IProcessor::statusToName(status), ErrorCodes::LOGICAL_ERROR); + } + } + }; + + size_t total_rows = 0; + + while (true) + { + size_t new_rows = 0; + exception_message.reset(); + if (buffer->poll()) + { + try + { + new_rows = read_file_log(); + } + catch (Exception & e) + { + throw; + } + } + if (new_rows) + { + total_rows = total_rows + new_rows; + } + + if (!buffer->hasMorePolledRecords() && (total_rows >= max_block_size || !checkTimeLimit())) + { + break; + } + } + + if (total_rows == 0) + return Block(); + + /// MATERIALIZED columns can be added here, but I think + // they are not needed here: + // and it's misleading to use them here, + // as columns 'materialized' that way stays 'ephemeral' + // i.e. will not be stored anythere + // If needed any extra columns can be added using DEFAULT they can be added at MV level if needed. + + auto result_block = non_virtual_header.cloneWithColumns(std::move(result_columns)); + // auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); + + // for (const auto & column : virtual_block.getColumnsWithTypeAndName()) + // result_block.insert(column); + + return ConvertingBlockInputStream( + std::make_shared(result_block), + getHeader(), + ConvertingBlockInputStream::MatchColumnsMode::Name) + .read(); +} + +void FileLogBlockInputStream::readSuffixImpl() +{ + if (buffer) + buffer->close(); +} + +} diff --git a/src/Storages/FileLog/FileLogBlockInputStream.h b/src/Storages/FileLog/FileLogBlockInputStream.h new file mode 100644 index 00000000000..9a625e8499d --- /dev/null +++ b/src/Storages/FileLog/FileLogBlockInputStream.h @@ -0,0 +1,49 @@ +#pragma once + +#include + +#include +#include + + +namespace Poco +{ + class Logger; +} +namespace DB +{ +class FileLogBlockInputStream : public IBlockInputStream +{ +public: + FileLogBlockInputStream( + StorageFileLog & storage_, + const StorageMetadataPtr & metadata_snapshot_, + const std::shared_ptr & context_, + const Names & columns, + Poco::Logger * log_, + size_t max_block_size_); + ~FileLogBlockInputStream() override = default; + + String getName() const override { return storage.getName(); } + Block getHeader() const override; + + void readPrefixImpl() override; + Block readImpl() override; + void readSuffixImpl() override; + +private: + StorageFileLog & storage; + StorageMetadataPtr metadata_snapshot; + ContextPtr context; + Names column_names; + Poco::Logger * log; + UInt64 max_block_size; + + ReadBufferFromFileLogPtr buffer; + bool finished = false; + + const Block non_virtual_header; + const Block virtual_header; +}; + +} diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp new file mode 100644 index 00000000000..f2dabdc2b83 --- /dev/null +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -0,0 +1,156 @@ +#include +#include + +#include +#include + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_COMMIT_OFFSET; +} + +using namespace std::chrono_literals; + +ReadBufferFromFileLog::ReadBufferFromFileLog( + const std::vector & log_files_, + Poco::Logger * log_, + size_t max_batch_size, + size_t poll_timeout_, + ContextPtr context_) + : ReadBuffer(nullptr, 0) + , log(log_) + , batch_size(max_batch_size) + , poll_timeout(poll_timeout_) + , context(context_) + , log_files(log_files_.begin(), log_files_.end()) +{ +} + +void ReadBufferFromFileLog::open() +{ + for (const auto & file : log_files) + file_status[file].reader = std::ifstream(file); + + wait_task = context->getMessageBrokerSchedulePool().createTask("waitTask", [this] { waitFunc(); }); + wait_task->deactivate(); + + select_task = context->getMessageBrokerSchedulePool().createTask("selectTask", [this] { selectFunc(); }); + select_task->activateAndSchedule(); + + cleanUnprocessed(); + allowed = false; +} + +void ReadBufferFromFileLog::cleanUnprocessed() +{ + records.clear(); + current = records.begin(); + BufferBase::set(nullptr, 0, 0); +} + +void ReadBufferFromFileLog::close() +{ + wait_task->deactivate(); + select_task->deactivate(); + + for (auto & status : file_status) + status.second.reader.close(); +} + +// it do the poll when needed +bool ReadBufferFromFileLog::poll() +{ + + if (hasMorePolledRecords()) + { + allowed = true; + return true; + } + + auto new_records = pollBatch(batch_size); + if (new_records.empty()) + { + LOG_TRACE(log, "No records returned"); + return false; + } + else + { + records = std::move(new_records); + current = records.begin(); + LOG_TRACE(log, "Polled batch of {} records. ", records.size()); + allowed = true; + return true; + } + + +} + +ReadBufferFromFileLog::Records ReadBufferFromFileLog::pollBatch(size_t batch_size_) +{ + Records new_records; + new_records.reserve(batch_size_); + + readNewRecords(new_records, batch_size); + if (new_records.size() == batch_size_) + return new_records; + + wait_task->activateAndSchedule(); + while (!time_out && new_records.size() != batch_size_) + { + readNewRecords(new_records, batch_size); + } + + wait_task->deactivate(); + time_out = false; + return new_records; +} + +void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_records, size_t batch_size_) +{ + size_t need_records_size = batch_size_ - new_records.size(); + size_t read_records_size = 0; + for (auto & status : file_status) + { + while (read_records_size < need_records_size && status.second.reader.good() && !status.second.reader.eof()) + { + Record record; + std::getline(status.second.reader, record); + new_records.emplace_back(record); + ++read_records_size; + } + if (read_records_size == need_records_size) + break; + } +} + +bool ReadBufferFromFileLog::nextImpl() +{ + if (!allowed || !hasMorePolledRecords()) + return false; + + auto * new_position = const_cast(current->data()); + BufferBase::set(new_position, current->size(), 0); + allowed = false; + + ++current; + + return true; +} + +void ReadBufferFromFileLog::waitFunc() +{ + sleepForMicroseconds(poll_timeout); + time_out = true; +} + +void ReadBufferFromFileLog::selectFunc() +{ +} +} + diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h new file mode 100644 index 00000000000..e6a635fcaeb --- /dev/null +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -0,0 +1,95 @@ +#pragma once + +#include +#include +#include +#include + +#include +#include +#include + +namespace Poco +{ + class Logger; +} + +namespace DB +{ +class ReadBufferFromFileLog : public ReadBuffer +{ +public: + ReadBufferFromFileLog( + const std::vector & log_files_, + Poco::Logger * log_, + size_t max_batch_size, + size_t poll_timeout_, + ContextPtr context_); + + ~ReadBufferFromFileLog() override = default; + + void open(); + void close(); + + auto pollTimeout() const { return poll_timeout; } + + inline bool hasMorePolledRecords() const { return current != records.end(); } + + bool poll(); + +private: + enum class FileStatus + { + BEGIN, + NO_CHANGE, + UPDATED, + REMOVED + }; + + struct FileContext + { + std::mutex status_mutex; + FileStatus status = FileStatus::BEGIN; + std::ifstream reader; + }; + + Poco::Logger * log; + const size_t batch_size = 1; + const size_t poll_timeout = 0; + + bool time_out = false; + + using NameToFile = std::unordered_map; + NameToFile file_status; + + ContextPtr context; + + bool allowed = true; + + const std::vector log_files; + + using Record = std::string; + using Records = std::vector; + + Records records; + Records::const_iterator current; + + using TaskThread = BackgroundSchedulePool::TaskHolder; + + TaskThread wait_task; + TaskThread select_task; + + Records pollBatch(size_t batch_size_); + + void readNewRecords(Records & new_records, size_t batch_size_); + + // void drain(); + void cleanUnprocessed(); + + bool nextImpl() override; + + void waitFunc(); + + void selectFunc(); +}; +} diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp new file mode 100644 index 00000000000..33026430e99 --- /dev/null +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -0,0 +1,326 @@ +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +namespace +{ + const auto RESCHEDULE_MS = 500; + const auto MAX_THREAD_WORK_DURATION_MS = 60000; // once per minute leave do reschedule (we can't lock threads in pool forever) +} + +StorageFileLog::StorageFileLog( + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + const String & path_, + const String & format_name_) + : IStorage(table_id_) + , WithContext(context_->getGlobalContext()) + , path(path_) + , format_name(format_name_) + , log(&Poco::Logger::get("StorageFile (" + table_id_.table_name + ")")) +{ + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + setInMemoryMetadata(storage_metadata); + + Poco::File file(path); + + if (file.isFile()) + { + log_files.emplace_back(path); + } + else if (file.isDirectory()) + { + Poco::DirectoryIterator dir_iter(file); + Poco::DirectoryIterator end; + while (dir_iter != end) + { + if (dir_iter->isFile()) + log_files.emplace_back(dir_iter->path()); + ++dir_iter; + } + } + + auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); + thread->deactivate(); + task = std::make_shared(std::move(thread)); +} + +Pipe StorageFileLog::read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + SelectQueryInfo & /* query_info */, + ContextPtr local_context, + QueryProcessingStage::Enum /* processed_stage */, + size_t /* max_block_size */, + unsigned /* num_streams */) +{ + auto modified_context = Context::createCopy(local_context); + + return Pipe(std::make_shared( + std::make_shared(*this, metadata_snapshot, modified_context, column_names, log, 1))); + +} + +void StorageFileLog::startup() +{ + try + { + createReadBuffer(); + } + catch (const Exception &) + { + tryLogCurrentException(log); + } + + task->holder->activateAndSchedule(); +} + + +void StorageFileLog::shutdown() +{ + task->stream_cancelled = true; + + LOG_TRACE(log, "Waiting for cleanup"); + task->holder->deactivate(); + + LOG_TRACE(log, "Closing files"); + destroyReadBuffer(); +} + +size_t StorageFileLog::getMaxBlockSize() const +{ + return getContext()->getSettingsRef().max_insert_block_size.value; +} + +size_t StorageFileLog::getPollMaxBatchSize() const +{ + size_t batch_size = getContext()->getSettingsRef().max_block_size.value; + + return std::min(batch_size,getMaxBlockSize()); +} + +size_t StorageFileLog::getPollTimeoutMillisecond() const +{ + return getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); +} + + +bool StorageFileLog::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 StorageFileLog::createReadBuffer() +{ + auto new_context = Context::createCopy(getContext()); + buffer = std::make_shared(log_files, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), new_context); +} + +void StorageFileLog::destroyReadBuffer() +{ + if (buffer) + buffer->close(); +} + +void StorageFileLog::threadFunc() +{ + try + { + auto table_id = getStorageID(); + // Check if at least one direct dependency is attached + size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); + if (dependencies_count) + { + auto start_time = std::chrono::steady_clock::now(); + + // Keep streaming as long as there are attached views and streaming is not cancelled + while (!task->stream_cancelled) + { + if (!checkDependencies(table_id)) + break; + + LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); + + // Exit the loop & reschedule if some stream stalled + auto some_stream_is_stalled = streamToViews(); + if (some_stream_is_stalled) + { + LOG_TRACE(log, "Stream(s) stalled. Reschedule."); + break; + } + + auto ts = std::chrono::steady_clock::now(); + auto duration = std::chrono::duration_cast(ts-start_time); + if (duration.count() > MAX_THREAD_WORK_DURATION_MS) + { + LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); + break; + } + } + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + // Wait for attached views + if (!task->stream_cancelled) + task->holder->scheduleAfter(RESCHEDULE_MS); +} + + +bool StorageFileLog::streamToViews() +{ + Stopwatch watch; + + 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); + auto metadata_snapshot = getInMemoryMetadataPtr(); + + // Create an INSERT query for streaming data + auto insert = std::make_shared(); + insert->table_id = table_id; + + size_t block_size = getMaxBlockSize(); + + auto new_context = Context::createCopy(getContext()); + + // Only insert into dependent views and expect that input blocks contain virtual columns + InterpreterInsertQuery interpreter(insert, new_context, false, true, true); + auto block_io = interpreter.execute(); + + auto stream = std::make_shared( + *this, metadata_snapshot, new_context, block_io.out->getHeader().getNames(), log, block_size); + + // Limit read batch to maximum block size to allow DDL + StreamLocalLimits limits; + + limits.speed_limits.max_execution_time = getContext()->getSettingsRef().stream_flush_interval_ms; + + limits.timeout_overflow_mode = OverflowMode::BREAK; + stream->setLimits(limits); + + // We can't cancel during copyData, as it's not aware of commits and other kafka-related stuff. + // It will be cancelled on underlying layer (kafka buffer) + std::atomic stub = {false}; + size_t rows = 0; + copyData( + *stream, *block_io.out, [&rows](const Block & block) { rows += block.rows(); }, &stub); + + UInt64 milliseconds = watch.elapsedMilliseconds(); + LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", + formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); + + return true; +} + +void registerStorageFileLog(StorageFactory & factory) +{ + auto creator_fn = [](const StorageFactory::Arguments & args) + { + ASTs & engine_args = args.engine_args; + size_t args_count = engine_args.size(); + + if (args_count != 2) + throw Exception("Arguments size of StorageFileLog should be 2, path and format name", ErrorCodes::BAD_ARGUMENTS); + + 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(); + + return StorageFileLog::create(args.table_id, args.getContext(), args.columns, path, format); + }; + + factory.registerStorage( + "FileLog", + creator_fn, + StorageFactory::StorageFeatures{ + .supports_settings = true, + }); +} + +NamesAndTypesList StorageFileLog::getVirtuals() const +{ + auto result = NamesAndTypesList{}; + return result; +} + +Names StorageFileLog::getVirtualColumnNames() +{ + auto result = Names{}; + return result; +} + +} diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h new file mode 100644 index 00000000000..9cb7556ea05 --- /dev/null +++ b/src/Storages/FileLog/StorageFileLog.h @@ -0,0 +1,92 @@ +#pragma once + +#include + +#include +#include +#include + +#include +#include +#include + +#include +#include +#include + +namespace DB +{ +class StorageFileLog final : public ext::shared_ptr_helper, public IStorage, WithContext +{ + friend struct ext::shared_ptr_helper; + +public: + using Files = std::vector; + + std::string getName() const override { return "FileLog"; } + + bool noPushingToViews() const override { return true; } + + void startup() override; + void shutdown() override; + + Pipe read( + const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + + const auto & getFormatName() const { return format_name; } + + NamesAndTypesList getVirtuals() const override; + static Names getVirtualColumnNames(); + + auto & getBuffer() { return buffer; } + +protected: + StorageFileLog( + const StorageID & table_id_, + ContextPtr context_, + const ColumnsDescription & columns_, + const String & path_, + const String & format_name_); + +private: + const String path; + const String format_name; + Poco::Logger * log; + + Files log_files; + + ReadBufferFromFileLogPtr buffer; + + std::mutex mutex; + + // Stream thread + struct TaskContext + { + BackgroundSchedulePool::TaskHolder holder; + std::atomic stream_cancelled {false}; + explicit TaskContext(BackgroundSchedulePool::TaskHolder&& task_) : holder(std::move(task_)) + { + } + }; + std::shared_ptr task; + + void createReadBuffer(); + void destroyReadBuffer(); + + void threadFunc(); + + size_t getPollMaxBatchSize() const; + size_t getMaxBlockSize() const; + size_t getPollTimeoutMillisecond() const; + + bool streamToViews(); + bool checkDependencies(const StorageID & table_id); +}; + +} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 0b302ee437a..7d32139ca64 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -50,6 +50,8 @@ void registerStorageMongoDB(StorageFactory & factory); void registerStorageKafka(StorageFactory & factory); #endif +void registerStorageFileLog(StorageFactory & factory); + #if USE_AMQPCPP void registerStorageRabbitMQ(StorageFactory & factory); #endif @@ -112,7 +114,9 @@ void registerStorages() registerStorageKafka(factory); #endif - #if USE_AMQPCPP + registerStorageFileLog(factory); + +#if USE_AMQPCPP registerStorageRabbitMQ(factory); #endif From b15099fe6fb241b1a83069350a787f90ae0b7e3e Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 3 Jul 2021 17:14:56 +0000 Subject: [PATCH 002/919] add directory watcher --- src/CMakeLists.txt | 6 +- src/Core/config_core.h.in | 1 + .../FileLog/FileLogBlockInputStream.cpp | 27 +----- .../FileLog/FileLogDirectoryWatcher.cpp | 89 +++++++++++++++++ .../FileLog/FileLogDirectoryWatcher.h | 48 ++++++++++ .../FileLog/ReadBufferFromFileLog.cpp | 96 +++++++++++++++---- src/Storages/FileLog/ReadBufferFromFileLog.h | 18 ++-- src/Storages/FileLog/StorageFileLog.cpp | 34 +------ src/Storages/FileLog/StorageFileLog.h | 2 - ...StorageSystemBuildOptions.generated.cpp.in | 2 + src/Storages/registerStorages.cpp | 8 +- 11 files changed, 243 insertions(+), 88 deletions(-) create mode 100644 src/Storages/FileLog/FileLogDirectoryWatcher.cpp create mode 100644 src/Storages/FileLog/FileLogDirectoryWatcher.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index b82c72d19f4..c05a39e7239 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -104,7 +104,11 @@ if (USE_HDFS) add_headers_and_sources(dbms Disks/HDFS) endif() -add_headers_and_sources(dbms Storages/FileLog) +set (USE_FILELOG 1) + +if(USE_FILELOG) + add_headers_and_sources(dbms Storages/FileLog) +endif() list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) list (APPEND clickhouse_common_io_headers ${CONFIG_VERSION} ${CONFIG_COMMON}) diff --git a/src/Core/config_core.h.in b/src/Core/config_core.h.in index e250e013913..a1d7f2b8d0d 100644 --- a/src/Core/config_core.h.in +++ b/src/Core/config_core.h.in @@ -15,3 +15,4 @@ #cmakedefine01 USE_LIBPQXX #cmakedefine01 USE_NURAFT #cmakedefine01 USE_KRB5 +#cmakedefine01 USE_FILELOG diff --git a/src/Storages/FileLog/FileLogBlockInputStream.cpp b/src/Storages/FileLog/FileLogBlockInputStream.cpp index 538fbb6049a..eb82c891fd6 100644 --- a/src/Storages/FileLog/FileLogBlockInputStream.cpp +++ b/src/Storages/FileLog/FileLogBlockInputStream.cpp @@ -15,10 +15,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -// with default poll timeout (500ms) it will give about 5 sec delay for doing 10 retries -// when selecting from empty topic -const auto MAX_FAILED_POLL_ATTEMPTS = 10; - FileLogBlockInputStream::FileLogBlockInputStream( StorageFileLog & storage_, const StorageMetadataPtr & metadata_snapshot_, @@ -55,15 +51,10 @@ void FileLogBlockInputStream::readPrefixImpl() Block FileLogBlockInputStream::readImpl() { - if (!buffer || finished) + if (!buffer) return Block(); - finished = true; - // now it's one-time usage InputStream - // one block of the needed size (or with desired flush timeout) is formed in one internal iteration - // otherwise external iteration will reuse that and logic will became even more fuzzy MutableColumns result_columns = non_virtual_header.cloneEmptyColumns(); - // MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); auto input_format = FormatFactory::instance().getInputFormat( storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); @@ -93,8 +84,6 @@ Block FileLogBlockInputStream::readImpl() { auto chunk = port.pull(); - // that was returning bad value before https://github.com/ClickHouse/ClickHouse/pull/8005 - // if will be backported should go together with #8005 auto chunk_rows = chunk.getNumRows(); new_rows += chunk_rows; @@ -125,7 +114,7 @@ Block FileLogBlockInputStream::readImpl() { new_rows = read_file_log(); } - catch (Exception & e) + catch (Exception &) { throw; } @@ -133,6 +122,7 @@ Block FileLogBlockInputStream::readImpl() if (new_rows) { total_rows = total_rows + new_rows; + LOG_INFO(log, "FileLogBlockInputStream, {} rows data polled from buffer.", new_rows); } if (!buffer->hasMorePolledRecords() && (total_rows >= max_block_size || !checkTimeLimit())) @@ -144,18 +134,7 @@ Block FileLogBlockInputStream::readImpl() if (total_rows == 0) return Block(); - /// MATERIALIZED columns can be added here, but I think - // they are not needed here: - // and it's misleading to use them here, - // as columns 'materialized' that way stays 'ephemeral' - // i.e. will not be stored anythere - // If needed any extra columns can be added using DEFAULT they can be added at MV level if needed. - auto result_block = non_virtual_header.cloneWithColumns(std::move(result_columns)); - // auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); - - // for (const auto & column : virtual_block.getColumnsWithTypeAndName()) - // result_block.insert(column); return ConvertingBlockInputStream( std::make_shared(result_block), diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp new file mode 100644 index 00000000000..e95f7c55c4d --- /dev/null +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -0,0 +1,89 @@ +#include +#include +#include + +FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_) + : path(path_), dw(std::make_shared(path)) +{ + dw->itemAdded += Poco::delegate(this, &FileLogDirectoryWatcher::onItemAdded); + dw->itemRemoved += Poco::delegate(this, &FileLogDirectoryWatcher::onItemRemoved); + dw->itemModified += Poco::delegate(this, &FileLogDirectoryWatcher::onItemModified); + dw->itemMovedFrom += Poco::delegate(this, &FileLogDirectoryWatcher::onItemMovedFrom); + dw->itemMovedTo += Poco::delegate(this, &FileLogDirectoryWatcher::onItemMovedTo); +} + +FileLogDirectoryWatcher::Events FileLogDirectoryWatcher::getEvents() +{ + std::lock_guard lock(mutex); + Events res; + res.swap(events); + return res; +} + +bool FileLogDirectoryWatcher::getError() const +{ + return error; +} + +const std::string & FileLogDirectoryWatcher::getPath() const +{ + return path; +} + +void FileLogDirectoryWatcher::onItemAdded(const Poco::DirectoryWatcher::DirectoryEvent& ev) +{ + std::lock_guard lock(mutex); + DirEvent de; + de.callback = "onItemAdded"; + de.path = ev.item.path(); + de.type = ev.event; + events.push_back(de); +} + + +void FileLogDirectoryWatcher::onItemRemoved(const Poco::DirectoryWatcher::DirectoryEvent& ev) +{ + std::lock_guard lock(mutex); + DirEvent de; + de.callback = "onItemRemoved"; + de.path = ev.item.path(); + de.type = ev.event; + events.push_back(de); +} + + +void FileLogDirectoryWatcher::onItemModified(const Poco::DirectoryWatcher::DirectoryEvent& ev) +{ + std::lock_guard lock(mutex); + DirEvent de; + de.callback = "onItemModified"; + de.path = ev.item.path(); + de.type = ev.event; + events.push_back(de); +} + +void FileLogDirectoryWatcher::onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent& ev) +{ + std::lock_guard lock(mutex); + DirEvent de; + de.callback = "onItemMovedFrom"; + de.path = ev.item.path(); + de.type = ev.event; + events.push_back(de); +} + +void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent& ev) +{ + std::lock_guard lock(mutex); + DirEvent de; + de.callback = "onItemMovedTo"; + de.path = ev.item.path(); + de.type = ev.event; + events.push_back(de); +} + + +void FileLogDirectoryWatcher::onError(const Poco::Exception &) +{ + error = true; +} diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h new file mode 100644 index 00000000000..b0fde72f57c --- /dev/null +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include +#include + +#include +#include + +class FileLogDirectoryWatcher +{ +public: + struct DirEvent + { + Poco::DirectoryWatcher::DirectoryEventType type; + std::string callback; + std::string path; + }; + + using Events = std::deque; + + explicit FileLogDirectoryWatcher(const std::string & path_); + ~FileLogDirectoryWatcher() = default; + + Events getEvents(); + + bool getError() const; + + const std::string & getPath() const; + +protected: + void onItemAdded(const Poco::DirectoryWatcher::DirectoryEvent& ev); + void onItemRemoved(const Poco::DirectoryWatcher::DirectoryEvent& ev); + void onItemModified(const Poco::DirectoryWatcher::DirectoryEvent& ev); + void onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent& ev); + void onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent& ev); + void onError(const Poco::Exception &); + +private: + const std::string path; + std::shared_ptr dw; + + std::mutex mutex; + + Events events; + + bool error = false; +}; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index f2dabdc2b83..1eedbee136a 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -1,5 +1,7 @@ #include #include +#include +#include #include #include @@ -15,33 +17,49 @@ namespace ErrorCodes extern const int CANNOT_COMMIT_OFFSET; } -using namespace std::chrono_literals; - ReadBufferFromFileLog::ReadBufferFromFileLog( - const std::vector & log_files_, - Poco::Logger * log_, - size_t max_batch_size, - size_t poll_timeout_, - ContextPtr context_) + const String & path_, Poco::Logger * log_, size_t max_batch_size, size_t poll_timeout_, ContextPtr context_) : ReadBuffer(nullptr, 0) + , path(path_) , log(log_) , batch_size(max_batch_size) , poll_timeout(poll_timeout_) , context(context_) - , log_files(log_files_.begin(), log_files_.end()) { } void ReadBufferFromFileLog::open() { - for (const auto & file : log_files) - file_status[file].reader = std::ifstream(file); + Poco::File file(path); + + bool path_is_directory = false; + + if (file.isFile()) + { + file_status[path].reader = std::ifstream(path); + } + else if (file.isDirectory()) + { + path_is_directory = true; + Poco::DirectoryIterator dir_iter(file); + Poco::DirectoryIterator end; + while (dir_iter != end) + { + if (dir_iter->isFile()) + file_status[dir_iter->path()].reader = std::ifstream(dir_iter->path()); + ++dir_iter; + } + } wait_task = context->getMessageBrokerSchedulePool().createTask("waitTask", [this] { waitFunc(); }); wait_task->deactivate(); - select_task = context->getMessageBrokerSchedulePool().createTask("selectTask", [this] { selectFunc(); }); - select_task->activateAndSchedule(); + if (path_is_directory) + { + FileLogDirectoryWatcher dw(path); + select_task = context->getMessageBrokerSchedulePool().createTask("watchTask", [this, &dw] { watchFunc(dw); }); + select_task->activateAndSchedule(); + } cleanUnprocessed(); allowed = false; @@ -63,7 +81,6 @@ void ReadBufferFromFileLog::close() status.second.reader.close(); } -// it do the poll when needed bool ReadBufferFromFileLog::poll() { @@ -113,10 +130,19 @@ ReadBufferFromFileLog::Records ReadBufferFromFileLog::pollBatch(size_t batch_siz void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_records, size_t batch_size_) { + std::lock_guard lock(status_mutex); + size_t need_records_size = batch_size_ - new_records.size(); size_t read_records_size = 0; + for (auto & status : file_status) { + if (status.second.status == FileStatus::NO_CHANGE) + continue; + + if (status.second.status == FileStatus::REMOVED) + file_status.erase(status.first); + while (read_records_size < need_records_size && status.second.reader.good() && !status.second.reader.eof()) { Record record; @@ -124,6 +150,11 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ new_records.emplace_back(record); ++read_records_size; } + + // Read to the end of the file + if (status.second.reader.eof()) + status.second.status = FileStatus::NO_CHANGE; + if (read_records_size == need_records_size) break; } @@ -149,8 +180,41 @@ void ReadBufferFromFileLog::waitFunc() time_out = true; } -void ReadBufferFromFileLog::selectFunc() +void ReadBufferFromFileLog::watchFunc(FileLogDirectoryWatcher & dw) { -} -} + while (true) + { + sleepForNanoseconds(poll_timeout); + auto error = dw.getError(); + if (error) + LOG_INFO(log, "Error happened during watching directory {}.", dw.getPath()); + + auto events = dw.getEvents(); + std::lock_guard lock(status_mutex); + + for (const auto & event : events) + { + switch (event.type) + { + case Poco::DirectoryWatcher::DW_ITEM_ADDED: + file_status[event.path].reader = std::ifstream(event.path); + break; + + case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: + file_status[event.path].status = FileStatus::UPDATED; + break; + + case Poco::DirectoryWatcher::DW_ITEM_REMOVED: + case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: + case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: + file_status[event.path].status = FileStatus::REMOVED; + break; + + default: + LOG_INFO(log, "Undefine event type"); + } + } + } +} +} diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index e6a635fcaeb..20b6b6d0c2d 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -19,12 +20,7 @@ namespace DB class ReadBufferFromFileLog : public ReadBuffer { public: - ReadBufferFromFileLog( - const std::vector & log_files_, - Poco::Logger * log_, - size_t max_batch_size, - size_t poll_timeout_, - ContextPtr context_); + ReadBufferFromFileLog(const String & path_, Poco::Logger * log_, size_t max_batch_size, size_t poll_timeout_, ContextPtr context_); ~ReadBufferFromFileLog() override = default; @@ -48,11 +44,12 @@ private: struct FileContext { - std::mutex status_mutex; FileStatus status = FileStatus::BEGIN; std::ifstream reader; }; + const String path; + Poco::Logger * log; const size_t batch_size = 1; const size_t poll_timeout = 0; @@ -62,12 +59,12 @@ private: using NameToFile = std::unordered_map; NameToFile file_status; + std::mutex status_mutex; + ContextPtr context; bool allowed = true; - const std::vector log_files; - using Record = std::string; using Records = std::vector; @@ -83,13 +80,12 @@ private: void readNewRecords(Records & new_records, size_t batch_size_); - // void drain(); void cleanUnprocessed(); bool nextImpl() override; void waitFunc(); - void selectFunc(); + void watchFunc(FileLogDirectoryWatcher & dw); }; } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 33026430e99..7970feb2996 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -23,7 +23,6 @@ #include #include #include -#include #include #include #include @@ -67,24 +66,6 @@ StorageFileLog::StorageFileLog( storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); - Poco::File file(path); - - if (file.isFile()) - { - log_files.emplace_back(path); - } - else if (file.isDirectory()) - { - Poco::DirectoryIterator dir_iter(file); - Poco::DirectoryIterator end; - while (dir_iter != end) - { - if (dir_iter->isFile()) - log_files.emplace_back(dir_iter->path()); - ++dir_iter; - } - } - auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); thread->deactivate(); task = std::make_shared(std::move(thread)); @@ -157,7 +138,6 @@ bool StorageFileLog::checkDependencies(const StorageID & 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()); @@ -180,7 +160,7 @@ bool StorageFileLog::checkDependencies(const StorageID & table_id) void StorageFileLog::createReadBuffer() { auto new_context = Context::createCopy(getContext()); - buffer = std::make_shared(log_files, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), new_context); + buffer = std::make_shared(path, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), new_context); } void StorageFileLog::destroyReadBuffer() @@ -208,13 +188,7 @@ void StorageFileLog::threadFunc() LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); - // Exit the loop & reschedule if some stream stalled - auto some_stream_is_stalled = streamToViews(); - if (some_stream_is_stalled) - { - LOG_TRACE(log, "Stream(s) stalled. Reschedule."); - break; - } + streamToViews(); auto ts = std::chrono::steady_clock::now(); auto duration = std::chrono::duration_cast(ts-start_time); @@ -255,14 +229,12 @@ bool StorageFileLog::streamToViews() auto new_context = Context::createCopy(getContext()); - // Only insert into dependent views and expect that input blocks contain virtual columns InterpreterInsertQuery interpreter(insert, new_context, false, true, true); auto block_io = interpreter.execute(); auto stream = std::make_shared( *this, metadata_snapshot, new_context, block_io.out->getHeader().getNames(), log, block_size); - // Limit read batch to maximum block size to allow DDL StreamLocalLimits limits; limits.speed_limits.max_execution_time = getContext()->getSettingsRef().stream_flush_interval_ms; @@ -270,8 +242,6 @@ bool StorageFileLog::streamToViews() limits.timeout_overflow_mode = OverflowMode::BREAK; stream->setLimits(limits); - // We can't cancel during copyData, as it's not aware of commits and other kafka-related stuff. - // It will be cancelled on underlying layer (kafka buffer) std::atomic stub = {false}; size_t rows = 0; copyData( diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 9cb7556ea05..dda759c7e38 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -59,8 +59,6 @@ private: const String format_name; Poco::Logger * log; - Files log_files; - ReadBufferFromFileLogPtr buffer; std::mutex mutex; diff --git a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in index 8fe574da643..18cab0700ec 100644 --- a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in @@ -50,6 +50,8 @@ const char * auto_config_build[] "USE_LDAP", "@USE_LDAP@", "TZDATA_VERSION", "@TZDATA_VERSION@", "USE_KRB5", "@USE_KRB5@", + "USE_KRB5", "@USE_KRB5@", + "USE_FILELOG", "@USE_FILELOG@", nullptr, nullptr }; diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 7d32139ca64..863015ed828 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -50,8 +50,6 @@ void registerStorageMongoDB(StorageFactory & factory); void registerStorageKafka(StorageFactory & factory); #endif -void registerStorageFileLog(StorageFactory & factory); - #if USE_AMQPCPP void registerStorageRabbitMQ(StorageFactory & factory); #endif @@ -69,6 +67,10 @@ void registerStorageMaterializedPostgreSQL(StorageFactory & factory); void registerStorageExternalDistributed(StorageFactory & factory); #endif +#if USE_FILELOG +void registerStorageFileLog(StorageFactory & factory); +#endif + void registerStorages() { auto & factory = StorageFactory::instance(); @@ -114,7 +116,9 @@ void registerStorages() registerStorageKafka(factory); #endif +#if USE_FILELOG registerStorageFileLog(factory); +#endif #if USE_AMQPCPP registerStorageRabbitMQ(factory); From eafe8a2e595cd25c5958e0c77dbd717f49d9da1d Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 3 Jul 2021 17:53:09 +0000 Subject: [PATCH 003/919] fix --- CMakeLists.txt | 2 ++ src/CMakeLists.txt | 2 -- src/Storages/FileLog/FileLogBlockInputStream.h | 1 - src/Storages/FileLog/ReadBufferFromFileLog.cpp | 3 --- src/Storages/FileLog/ReadBufferFromFileLog.h | 2 +- 5 files changed, 3 insertions(+), 7 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index d23e5f540d3..8522a6ff42b 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -553,6 +553,8 @@ include (cmake/find/sentry.cmake) include (cmake/find/stats.cmake) include (cmake/find/datasketches.cmake) +set (USE_FILELOG ON) + set (USE_INTERNAL_CITYHASH_LIBRARY ON CACHE INTERNAL "") find_contrib_lib(cityhash) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index c05a39e7239..c906c730641 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -104,8 +104,6 @@ if (USE_HDFS) add_headers_and_sources(dbms Disks/HDFS) endif() -set (USE_FILELOG 1) - if(USE_FILELOG) add_headers_and_sources(dbms Storages/FileLog) endif() diff --git a/src/Storages/FileLog/FileLogBlockInputStream.h b/src/Storages/FileLog/FileLogBlockInputStream.h index 9a625e8499d..290d3cf07f8 100644 --- a/src/Storages/FileLog/FileLogBlockInputStream.h +++ b/src/Storages/FileLog/FileLogBlockInputStream.h @@ -40,7 +40,6 @@ private: UInt64 max_block_size; ReadBufferFromFileLogPtr buffer; - bool finished = false; const Block non_virtual_header; const Block virtual_header; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 1eedbee136a..171bc8b0668 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -210,9 +210,6 @@ void ReadBufferFromFileLog::watchFunc(FileLogDirectoryWatcher & dw) case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: file_status[event.path].status = FileStatus::REMOVED; break; - - default: - LOG_INFO(log, "Undefine event type"); } } } diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index 20b6b6d0c2d..090b8581827 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -86,6 +86,6 @@ private: void waitFunc(); - void watchFunc(FileLogDirectoryWatcher & dw); + [[noreturn ]] void watchFunc(FileLogDirectoryWatcher & dw); }; } From 4be47ca7e75e75135225f1b5ada6b6e850f308ce Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 4 Jul 2021 06:16:40 +0000 Subject: [PATCH 004/919] update --- .../FileLog/FileLogBlockInputStream.cpp | 3 --- .../FileLog/FileLogBlockInputStream.h | 4 ++-- .../FileLog/ReadBufferFromFileLog.cpp | 19 +++++++++++++------ src/Storages/FileLog/ReadBufferFromFileLog.h | 13 ++++++++++++- src/Storages/FileLog/StorageFileLog.cpp | 18 +++++++++++++----- 5 files changed, 40 insertions(+), 17 deletions(-) diff --git a/src/Storages/FileLog/FileLogBlockInputStream.cpp b/src/Storages/FileLog/FileLogBlockInputStream.cpp index eb82c891fd6..5c206bfae6d 100644 --- a/src/Storages/FileLog/FileLogBlockInputStream.cpp +++ b/src/Storages/FileLog/FileLogBlockInputStream.cpp @@ -20,13 +20,11 @@ FileLogBlockInputStream::FileLogBlockInputStream( const StorageMetadataPtr & metadata_snapshot_, const std::shared_ptr & context_, const Names & columns, - Poco::Logger * log_, size_t max_block_size_) : storage(storage_) , metadata_snapshot(metadata_snapshot_) , context(context_) , column_names(columns) - , log(log_) , max_block_size(max_block_size_) , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) , virtual_header( @@ -122,7 +120,6 @@ Block FileLogBlockInputStream::readImpl() if (new_rows) { total_rows = total_rows + new_rows; - LOG_INFO(log, "FileLogBlockInputStream, {} rows data polled from buffer.", new_rows); } if (!buffer->hasMorePolledRecords() && (total_rows >= max_block_size || !checkTimeLimit())) diff --git a/src/Storages/FileLog/FileLogBlockInputStream.h b/src/Storages/FileLog/FileLogBlockInputStream.h index 290d3cf07f8..d72ab5649ee 100644 --- a/src/Storages/FileLog/FileLogBlockInputStream.h +++ b/src/Storages/FileLog/FileLogBlockInputStream.h @@ -20,7 +20,6 @@ public: const StorageMetadataPtr & metadata_snapshot_, const std::shared_ptr & context_, const Names & columns, - Poco::Logger * log_, size_t max_block_size_); ~FileLogBlockInputStream() override = default; @@ -31,12 +30,13 @@ public: Block readImpl() override; void readSuffixImpl() override; + bool isStalled() { return !buffer || buffer->isStalled(); } + private: StorageFileLog & storage; StorageMetadataPtr metadata_snapshot; ContextPtr context; Names column_names; - Poco::Logger * log; UInt64 max_block_size; ReadBufferFromFileLogPtr buffer; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 171bc8b0668..1ba111be8f8 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -32,8 +32,6 @@ void ReadBufferFromFileLog::open() { Poco::File file(path); - bool path_is_directory = false; - if (file.isFile()) { file_status[path].reader = std::ifstream(path); @@ -56,8 +54,7 @@ void ReadBufferFromFileLog::open() if (path_is_directory) { - FileLogDirectoryWatcher dw(path); - select_task = context->getMessageBrokerSchedulePool().createTask("watchTask", [this, &dw] { watchFunc(dw); }); + select_task = context->getMessageBrokerSchedulePool().createTask("watchTask", [this] { watchFunc(); }); select_task->activateAndSchedule(); } @@ -75,7 +72,9 @@ void ReadBufferFromFileLog::cleanUnprocessed() void ReadBufferFromFileLog::close() { wait_task->deactivate(); - select_task->deactivate(); + + if (path_is_directory) + select_task->deactivate(); for (auto & status : file_status) status.second.reader.close(); @@ -90,6 +89,8 @@ bool ReadBufferFromFileLog::poll() return true; } + buffer_status = BufferStatus::NO_RECORD_RETURNED; + auto new_records = pollBatch(batch_size); if (new_records.empty()) { @@ -100,7 +101,10 @@ bool ReadBufferFromFileLog::poll() { records = std::move(new_records); current = records.begin(); + LOG_TRACE(log, "Polled batch of {} records. ", records.size()); + + buffer_status = BufferStatus::NOT_STALLED; allowed = true; return true; } @@ -180,8 +184,9 @@ void ReadBufferFromFileLog::waitFunc() time_out = true; } -void ReadBufferFromFileLog::watchFunc(FileLogDirectoryWatcher & dw) +void ReadBufferFromFileLog::watchFunc() { + FileLogDirectoryWatcher dw(path); while (true) { sleepForNanoseconds(poll_timeout); @@ -197,6 +202,8 @@ void ReadBufferFromFileLog::watchFunc(FileLogDirectoryWatcher & dw) { switch (event.type) { + LOG_TRACE(log, "New event {} watched.", event.callback); + case Poco::DirectoryWatcher::DW_ITEM_ADDED: file_status[event.path].reader = std::ifstream(event.path); break; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index 090b8581827..ce14acc9f67 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -33,7 +33,14 @@ public: bool poll(); + bool isStalled() { return buffer_status != BufferStatus::NOT_STALLED; } + private: + enum class BufferStatus + { + NO_RECORD_RETURNED, + NOT_STALLED, + }; enum class FileStatus { BEGIN, @@ -48,8 +55,12 @@ private: std::ifstream reader; }; + BufferStatus buffer_status; + const String path; + bool path_is_directory = false; + Poco::Logger * log; const size_t batch_size = 1; const size_t poll_timeout = 0; @@ -86,6 +97,6 @@ private: void waitFunc(); - [[noreturn ]] void watchFunc(FileLogDirectoryWatcher & dw); + [[noreturn]] void watchFunc(); }; } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 7970feb2996..f6f1535c97e 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -83,8 +83,7 @@ Pipe StorageFileLog::read( auto modified_context = Context::createCopy(local_context); return Pipe(std::make_shared( - std::make_shared(*this, metadata_snapshot, modified_context, column_names, log, 1))); - + std::make_shared(*this, metadata_snapshot, modified_context, column_names, 1))); } void StorageFileLog::startup() @@ -188,7 +187,12 @@ void StorageFileLog::threadFunc() LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); - streamToViews(); + auto stream_is_stalled = streamToViews(); + if (stream_is_stalled) + { + LOG_TRACE(log, "Stream stalled. Reschedule."); + break; + } auto ts = std::chrono::steady_clock::now(); auto duration = std::chrono::duration_cast(ts-start_time); @@ -233,7 +237,7 @@ bool StorageFileLog::streamToViews() auto block_io = interpreter.execute(); auto stream = std::make_shared( - *this, metadata_snapshot, new_context, block_io.out->getHeader().getNames(), log, block_size); + *this, metadata_snapshot, new_context, block_io.out->getHeader().getNames(), block_size); StreamLocalLimits limits; @@ -247,11 +251,15 @@ bool StorageFileLog::streamToViews() copyData( *stream, *block_io.out, [&rows](const Block & block) { rows += block.rows(); }, &stub); + bool stream_is_stalled = false; + + stream_is_stalled = stream->as()->isStalled(); + UInt64 milliseconds = watch.elapsedMilliseconds(); LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); - return true; + return stream_is_stalled; } void registerStorageFileLog(StorageFactory & factory) From f9a76666c9aca03278ec20988c1c23b90bc4f451 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 4 Jul 2021 06:31:41 +0000 Subject: [PATCH 005/919] fix fix fix fix --- .../FileLog/FileLogDirectoryWatcher.cpp | 49 +++++++++---------- .../FileLog/FileLogDirectoryWatcher.h | 21 ++++---- src/Storages/FileLog/StorageFileLog.cpp | 28 +++++------ ...StorageSystemBuildOptions.generated.cpp.in | 1 - 4 files changed, 48 insertions(+), 51 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index e95f7c55c4d..931f9c9aa06 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -34,56 +34,55 @@ void FileLogDirectoryWatcher::onItemAdded(const Poco::DirectoryWatcher::Director { std::lock_guard lock(mutex); DirEvent de; - de.callback = "onItemAdded"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + de.callback = "onItemAdded"; + de.path = ev.item.path(); + de.type = ev.event; + events.emplace_back(de); } void FileLogDirectoryWatcher::onItemRemoved(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); - DirEvent de; - de.callback = "onItemRemoved"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + DirEvent de; + de.callback = "onItemRemoved"; + de.path = ev.item.path(); + de.type = ev.event; + events.emplace_back(de); } void FileLogDirectoryWatcher::onItemModified(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); - DirEvent de; - de.callback = "onItemModified"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + DirEvent de; + de.callback = "onItemModified"; + de.path = ev.item.path(); + de.type = ev.event; + events.emplace_back(de); } void FileLogDirectoryWatcher::onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); DirEvent de; - de.callback = "onItemMovedFrom"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + de.callback = "onItemMovedFrom"; + de.path = ev.item.path(); + de.type = ev.event; + events.emplace_back(de); } void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); - DirEvent de; - de.callback = "onItemMovedTo"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + DirEvent de; + de.callback = "onItemMovedTo"; + de.path = ev.item.path(); + de.type = ev.event; + events.emplace_back(de); } - void FileLogDirectoryWatcher::onError(const Poco::Exception &) { - error = true; + error = true; } diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index b0fde72f57c..de15186a3b8 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -4,20 +4,19 @@ #include #include -#include #include class FileLogDirectoryWatcher { public: struct DirEvent - { - Poco::DirectoryWatcher::DirectoryEventType type; - std::string callback; - std::string path; - }; + { + Poco::DirectoryWatcher::DirectoryEventType type; + std::string callback; + std::string path; + }; - using Events = std::deque; + using Events = std::vector; explicit FileLogDirectoryWatcher(const std::string & path_); ~FileLogDirectoryWatcher() = default; @@ -30,10 +29,10 @@ public: protected: void onItemAdded(const Poco::DirectoryWatcher::DirectoryEvent& ev); - void onItemRemoved(const Poco::DirectoryWatcher::DirectoryEvent& ev); - void onItemModified(const Poco::DirectoryWatcher::DirectoryEvent& ev); - void onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent& ev); - void onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent& ev); + void onItemRemoved(const Poco::DirectoryWatcher::DirectoryEvent & ev); + void onItemModified(const Poco::DirectoryWatcher::DirectoryEvent& ev); + void onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent & ev); + void onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent & ev); void onError(const Poco::Exception &); private: diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index f6f1535c97e..a5e1b8731da 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -88,28 +88,28 @@ Pipe StorageFileLog::read( void StorageFileLog::startup() { - try - { - createReadBuffer(); - } - catch (const Exception &) - { - tryLogCurrentException(log); - } + try + { + createReadBuffer(); + } + catch (const Exception &) + { + tryLogCurrentException(log); + } - task->holder->activateAndSchedule(); + task->holder->activateAndSchedule(); } void StorageFileLog::shutdown() { - task->stream_cancelled = true; + task->stream_cancelled = true; - LOG_TRACE(log, "Waiting for cleanup"); - task->holder->deactivate(); + LOG_TRACE(log, "Waiting for cleanup"); + task->holder->deactivate(); - LOG_TRACE(log, "Closing files"); - destroyReadBuffer(); + LOG_TRACE(log, "Closing files"); + destroyReadBuffer(); } size_t StorageFileLog::getMaxBlockSize() const diff --git a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in index 18cab0700ec..bb25342bd87 100644 --- a/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in +++ b/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in @@ -50,7 +50,6 @@ const char * auto_config_build[] "USE_LDAP", "@USE_LDAP@", "TZDATA_VERSION", "@TZDATA_VERSION@", "USE_KRB5", "@USE_KRB5@", - "USE_KRB5", "@USE_KRB5@", "USE_FILELOG", "@USE_FILELOG@", nullptr, nullptr From fdc256e30dab3a5817764d9a4a96580ecc2f1182 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 4 Jul 2021 08:52:05 +0000 Subject: [PATCH 006/919] fix build --- src/Storages/FileLog/StorageFileLog.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index dda759c7e38..93b67fe5a31 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -8,7 +8,7 @@ #include #include -#include +#include #include #include @@ -16,9 +16,9 @@ namespace DB { -class StorageFileLog final : public ext::shared_ptr_helper, public IStorage, WithContext +class StorageFileLog final : public shared_ptr_helper, public IStorage, WithContext { - friend struct ext::shared_ptr_helper; + friend struct shared_ptr_helper; public: using Files = std::vector; From 85aa71c377a783f115852bca7827460ec3328e27 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 11 Jul 2021 03:37:48 +0000 Subject: [PATCH 007/919] fix build --- src/Storages/FileLog/ReadBufferFromFileLog.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 1ba111be8f8..47985e32cb4 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -202,19 +202,21 @@ void ReadBufferFromFileLog::watchFunc() { switch (event.type) { - LOG_TRACE(log, "New event {} watched.", event.callback); case Poco::DirectoryWatcher::DW_ITEM_ADDED: + LOG_TRACE(log, "New event {} watched.", event.callback); file_status[event.path].reader = std::ifstream(event.path); break; case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: + LOG_TRACE(log, "New event {} watched.", event.callback); file_status[event.path].status = FileStatus::UPDATED; break; case Poco::DirectoryWatcher::DW_ITEM_REMOVED: case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: + LOG_TRACE(log, "New event {} watched.", event.callback); file_status[event.path].status = FileStatus::REMOVED; break; } From 8dbb2a3ed9ad0db79ce6262593893cf68dc8c5e1 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 11 Jul 2021 06:18:03 +0000 Subject: [PATCH 008/919] fix --- src/Storages/FileLog/FileLogDirectoryWatcher.h | 1 + src/Storages/FileLog/ReadBufferFromFileLog.cpp | 1 - src/Storages/FileLog/ReadBufferFromFileLog.h | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 5 ++--- 4 files changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index de15186a3b8..99fd5cf0af1 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -4,6 +4,7 @@ #include #include +#include #include class FileLogDirectoryWatcher diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 47985e32cb4..e5849b50894 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -14,7 +14,6 @@ namespace DB namespace ErrorCodes { - extern const int CANNOT_COMMIT_OFFSET; } ReadBufferFromFileLog::ReadBufferFromFileLog( diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index ce14acc9f67..e74c93d189e 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -25,7 +25,7 @@ public: ~ReadBufferFromFileLog() override = default; void open(); - void close(); + void close(); auto pollTimeout() const { return poll_timeout; } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index a5e1b8731da..565b206e2cc 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -38,9 +38,7 @@ namespace DB namespace ErrorCodes { - extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; - extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -270,7 +268,8 @@ void registerStorageFileLog(StorageFactory & factory) size_t args_count = engine_args.size(); if (args_count != 2) - throw Exception("Arguments size of StorageFileLog should be 2, path and format name", ErrorCodes::BAD_ARGUMENTS); + throw Exception( + "Arguments size of StorageFileLog should be 2, path and format name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); auto path_ast = evaluateConstantExpressionAsLiteral(engine_args[0], args.getContext()); auto format_ast = evaluateConstantExpressionAsLiteral(engine_args[1], args.getContext()); From c8fa36e3d67e95241e8cf3c5b5a9af2b038c89e6 Mon Sep 17 00:00:00 2001 From: adevyatova Date: Wed, 18 Aug 2021 15:16:37 +0000 Subject: [PATCH 009/919] Added the description of secured communication with Zookeeper --- .../settings.md | 1 + docs/en/operations/ssl-zookeeper.md | 89 +++++++++++++++++++ 2 files changed, 90 insertions(+) create mode 100644 docs/en/operations/ssl-zookeeper.md diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index a620565b71a..ec84bce1582 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1185,6 +1185,7 @@ This section contains the following parameters: - [Replication](../../engines/table-engines/mergetree-family/replication.md) - [ZooKeeper Programmer’s Guide](http://zookeeper.apache.org/doc/current/zookeeperProgrammers.html) +- [Optional secured communication between ClickHouse and Zookeeper](../ssl-zookeeper.md#secured-communication-with-zookeeper) ## use_minimalistic_part_header_in_zookeeper {#server-settings-use_minimalistic_part_header_in_zookeeper} diff --git a/docs/en/operations/ssl-zookeeper.md b/docs/en/operations/ssl-zookeeper.md new file mode 100644 index 00000000000..37230531ccd --- /dev/null +++ b/docs/en/operations/ssl-zookeeper.md @@ -0,0 +1,89 @@ +--- +toc_priority: 45 +toc_title: Secured communication with Zookeeper +--- + +# Optional secured communication between ClickHouse and Zookeeper {#secured-communication-with-zookeeper} + +You should specified `ssl.keyStore.location`, `ssl.keyStore.password` and `ssl.trustStore.location`, `ssl.trustStore.password` for communication with ClickHouse client over SSL. These options are available from Zookeeper version 3.5.2. + +You have two options how to handle custom certificate verification on the ClickHouse-side: + +1. Turn off certificate verification. + +Client section in `config.xml` will look like: + + + /etc/clickhouse-server/client.crt + /etc/clickhouse-server/client.key + true + true + sslv2,sslv3 + true + none + + RejectCertificateHandler + + + +2. Add `zookeeper.crt` to trusted certificates: + +```bash +sudo cp zookeeper.crt /usr/local/share/ca-certificates/zookeeper.crt +sudo update-ca-certificates +``` + +Client section in `config.xml` will look like: + + + /etc/clickhouse-server/client.crt + /etc/clickhouse-server/client.key + true + true + sslv2,sslv3 + true + + RejectCertificateHandler + + + +Add Zookeeper to ClickHouse config with some cluster and macros: + + + + + localhost + 2281 + 1 + + + + +Start `clickhouse-server`. In logs you should see: + +```text + ZooKeeper: initialized, hosts: secure://localhost:2281 +``` + +Prefix `secure://` indicates that connection is secured by SSL. + +To ensure traffic is encrypted run `tcpdump` on secured port: + +```text +tcpdump -i any dst port 2281 -nnXS +``` + +And query in `clickhouse-client`: + +```sql +SELECT * FROM system.zookeeper WHERE path = '/'; +``` + +On unencrypted connection you will see in `tcpdump` output something like this: + +```text +..../zookeeper/q +uota. +``` + +On encrypted connection you should not see this. \ No newline at end of file From 69730ff0f2a00ac091cb5335f81b91f371658845 Mon Sep 17 00:00:00 2001 From: adevyatova Date: Wed, 18 Aug 2021 17:31:27 +0000 Subject: [PATCH 010/919] Small fixes --- docs/en/operations/ssl-zookeeper.md | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/ssl-zookeeper.md b/docs/en/operations/ssl-zookeeper.md index 37230531ccd..7124240e231 100644 --- a/docs/en/operations/ssl-zookeeper.md +++ b/docs/en/operations/ssl-zookeeper.md @@ -13,6 +13,7 @@ You have two options how to handle custom certificate verification on the ClickH Client section in `config.xml` will look like: +``` xml /etc/clickhouse-server/client.crt /etc/clickhouse-server/client.key @@ -25,16 +26,18 @@ Client section in `config.xml` will look like: RejectCertificateHandler +``` -2. Add `zookeeper.crt` to trusted certificates: +2. Add `zookeeper.crt` to trusted certificates. -```bash +``` bash sudo cp zookeeper.crt /usr/local/share/ca-certificates/zookeeper.crt sudo update-ca-certificates ``` Client section in `config.xml` will look like: +``` xml /etc/clickhouse-server/client.crt /etc/clickhouse-server/client.key @@ -46,9 +49,11 @@ Client section in `config.xml` will look like: RejectCertificateHandler +``` Add Zookeeper to ClickHouse config with some cluster and macros: +``` xml @@ -58,6 +63,7 @@ Add Zookeeper to ClickHouse config with some cluster and macros: +``` Start `clickhouse-server`. In logs you should see: @@ -69,7 +75,7 @@ Prefix `secure://` indicates that connection is secured by SSL. To ensure traffic is encrypted run `tcpdump` on secured port: -```text +```bash tcpdump -i any dst port 2281 -nnXS ``` From 595005eb211b60b548819963232de325312dd893 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 4 Sep 2021 17:04:35 +0000 Subject: [PATCH 011/919] refactor some code --- src/Storages/FileLog/FileLogSettings.cpp | 41 +++ src/Storages/FileLog/FileLogSettings.h | 36 +++ ...BlockInputStream.cpp => FileLogSource.cpp} | 56 ++-- ...eLogBlockInputStream.h => FileLogSource.h} | 33 +- .../FileLog/ReadBufferFromFileLog.cpp | 69 +---- src/Storages/FileLog/ReadBufferFromFileLog.h | 23 +- src/Storages/FileLog/StorageFileLog.cpp | 281 +++++++++++++----- src/Storages/FileLog/StorageFileLog.h | 39 ++- 8 files changed, 366 insertions(+), 212 deletions(-) create mode 100644 src/Storages/FileLog/FileLogSettings.cpp create mode 100644 src/Storages/FileLog/FileLogSettings.h rename src/Storages/FileLog/{FileLogBlockInputStream.cpp => FileLogSource.cpp} (73%) rename src/Storages/FileLog/{FileLogBlockInputStream.h => FileLogSource.h} (57%) diff --git a/src/Storages/FileLog/FileLogSettings.cpp b/src/Storages/FileLog/FileLogSettings.cpp new file mode 100644 index 00000000000..2cd42c35870 --- /dev/null +++ b/src/Storages/FileLog/FileLogSettings.cpp @@ -0,0 +1,41 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} + +IMPLEMENT_SETTINGS_TRAITS(FileLogSettingsTraits, LIST_OF_FILELOG_SETTINGS) + +void FileLogSettings::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/FileLog/FileLogSettings.h b/src/Storages/FileLog/FileLogSettings.h new file mode 100644 index 00000000000..cfea9d1e195 --- /dev/null +++ b/src/Storages/FileLog/FileLogSettings.h @@ -0,0 +1,36 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class ASTStorage; + + +#define FILELOG_RELATED_SETTINGS(M) \ + /* default is stream_poll_timeout_ms */ \ + M(Milliseconds, filelog_poll_timeout_ms, 0, "Timeout for single poll from FileLog.", 0) \ + /* default is min(max_block_size, kafka_max_block_size)*/ \ + M(UInt64, filelog_poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single Kafka poll.", 0) \ + /* default is = max_insert_block_size / kafka_num_consumers */ \ + M(UInt64, filelog_max_block_size, 0, "Number of row collected by poll(s) for flushing data from Kafka.", 0) \ + M(UInt64, filelog_max_threads, 8, "Number of max threads to parse files, default is 8", 0) + +#define LIST_OF_FILELOG_SETTINGS(M) \ + FILELOG_RELATED_SETTINGS(M) \ + FORMAT_FACTORY_SETTINGS(M) + +DECLARE_SETTINGS_TRAITS(FileLogSettingsTraits, LIST_OF_FILELOG_SETTINGS) + + +/** Settings for the Kafka engine. + * Could be loaded from a CREATE TABLE query (SETTINGS clause). + */ +struct FileLogSettings : public BaseSettings +{ + void loadFromQuery(ASTStorage & storage_def); +}; + +} diff --git a/src/Storages/FileLog/FileLogBlockInputStream.cpp b/src/Storages/FileLog/FileLogSource.cpp similarity index 73% rename from src/Storages/FileLog/FileLogBlockInputStream.cpp rename to src/Storages/FileLog/FileLogSource.cpp index 5c206bfae6d..4a15cd07f25 100644 --- a/src/Storages/FileLog/FileLogBlockInputStream.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -1,11 +1,11 @@ -#include - #include #include #include #include #include +#include #include +#include #include namespace DB @@ -15,42 +15,34 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -FileLogBlockInputStream::FileLogBlockInputStream( +FileLogSource::FileLogSource( StorageFileLog & storage_, const StorageMetadataPtr & metadata_snapshot_, - const std::shared_ptr & context_, + const ContextPtr & context_, const Names & columns, - size_t max_block_size_) + size_t max_block_size_, + size_t poll_time_out_, + size_t stream_number_, + size_t max_streams_number_) : storage(storage_) , metadata_snapshot(metadata_snapshot_) , context(context_) , column_names(columns) , max_block_size(max_block_size_) + , poll_time_out(poll_time_out_) + , stream_number(stream_number_) + , max_streams_number(max_streams_number_) , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) , virtual_header( metadata_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames(), storage.getVirtuals(), storage.getStorageID())) { + createReadBuffer(); } -Block FileLogBlockInputStream::getHeader() const -{ - return metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID()); -} - -void FileLogBlockInputStream::readPrefixImpl() -{ - buffer = storage.getBuffer(); - - if (!buffer) - return; - - buffer->open(); -} - -Block FileLogBlockInputStream::readImpl() +Chunk FileLogSource::generate() { if (!buffer) - return Block(); + return {}; MutableColumns result_columns = non_virtual_header.cloneEmptyColumns(); @@ -104,8 +96,8 @@ Block FileLogBlockInputStream::readImpl() while (true) { + Stopwatch watch; size_t new_rows = 0; - exception_message.reset(); if (buffer->poll()) { try @@ -122,28 +114,16 @@ Block FileLogBlockInputStream::readImpl() total_rows = total_rows + new_rows; } - if (!buffer->hasMorePolledRecords() && (total_rows >= max_block_size || !checkTimeLimit())) + if ((!buffer->hasMorePolledRecords() && (total_rows >= max_block_size)) || watch.elapsedMilliseconds() > poll_time_out) { break; } } if (total_rows == 0) - return Block(); + return {}; - auto result_block = non_virtual_header.cloneWithColumns(std::move(result_columns)); - - return ConvertingBlockInputStream( - std::make_shared(result_block), - getHeader(), - ConvertingBlockInputStream::MatchColumnsMode::Name) - .read(); -} - -void FileLogBlockInputStream::readSuffixImpl() -{ - if (buffer) - buffer->close(); + return Chunk(std::move(result_columns), total_rows); } } diff --git a/src/Storages/FileLog/FileLogBlockInputStream.h b/src/Storages/FileLog/FileLogSource.h similarity index 57% rename from src/Storages/FileLog/FileLogBlockInputStream.h rename to src/Storages/FileLog/FileLogSource.h index d72ab5649ee..042a973986c 100644 --- a/src/Storages/FileLog/FileLogBlockInputStream.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -12,26 +12,24 @@ namespace Poco } namespace DB { -class FileLogBlockInputStream : public IBlockInputStream +class FileLogSource : public SourceWithProgress { public: - FileLogBlockInputStream( + FileLogSource( StorageFileLog & storage_, const StorageMetadataPtr & metadata_snapshot_, - const std::shared_ptr & context_, + const ContextPtr & context_, const Names & columns, - size_t max_block_size_); - ~FileLogBlockInputStream() override = default; - - String getName() const override { return storage.getName(); } - Block getHeader() const override; - - void readPrefixImpl() override; - Block readImpl() override; - void readSuffixImpl() override; + size_t max_block_size_, + size_t poll_time_out_, + size_t stream_number_, + size_t max_streams_number_); bool isStalled() { return !buffer || buffer->isStalled(); } +protected: + Chunk generate() override; + private: StorageFileLog & storage; StorageMetadataPtr metadata_snapshot; @@ -39,10 +37,19 @@ private: Names column_names; UInt64 max_block_size; + size_t poll_time_out; + + size_t stream_number; + size_t max_streams_number; + ReadBufferFromFileLogPtr buffer; + bool started = false; + const Block non_virtual_header; const Block virtual_header; + + void createReadBuffer(); }; } diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index e5849b50894..1c39296dabb 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -1,13 +1,11 @@ #include #include -#include -#include #include -#include -#include #include +#include +#include namespace DB { @@ -29,33 +27,9 @@ ReadBufferFromFileLog::ReadBufferFromFileLog( void ReadBufferFromFileLog::open() { - Poco::File file(path); - - if (file.isFile()) - { - file_status[path].reader = std::ifstream(path); - } - else if (file.isDirectory()) - { - path_is_directory = true; - Poco::DirectoryIterator dir_iter(file); - Poco::DirectoryIterator end; - while (dir_iter != end) - { - if (dir_iter->isFile()) - file_status[dir_iter->path()].reader = std::ifstream(dir_iter->path()); - ++dir_iter; - } - } - wait_task = context->getMessageBrokerSchedulePool().createTask("waitTask", [this] { waitFunc(); }); wait_task->deactivate(); - if (path_is_directory) - { - select_task = context->getMessageBrokerSchedulePool().createTask("watchTask", [this] { watchFunc(); }); - select_task->activateAndSchedule(); - } cleanUnprocessed(); allowed = false; @@ -183,43 +157,4 @@ void ReadBufferFromFileLog::waitFunc() time_out = true; } -void ReadBufferFromFileLog::watchFunc() -{ - FileLogDirectoryWatcher dw(path); - while (true) - { - sleepForNanoseconds(poll_timeout); - - auto error = dw.getError(); - if (error) - LOG_INFO(log, "Error happened during watching directory {}.", dw.getPath()); - - auto events = dw.getEvents(); - std::lock_guard lock(status_mutex); - - for (const auto & event : events) - { - switch (event.type) - { - - case Poco::DirectoryWatcher::DW_ITEM_ADDED: - LOG_TRACE(log, "New event {} watched.", event.callback); - file_status[event.path].reader = std::ifstream(event.path); - break; - - case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: - LOG_TRACE(log, "New event {} watched.", event.callback); - file_status[event.path].status = FileStatus::UPDATED; - break; - - case Poco::DirectoryWatcher::DW_ITEM_REMOVED: - case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: - case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: - LOG_TRACE(log, "New event {} watched.", event.callback); - file_status[event.path].status = FileStatus::REMOVED; - break; - } - } - } -} } diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index e74c93d189e..e7385282f71 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -33,26 +33,13 @@ public: bool poll(); - bool isStalled() { return buffer_status != BufferStatus::NOT_STALLED; } + bool noRecords() { return buffer_status == BufferStatus::NO_RECORD_RETURNED; } private: enum class BufferStatus { NO_RECORD_RETURNED, - NOT_STALLED, - }; - enum class FileStatus - { - BEGIN, - NO_CHANGE, - UPDATED, - REMOVED - }; - - struct FileContext - { - FileStatus status = FileStatus::BEGIN; - std::ifstream reader; + POLLED_OK, }; BufferStatus buffer_status; @@ -67,10 +54,6 @@ private: bool time_out = false; - using NameToFile = std::unordered_map; - NameToFile file_status; - - std::mutex status_mutex; ContextPtr context; @@ -85,7 +68,6 @@ private: using TaskThread = BackgroundSchedulePool::TaskHolder; TaskThread wait_task; - TaskThread select_task; Records pollBatch(size_t batch_size_); @@ -97,6 +79,5 @@ private: void waitFunc(); - [[noreturn]] void watchFunc(); }; } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 565b206e2cc..a6f819580d5 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -1,16 +1,3 @@ -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include #include @@ -18,20 +5,25 @@ #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 { @@ -53,19 +45,41 @@ StorageFileLog::StorageFileLog( ContextPtr context_, const ColumnsDescription & columns_, const String & path_, - const String & format_name_) + const String & format_name_, + std::unique_ptr settings) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) + , filelog_settings(std::move(settings)) , path(path_) , format_name(format_name_) - , log(&Poco::Logger::get("StorageFile (" + table_id_.table_name + ")")) + , log(&Poco::Logger::get("StorageFileLog (" + table_id_.table_name + ")")) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); + if (std::filesystem::is_regular_file(path)) + { + file_status[path].reader = std::ifstream(path); + file_names.push_back(path); + } + else if (std::filesystem::is_directory(path)) + { + path_is_directory = true; + /// Just consider file with depth 1 + for (const auto & dir_entry : std::filesystem::directory_iterator{path}) + { + if (dir_entry.is_regular_file()) + { + file_status[dir_entry.path()].reader = std::ifstream(dir_entry.path()); + file_names.push_back(dir_entry.path()); + } + } + } + + watch_task = getContext()->getMessageBrokerSchedulePool().createTask("watchTask", [this] { watchFunc(); }); + auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); - thread->deactivate(); task = std::make_shared(std::move(thread)); } @@ -78,23 +92,38 @@ Pipe StorageFileLog::read( size_t /* max_block_size */, unsigned /* num_streams */) { + std::lock_guard lock(status_mutex); auto modified_context = Context::createCopy(local_context); - return Pipe(std::make_shared( - std::make_shared(*this, metadata_snapshot, modified_context, column_names, 1))); + clearInvalidFiles(); + + auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_names.size()); + /// No files to parse + if (max_streams_number == 0) + { + return Pipe{}; + } + + Pipes pipes; + pipes.reserve(max_streams_number); + for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) + { + pipes.emplace_back(std::make_shared( + *this, + metadata_snapshot, + modified_context, + column_names, + getMaxBlockSize(), + getPollTimeoutMillisecond(), + stream_number, + max_streams_number)); + } + + return Pipe::unitePipes(std::move(pipes)); } void StorageFileLog::startup() { - try - { - createReadBuffer(); - } - catch (const Exception &) - { - tryLogCurrentException(log); - } - task->holder->activateAndSchedule(); } @@ -105,26 +134,32 @@ void StorageFileLog::shutdown() LOG_TRACE(log, "Waiting for cleanup"); task->holder->deactivate(); + watch_task->deactivate(); - LOG_TRACE(log, "Closing files"); - destroyReadBuffer(); + for (auto & file : file_status) + { + file.second.reader.close(); + } } size_t StorageFileLog::getMaxBlockSize() const { - return getContext()->getSettingsRef().max_insert_block_size.value; + return filelog_settings->filelog_max_block_size.changed ? filelog_settings->filelog_max_block_size.value + : getContext()->getSettingsRef().max_insert_block_size.value; } size_t StorageFileLog::getPollMaxBatchSize() const { - size_t batch_size = getContext()->getSettingsRef().max_block_size.value; + size_t batch_size = filelog_settings->filelog_poll_max_batch_size.changed ? filelog_settings->filelog_poll_max_batch_size.value + : getContext()->getSettingsRef().max_block_size.value; - return std::min(batch_size,getMaxBlockSize()); + return std::min(batch_size, getMaxBlockSize()); } size_t StorageFileLog::getPollTimeoutMillisecond() const { - return getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); + return filelog_settings->filelog_poll_timeout_ms.changed ? filelog_settings->filelog_poll_timeout_ms.totalMilliseconds() + : getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); } @@ -135,9 +170,9 @@ bool StorageFileLog::checkDependencies(const StorageID & table_id) if (dependencies.empty()) return true; - for (const auto & db_tab : dependencies) + for (const auto & storage : dependencies) { - auto table = DatabaseCatalog::instance().tryGetTable(db_tab, getContext()); + auto table = DatabaseCatalog::instance().tryGetTable(storage, getContext()); if (!table) return false; @@ -147,25 +182,13 @@ bool StorageFileLog::checkDependencies(const StorageID & table_id) return false; // Check all its dependencies - if (!checkDependencies(db_tab)) + if (!checkDependencies(storage)) return false; } return true; } -void StorageFileLog::createReadBuffer() -{ - auto new_context = Context::createCopy(getContext()); - buffer = std::make_shared(path, log, getPollMaxBatchSize(), getPollTimeoutMillisecond(), new_context); -} - -void StorageFileLog::destroyReadBuffer() -{ - if (buffer) - buffer->close(); -} - void StorageFileLog::threadFunc() { try @@ -177,6 +200,8 @@ void StorageFileLog::threadFunc() { auto start_time = std::chrono::steady_clock::now(); + watch_task->activateAndSchedule(); + // Keep streaming as long as there are attached views and streaming is not cancelled while (!task->stream_cancelled) { @@ -207,6 +232,7 @@ void StorageFileLog::threadFunc() tryLogCurrentException(__PRETTY_FUNCTION__); } + watch_task->deactivate(); // Wait for attached views if (!task->stream_cancelled) task->holder->scheduleAfter(RESCHEDULE_MS); @@ -215,6 +241,7 @@ void StorageFileLog::threadFunc() bool StorageFileLog::streamToViews() { + std::lock_guard lock(status_mutex); Stopwatch watch; auto table_id = getStorageID(); @@ -223,41 +250,79 @@ bool StorageFileLog::streamToViews() throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); auto metadata_snapshot = getInMemoryMetadataPtr(); + clearInvalidFiles(); + + auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_names.size()); + /// No files to parse + if (max_streams_number == 0) + { + return false; + } + // Create an INSERT query for streaming data auto insert = std::make_shared(); insert->table_id = table_id; - size_t block_size = getMaxBlockSize(); - auto new_context = Context::createCopy(getContext()); InterpreterInsertQuery interpreter(insert, new_context, false, true, true); auto block_io = interpreter.execute(); - auto stream = std::make_shared( - *this, metadata_snapshot, new_context, block_io.out->getHeader().getNames(), block_size); + Pipes pipes; + pipes.reserve(max_streams_number); + for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) + { + pipes.emplace_back(std::make_shared( + *this, + metadata_snapshot, + new_context, + block_io.out->getHeader().getNames(), + getPollMaxBatchSize(), + getPollTimeoutMillisecond(), + stream_number, + max_streams_number)); + } - StreamLocalLimits limits; + QueryPipeline pipeline; + pipeline.init(Pipe::unitePipes(std::move(pipes))); - limits.speed_limits.max_execution_time = getContext()->getSettingsRef().stream_flush_interval_ms; - - limits.timeout_overflow_mode = OverflowMode::BREAK; - stream->setLimits(limits); - - std::atomic stub = {false}; size_t rows = 0; - copyData( - *stream, *block_io.out, [&rows](const Block & block) { rows += block.rows(); }, &stub); - bool stream_is_stalled = false; - - stream_is_stalled = stream->as()->isStalled(); + PullingPipelineExecutor executor(pipeline); + Block block; + block_io.out->writePrefix(); + while (executor.pull(block)) + { + block_io.out->write(block); + rows += block.rows(); + } + block_io.out->writeSuffix(); UInt64 milliseconds = watch.elapsedMilliseconds(); LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); - return stream_is_stalled; + return true; +} + +void StorageFileLog::clearInvalidFiles() +{ + /// Do not need to hold file_status lock, since it will be holded + /// by caller when call this function + std::vector valid_files; + for (const auto & it : file_names) + { + if (file_status.at(it).status == FileStatus::REMOVED) + { + file_status.erase(it); + } + else + { + valid_files.push_back(it); + } + } + + file_names.swap(valid_files); } void registerStorageFileLog(StorageFactory & factory) @@ -267,6 +332,36 @@ void registerStorageFileLog(StorageFactory & factory) ASTs & engine_args = args.engine_args; size_t args_count = engine_args.size(); + bool has_settings = args.storage_def->settings; + + auto filelog_settings = std::make_unique(); + if (has_settings) + { + filelog_settings->loadFromQuery(*args.storage_def); + } + + auto physical_cpu_cores = getNumberOfPhysicalCPUCores(); + auto num_threads = filelog_settings->filelog_max_threads.value; + + if (num_threads > physical_cpu_cores) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Number of threads to parse files can not be bigger than {}", physical_cpu_cores); + } + else if (num_threads < 1) + { + throw Exception("Number of threads to parse files can not be lower than 1", ErrorCodes::BAD_ARGUMENTS); + } + + if (filelog_settings->filelog_max_block_size.changed && filelog_settings->filelog_max_block_size.value < 1) + { + throw Exception("filelog_max_block_size can not be lower than 1", ErrorCodes::BAD_ARGUMENTS); + } + + if (filelog_settings->filelog_poll_max_batch_size.changed && filelog_settings->filelog_poll_max_batch_size.value < 1) + { + throw Exception("filelog_poll_max_batch_size can not be lower than 1", ErrorCodes::BAD_ARGUMENTS); + } + if (args_count != 2) throw Exception( "Arguments size of StorageFileLog should be 2, path and format name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -277,7 +372,7 @@ void registerStorageFileLog(StorageFactory & factory) auto path = path_ast->as().value.safeGet(); auto format = format_ast->as().value.safeGet(); - return StorageFileLog::create(args.table_id, args.getContext(), args.columns, path, format); + return StorageFileLog::create(args.table_id, args.getContext(), args.columns, path, format, std::move(filelog_settings)); }; factory.registerStorage( @@ -300,4 +395,54 @@ Names StorageFileLog::getVirtualColumnNames() return result; } +void StorageFileLog::watchFunc() +{ + FileLogDirectoryWatcher dw(path); + while (true) + { + sleepForMicroseconds(filelog_settings->filelog_poll_timeout_ms.totalMilliseconds()); + + auto error = dw.getError(); + if (error) + LOG_INFO(log, "Error happened during watching directory {}.", dw.getPath()); + + auto events = dw.getEvents(); + std::lock_guard lock(status_mutex); + + for (const auto & event : events) + { + switch (event.type) + { + + case Poco::DirectoryWatcher::DW_ITEM_ADDED: + LOG_TRACE(log, "New event {} watched.", event.callback); + if (std::filesystem::is_regular_file(event.path)) + { + file_status[event.path].reader = std::ifstream(event.path); + file_names.push_back(event.path); + } + break; + + case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: + LOG_TRACE(log, "New event {} watched.", event.callback); + if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) + { + file_status[event.path].status = FileStatus::UPDATED; + } + break; + + case Poco::DirectoryWatcher::DW_ITEM_REMOVED: + case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: + case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: + LOG_TRACE(log, "New event {} watched.", event.callback); + if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) + { + file_status[event.path].status = FileStatus::REMOVED; + } + break; + } + } + } +} + } diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 93b67fe5a31..314724d73be 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -11,8 +12,8 @@ #include #include -#include #include +#include namespace DB { @@ -52,16 +53,39 @@ protected: ContextPtr context_, const ColumnsDescription & columns_, const String & path_, - const String & format_name_); + const String & format_name_, + std::unique_ptr settings); private: + std::unique_ptr filelog_settings; const String path; + bool path_is_directory = false; + const String format_name; Poco::Logger * log; ReadBufferFromFileLogPtr buffer; - std::mutex mutex; + enum class FileStatus + { + BEGIN, + NO_CHANGE, + UPDATED, + REMOVED + }; + + struct FileContext + { + FileStatus status = FileStatus::BEGIN; + std::ifstream reader; + }; + + using NameToFile = std::unordered_map; + NameToFile file_status; + + std::vector file_names; + + std::mutex status_mutex; // Stream thread struct TaskContext @@ -74,17 +98,22 @@ private: }; std::shared_ptr task; - void createReadBuffer(); - void destroyReadBuffer(); + using TaskThread = BackgroundSchedulePool::TaskHolder; + + TaskThread watch_task; void threadFunc(); + void clearInvalidFiles(); + size_t getPollMaxBatchSize() const; size_t getMaxBlockSize() const; size_t getPollTimeoutMillisecond() const; bool streamToViews(); bool checkDependencies(const StorageID & table_id); + + [[noreturn]] void watchFunc(); }; } From 09bc3d723af90e2ae412e1728b775ee0a99a0333 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 5 Sep 2021 06:32:32 +0000 Subject: [PATCH 012/919] refactor some code --- src/Storages/FileLog/FileLogSource.cpp | 9 +-- src/Storages/FileLog/FileLogSource.h | 11 +-- .../FileLog/ReadBufferFromFileLog.cpp | 73 ++++++++----------- src/Storages/FileLog/ReadBufferFromFileLog.h | 32 ++++---- src/Storages/FileLog/StorageFileLog.cpp | 10 +-- src/Storages/FileLog/StorageFileLog.h | 22 +++--- 6 files changed, 64 insertions(+), 93 deletions(-) diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index 4a15cd07f25..4827a048d6e 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -24,19 +24,18 @@ FileLogSource::FileLogSource( size_t poll_time_out_, size_t stream_number_, size_t max_streams_number_) - : storage(storage_) + : SourceWithProgress(metadata_snapshot->getSampleBlockForColumns(columns, storage_.getVirtuals(), storage_.getStorageID())) + , storage(storage_) , metadata_snapshot(metadata_snapshot_) , context(context_) , column_names(columns) , max_block_size(max_block_size_) , poll_time_out(poll_time_out_) - , stream_number(stream_number_) - , max_streams_number(max_streams_number_) , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) , virtual_header( metadata_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames(), storage.getVirtuals(), storage.getStorageID())) { - createReadBuffer(); + buffer = std::make_unique(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_); } Chunk FileLogSource::generate() @@ -114,7 +113,7 @@ Chunk FileLogSource::generate() total_rows = total_rows + new_rows; } - if ((!buffer->hasMorePolledRecords() && (total_rows >= max_block_size)) || watch.elapsedMilliseconds() > poll_time_out) + if (!buffer->hasMorePolledRecords() && ((total_rows >= max_block_size) || watch.elapsedMilliseconds() > poll_time_out)) { break; } diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index 042a973986c..7db2602dc8c 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -25,7 +25,9 @@ public: size_t stream_number_, size_t max_streams_number_); - bool isStalled() { return !buffer || buffer->isStalled(); } + String getName() const override { return "FileLog"; } + + bool noRecords() { return !buffer || buffer->noRecords(); } protected: Chunk generate() override; @@ -39,17 +41,12 @@ private: size_t poll_time_out; - size_t stream_number; - size_t max_streams_number; - - ReadBufferFromFileLogPtr buffer; + std::unique_ptr buffer; bool started = false; const Block non_virtual_header; const Block virtual_header; - - void createReadBuffer(); }; } diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 1c39296dabb..4df13199901 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -1,5 +1,6 @@ #include #include +#include #include @@ -15,22 +16,21 @@ namespace ErrorCodes } ReadBufferFromFileLog::ReadBufferFromFileLog( - const String & path_, Poco::Logger * log_, size_t max_batch_size, size_t poll_timeout_, ContextPtr context_) + StorageFileLog & storage_, + size_t max_batch_size, + size_t poll_timeout_, + ContextPtr context_, + size_t stream_number_, + size_t max_streams_number_) : ReadBuffer(nullptr, 0) - , path(path_) - , log(log_) + , log(&Poco::Logger::get("ReadBufferFromFileLog " + toString(stream_number))) + , storage(storage_) , batch_size(max_batch_size) , poll_timeout(poll_timeout_) , context(context_) + , stream_number(stream_number_) + , max_streams_number(max_streams_number_) { -} - -void ReadBufferFromFileLog::open() -{ - wait_task = context->getMessageBrokerSchedulePool().createTask("waitTask", [this] { waitFunc(); }); - wait_task->deactivate(); - - cleanUnprocessed(); allowed = false; } @@ -42,17 +42,6 @@ void ReadBufferFromFileLog::cleanUnprocessed() BufferBase::set(nullptr, 0, 0); } -void ReadBufferFromFileLog::close() -{ - wait_task->deactivate(); - - if (path_is_directory) - select_task->deactivate(); - - for (auto & status : file_status) - status.second.reader.close(); -} - bool ReadBufferFromFileLog::poll() { @@ -77,7 +66,7 @@ bool ReadBufferFromFileLog::poll() LOG_TRACE(log, "Polled batch of {} records. ", records.size()); - buffer_status = BufferStatus::NOT_STALLED; + buffer_status = BufferStatus::POLLED_OK; allowed = true; return true; } @@ -94,43 +83,45 @@ ReadBufferFromFileLog::Records ReadBufferFromFileLog::pollBatch(size_t batch_siz if (new_records.size() == batch_size_) return new_records; - wait_task->activateAndSchedule(); - while (!time_out && new_records.size() != batch_size_) + Stopwatch watch; + while (watch.elapsedMilliseconds() < poll_timeout && new_records.size() != batch_size_) { readNewRecords(new_records, batch_size); } - wait_task->deactivate(); - time_out = false; return new_records; } +// TODO void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_records, size_t batch_size_) { - std::lock_guard lock(status_mutex); - size_t need_records_size = batch_size_ - new_records.size(); size_t read_records_size = 0; - for (auto & status : file_status) + const auto & file_names = storage.getFileNames(); + auto & file_status = storage.getFileStatus(); + + size_t files_per_stream = file_names.size() / max_streams_number; + size_t start = stream_number * files_per_stream; + size_t end = stream_number == max_streams_number - 1 ? file_names.size() : (stream_number + 1) * files_per_stream; + + for (size_t i = start; i < end; ++i) { - if (status.second.status == FileStatus::NO_CHANGE) + auto & file = file_status[file_names[i]]; + if (file.status == StorageFileLog::FileStatus::NO_CHANGE || file.status == StorageFileLog::FileStatus::REMOVED) continue; - if (status.second.status == FileStatus::REMOVED) - file_status.erase(status.first); - - while (read_records_size < need_records_size && status.second.reader.good() && !status.second.reader.eof()) + while (read_records_size < need_records_size && file.reader.good() && !file.reader.eof()) { Record record; - std::getline(status.second.reader, record); + std::getline(file.reader, record); new_records.emplace_back(record); ++read_records_size; } // Read to the end of the file - if (status.second.reader.eof()) - status.second.status = FileStatus::NO_CHANGE; + if (file.reader.eof()) + file.status = StorageFileLog::FileStatus::NO_CHANGE; if (read_records_size == need_records_size) break; @@ -151,10 +142,4 @@ bool ReadBufferFromFileLog::nextImpl() return true; } -void ReadBufferFromFileLog::waitFunc() -{ - sleepForMicroseconds(poll_timeout); - time_out = true; -} - } diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index e7385282f71..dc5f4028b8d 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -3,12 +3,11 @@ #include #include #include -#include +#include #include #include #include -#include namespace Poco { @@ -20,16 +19,19 @@ namespace DB class ReadBufferFromFileLog : public ReadBuffer { public: - ReadBufferFromFileLog(const String & path_, Poco::Logger * log_, size_t max_batch_size, size_t poll_timeout_, ContextPtr context_); + ReadBufferFromFileLog( + StorageFileLog & storage_, + size_t max_batch_size, + size_t poll_timeout_, + ContextPtr context_, + size_t stream_number_, + size_t max_streams_number_); ~ReadBufferFromFileLog() override = default; - void open(); - void close(); - auto pollTimeout() const { return poll_timeout; } - inline bool hasMorePolledRecords() const { return current != records.end(); } + bool hasMorePolledRecords() const { return current != records.end(); } bool poll(); @@ -44,19 +46,18 @@ private: BufferStatus buffer_status; - const String path; - - bool path_is_directory = false; - Poco::Logger * log; + + StorageFileLog & storage; + const size_t batch_size = 1; const size_t poll_timeout = 0; - bool time_out = false; - - ContextPtr context; + size_t stream_number; + size_t max_streams_number; + bool allowed = true; using Record = std::string; @@ -76,8 +77,5 @@ private: void cleanUnprocessed(); bool nextImpl() override; - - void waitFunc(); - }; } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index a6f819580d5..5df8a416701 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -383,16 +384,9 @@ void registerStorageFileLog(StorageFactory & factory) }); } -NamesAndTypesList StorageFileLog::getVirtuals() const -{ - auto result = NamesAndTypesList{}; - return result; -} - Names StorageFileLog::getVirtualColumnNames() { - auto result = Names{}; - return result; + return {}; } void StorageFileLog::watchFunc() diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 314724d73be..3e5edea33ef 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -22,6 +22,14 @@ class StorageFileLog final : public shared_ptr_helper, public IS friend struct shared_ptr_helper; public: + enum class FileStatus + { + BEGIN, + NO_CHANGE, + UPDATED, + REMOVED + }; + using Files = std::vector; std::string getName() const override { return "FileLog"; } @@ -42,10 +50,10 @@ public: const auto & getFormatName() const { return format_name; } - NamesAndTypesList getVirtuals() const override; static Names getVirtualColumnNames(); - auto & getBuffer() { return buffer; } + auto & getFileNames() { return file_names; } + auto & getFileStatus() { return file_status; } protected: StorageFileLog( @@ -64,16 +72,6 @@ private: const String format_name; Poco::Logger * log; - ReadBufferFromFileLogPtr buffer; - - enum class FileStatus - { - BEGIN, - NO_CHANGE, - UPDATED, - REMOVED - }; - struct FileContext { FileStatus status = FileStatus::BEGIN; From dfea640c67ae88e7ab4785993f717b1517449816 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 5 Sep 2021 11:41:13 +0000 Subject: [PATCH 013/919] fix --- .../FileLog/FileLogDirectoryWatcher.cpp | 2 +- .../FileLog/FileLogDirectoryWatcher.h | 2 +- src/Storages/FileLog/FileLogSource.cpp | 24 ++++++++++++------- src/Storages/FileLog/FileLogSource.h | 6 ++--- .../FileLog/ReadBufferFromFileLog.cpp | 1 - src/Storages/FileLog/ReadBufferFromFileLog.h | 4 ++-- src/Storages/FileLog/StorageFileLog.cpp | 14 +++++------ src/Storages/FileLog/StorageFileLog.h | 2 -- 8 files changed, 29 insertions(+), 26 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 931f9c9aa06..bc731500d61 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -20,7 +20,7 @@ FileLogDirectoryWatcher::Events FileLogDirectoryWatcher::getEvents() return res; } -bool FileLogDirectoryWatcher::getError() const +bool FileLogDirectoryWatcher::hasError() const { return error; } diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index 99fd5cf0af1..651d1d9db2d 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -24,7 +24,7 @@ public: Events getEvents(); - bool getError() const; + bool hasError() const; const std::string & getPath() const; diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index 4827a048d6e..ecc43c579c7 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -24,16 +24,15 @@ FileLogSource::FileLogSource( size_t poll_time_out_, size_t stream_number_, size_t max_streams_number_) - : SourceWithProgress(metadata_snapshot->getSampleBlockForColumns(columns, storage_.getVirtuals(), storage_.getStorageID())) + : SourceWithProgress(metadata_snapshot_->getSampleBlockForColumns(columns, storage_.getVirtuals(), storage_.getStorageID())) , storage(storage_) , metadata_snapshot(metadata_snapshot_) , context(context_) , column_names(columns) , max_block_size(max_block_size_) , poll_time_out(poll_time_out_) - , non_virtual_header(metadata_snapshot->getSampleBlockNonMaterialized()) - , virtual_header( - metadata_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames(), storage.getVirtuals(), storage.getStorageID())) + , non_virtual_header(metadata_snapshot_->getSampleBlockNonMaterialized()) + , column_names_and_types(metadata_snapshot_->getColumns().getByNames(ColumnsDescription::All, columns, true)) { buffer = std::make_unique(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_); } @@ -43,7 +42,7 @@ Chunk FileLogSource::generate() if (!buffer) return {}; - MutableColumns result_columns = non_virtual_header.cloneEmptyColumns(); + MutableColumns read_columns = non_virtual_header.cloneEmptyColumns(); auto input_format = FormatFactory::instance().getInputFormat( storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size); @@ -79,7 +78,7 @@ Chunk FileLogSource::generate() auto columns = chunk.detachColumns(); for (size_t i = 0, s = columns.size(); i < s; ++i) { - result_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); + read_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); } break; } @@ -103,9 +102,9 @@ Chunk FileLogSource::generate() { new_rows = read_file_log(); } - catch (Exception &) + catch (...) { - throw; + tryLogCurrentException(__PRETTY_FUNCTION__); } } if (new_rows) @@ -122,6 +121,15 @@ Chunk FileLogSource::generate() if (total_rows == 0) return {}; + Columns result_columns; + result_columns.reserve(column_names_and_types.size()); + + for (const auto & elem : column_names_and_types) + { + auto index = non_virtual_header.getPositionByName(elem.getNameInStorage()); + result_columns.emplace_back(std::move(read_columns[index])); + } + return Chunk(std::move(result_columns), total_rows); } diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index 7db2602dc8c..1dc55f2c0c2 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -43,10 +43,8 @@ private: std::unique_ptr buffer; - bool started = false; - - const Block non_virtual_header; - const Block virtual_header; + Block non_virtual_header; + const NamesAndTypesList column_names_and_types; }; } diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 4df13199901..0c164c6b070 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -92,7 +92,6 @@ ReadBufferFromFileLog::Records ReadBufferFromFileLog::pollBatch(size_t batch_siz return new_records; } -// TODO void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_records, size_t batch_size_) { size_t need_records_size = batch_size_ - new_records.size(); diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index dc5f4028b8d..a7447e5edcd 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -50,8 +50,8 @@ private: StorageFileLog & storage; - const size_t batch_size = 1; - const size_t poll_timeout = 0; + size_t batch_size; + size_t poll_timeout; ContextPtr context; diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 5df8a416701..cb083baddb2 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -77,6 +77,10 @@ StorageFileLog::StorageFileLog( } } } + else + { + throw Exception("The path neigher a regular file, nor a directory", ErrorCodes::BAD_ARGUMENTS); + } watch_task = getContext()->getMessageBrokerSchedulePool().createTask("watchTask", [this] { watchFunc(); }); @@ -384,23 +388,19 @@ void registerStorageFileLog(StorageFactory & factory) }); } -Names StorageFileLog::getVirtualColumnNames() -{ - return {}; -} - void StorageFileLog::watchFunc() { FileLogDirectoryWatcher dw(path); while (true) { - sleepForMicroseconds(filelog_settings->filelog_poll_timeout_ms.totalMilliseconds()); + sleepForMilliseconds(filelog_settings->filelog_poll_timeout_ms.totalMilliseconds()); - auto error = dw.getError(); + auto error = dw.hasError(); if (error) LOG_INFO(log, "Error happened during watching directory {}.", dw.getPath()); auto events = dw.getEvents(); + std::lock_guard lock(status_mutex); for (const auto & event : events) diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 3e5edea33ef..ada9ebf4004 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -50,8 +50,6 @@ public: const auto & getFormatName() const { return format_name; } - static Names getVirtualColumnNames(); - auto & getFileNames() { return file_names; } auto & getFileStatus() { return file_status; } From b97faf8b25d9464468b9394342330606ed45c7f3 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 5 Sep 2021 12:12:04 +0000 Subject: [PATCH 014/919] fix style --- src/Storages/FileLog/StorageFileLog.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index cb083baddb2..7d14df0379f 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -33,6 +33,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; } namespace @@ -79,7 +80,7 @@ StorageFileLog::StorageFileLog( } else { - throw Exception("The path neigher a regular file, nor a directory", ErrorCodes::BAD_ARGUMENTS); + throw Exception("The path neither a regular file, nor a directory", ErrorCodes::BAD_ARGUMENTS); } watch_task = getContext()->getMessageBrokerSchedulePool().createTask("watchTask", [this] { watchFunc(); }); From 580b047d9699ea1633611f867b3e1e1feae6478f Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 5 Sep 2021 13:45:08 +0000 Subject: [PATCH 015/919] fix --- src/Storages/FileLog/StorageFileLog.cpp | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 7d14df0379f..63a282b3a20 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -98,7 +98,7 @@ Pipe StorageFileLog::read( size_t /* max_block_size */, unsigned /* num_streams */) { - std::lock_guard lock(status_mutex); + std::lock_guard lock(status_mutex); auto modified_context = Context::createCopy(local_context); clearInvalidFiles(); @@ -206,7 +206,7 @@ void StorageFileLog::threadFunc() { auto start_time = std::chrono::steady_clock::now(); - watch_task->activateAndSchedule(); + watch_task->activateAndSchedule(); // Keep streaming as long as there are attached views and streaming is not cancelled while (!task->stream_cancelled) @@ -238,7 +238,7 @@ void StorageFileLog::threadFunc() tryLogCurrentException(__PRETTY_FUNCTION__); } - watch_task->deactivate(); + watch_task->deactivate(); // Wait for attached views if (!task->stream_cancelled) task->holder->scheduleAfter(RESCHEDULE_MS); @@ -247,7 +247,7 @@ void StorageFileLog::threadFunc() bool StorageFileLog::streamToViews() { - std::lock_guard lock(status_mutex); + std::lock_guard lock(status_mutex); Stopwatch watch; auto table_id = getStorageID(); @@ -292,6 +292,8 @@ bool StorageFileLog::streamToViews() QueryPipeline pipeline; pipeline.init(Pipe::unitePipes(std::move(pipes))); + assertBlocksHaveEqualStructure(pipeline.getHeader(), block_io.out->getHeader(), "StorageFileLog streamToViews"); + size_t rows = 0; PullingPipelineExecutor executor(pipeline); @@ -394,7 +396,7 @@ void StorageFileLog::watchFunc() FileLogDirectoryWatcher dw(path); while (true) { - sleepForMilliseconds(filelog_settings->filelog_poll_timeout_ms.totalMilliseconds()); + sleepForMilliseconds(getPollTimeoutMillisecond()); auto error = dw.hasError(); if (error) @@ -402,10 +404,9 @@ void StorageFileLog::watchFunc() auto events = dw.getEvents(); - std::lock_guard lock(status_mutex); - for (const auto & event : events) { + std::lock_guard lock(status_mutex); switch (event.type) { From a22a0d6c68459763c2e0fd6a3a63f84aa64cfdfa Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 5 Sep 2021 13:47:20 +0000 Subject: [PATCH 016/919] fix style --- src/Storages/FileLog/StorageFileLog.cpp | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 63a282b3a20..ba64a590e63 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -98,7 +98,8 @@ Pipe StorageFileLog::read( size_t /* max_block_size */, unsigned /* num_streams */) { - std::lock_guard lock(status_mutex); + std::lock_guard lock(status_mutex); + auto modified_context = Context::createCopy(local_context); clearInvalidFiles(); @@ -206,7 +207,7 @@ void StorageFileLog::threadFunc() { auto start_time = std::chrono::steady_clock::now(); - watch_task->activateAndSchedule(); + watch_task->activateAndSchedule(); // Keep streaming as long as there are attached views and streaming is not cancelled while (!task->stream_cancelled) @@ -238,7 +239,7 @@ void StorageFileLog::threadFunc() tryLogCurrentException(__PRETTY_FUNCTION__); } - watch_task->deactivate(); + watch_task->deactivate(); // Wait for attached views if (!task->stream_cancelled) task->holder->scheduleAfter(RESCHEDULE_MS); @@ -247,7 +248,7 @@ void StorageFileLog::threadFunc() bool StorageFileLog::streamToViews() { - std::lock_guard lock(status_mutex); + std::lock_guard lock(status_mutex); Stopwatch watch; auto table_id = getStorageID(); @@ -406,7 +407,7 @@ void StorageFileLog::watchFunc() for (const auto & event : events) { - std::lock_guard lock(status_mutex); + std::lock_guard lock(status_mutex); switch (event.type) { From 3ce733121131da78380bab99db545dff00abf21b Mon Sep 17 00:00:00 2001 From: George Date: Tue, 7 Sep 2021 20:07:27 +0300 Subject: [PATCH 017/919] Edited and translated to Russian --- .../table-engines/integrations/mongodb.md | 4 ++-- .../external-dicts-dict-layout.md | 5 +++-- .../table-engines/integrations/mongodb.md | 20 ++++++++++++++++--- docs/ru/operations/settings/settings-users.md | 2 +- .../external-dicts-dict-layout.md | 20 ++++++++++++++++--- .../sql-reference/statements/create/user.md | 1 + 6 files changed, 41 insertions(+), 11 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/mongodb.md b/docs/en/engines/table-engines/integrations/mongodb.md index 9839893d4e8..899db9ac6ae 100644 --- a/docs/en/engines/table-engines/integrations/mongodb.md +++ b/docs/en/engines/table-engines/integrations/mongodb.md @@ -36,7 +36,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name Create a table in ClickHouse which allows to read data from MongoDB collection: -``` text +``` sql CREATE TABLE mongo_table ( key UInt64, @@ -46,7 +46,7 @@ CREATE TABLE mongo_table To read from an SSL secured MongoDB server: -``` text +``` sql CREATE TABLE mongo_table_ssl ( key UInt64, diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 00a9610ce91..eb8ca425d24 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -300,8 +300,9 @@ When searching for a dictionary, the cache is searched first. For each block of If keys are not found in dictionary, then update cache task is created and added into update queue. Update queue properties can be controlled with settings `max_update_queue_size`, `update_queue_push_timeout_milliseconds`, `query_wait_timeout_milliseconds`, `max_threads_for_updates`. -For cache dictionaries, the expiration [lifetime](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) of data in the cache can be set. If more time than `lifetime` has passed since loading the data in a cell, the cell’s value is not used and key becomes expired, and it is re-requested the next time it needs to be used this behaviour can be configured with setting `allow_read_expired_keys`. -This is the least effective of all the ways to store dictionaries. The speed of the cache depends strongly on correct settings and the usage scenario. A cache type dictionary performs well only when the hit rates are high enough (recommended 99% and higher). You can view the average hit rate in the `system.dictionaries` table. +For cache dictionaries, the expiration [lifetime](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) of data in the cache can be set. If more time than `lifetime` has passed since loading the data in a cell, the cell’s value is not used and key becomes expired. The key is re-requested the next time it needs to be used. This behaviour can be configured with setting `allow_read_expired_keys`. + +This is the least effective of all the ways to store dictionaries. The speed of the cache depends strongly on correct settings and the usage scenario. A cache type dictionary performs well only when the hit rates are high enough (recommended 99% and higher). You can view the average hit rate in the [system.dictionaries](../../../operations/system-tables/dictionaries.md) table. If setting `allow_read_expired_keys` is set to 1, by default 0. Then dictionary can support asynchronous updates. If a client requests keys and all of them are in cache, but some of them are expired, then dictionary will return expired keys for a client and request them asynchronously from the source. diff --git a/docs/ru/engines/table-engines/integrations/mongodb.md b/docs/ru/engines/table-engines/integrations/mongodb.md index 05820d03fe6..0fd463d511a 100644 --- a/docs/ru/engines/table-engines/integrations/mongodb.md +++ b/docs/ru/engines/table-engines/integrations/mongodb.md @@ -15,7 +15,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name name1 [type1], name2 [type2], ... -) ENGINE = MongoDB(host:port, database, collection, user, password); +) ENGINE = MongoDB(host:port, database, collection, user, password [, options]); ``` **Параметры движка** @@ -30,11 +30,13 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name - `password` — пароль пользователя. +- `options` — MongoDB connection string options (optional parameter). + ## Примеры использования {#usage-example} -Таблица в ClickHouse для чтения данных из колекции MongoDB: +Создание таблицы в ClickHouse для чтения данных из колекции MongoDB: -``` text +``` sql CREATE TABLE mongo_table ( key UInt64, @@ -42,6 +44,18 @@ CREATE TABLE mongo_table ) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table', 'testuser', 'clickhouse'); ``` +Чтение из сервера MongoDB, защищенного SSL: + +``` sql +CREATE TABLE mongo_table_ssl +( + key UInt64, + data String +) ENGINE = MongoDB('mongo2:27017', 'test', 'simple_table', 'testuser', 'clickhouse', 'ssl=true'); +``` + + + Запрос к таблице: ``` sql diff --git a/docs/ru/operations/settings/settings-users.md b/docs/ru/operations/settings/settings-users.md index 6a10e518817..0f85f22f7ea 100644 --- a/docs/ru/operations/settings/settings-users.md +++ b/docs/ru/operations/settings/settings-users.md @@ -28,7 +28,7 @@ toc_title: "Настройки пользователей" profile_name default - + default_database>default diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 06fe4ae327a..b61c2cbcbd7 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -297,9 +297,13 @@ RANGE(MIN StartDate MAX EndDate); При поиске в словаре сначала просматривается кэш. На каждый блок данных, все не найденные в кэше или устаревшие ключи запрашиваются у источника с помощью `SELECT attrs... FROM db.table WHERE id IN (k1, k2, ...)`. Затем, полученные данные записываются в кэш. -Для cache-словарей может быть задано время устаревания [lifetime](external-dicts-dict-lifetime.md) данных в кэше. Если от загрузки данных в ячейке прошло больше времени, чем `lifetime`, то значение не используется, и будет запрошено заново при следующей необходимости его использовать. +Если ключи не были найдены в словаре, то создается задание для обновления кэша, которое добавляется в очередь обновлений. Параметры очереди обновлений можно устанавливать настройками `max_update_queue_size`, `update_queue_push_timeout_milliseconds`, `query_wait_timeout_milliseconds`, `max_threads_for_updates` -Это наименее эффективный из всех способов размещения словарей. Скорость работы кэша очень сильно зависит от правильности настройки и сценария использования. Словарь типа cache показывает высокую производительность лишь при достаточно больших hit rate-ах (рекомендуется 99% и выше). Посмотреть средний hit rate можно в таблице `system.dictionaries`. +Для cache-словарей может быть задано время устаревания [lifetime](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) данных в кэше. Если от загрузки данных в ячейке прошло больше времени, чем `lifetime`, то значение не используется, а ключ устаревает. Ключ будет запрошен заново при следующей необходимости его использовать. Это можно настроить с помощью `allow_read_expired_keys`. + +Это наименее эффективный из всех способов размещения словарей. Скорость работы кэша очень сильно зависит от правильности настройки и сценария использования. Словарь типа cache показывает высокую производительность лишь при достаточно больших hit rate-ах (рекомендуется 99% и выше). Посмотреть средний hit rate можно в таблице [system.dictionaries](../../../operations/system-tables/dictionaries.md). + +Если параметр `allow_read_expired_keys` выставлен на 1 (0 по умолчанию), то словарь поддерживает асинхронные обновления. Если клиент запрашивает ключи, которые находятся в кэше, но при этом некоторые из них устарели, то словарь вернет устаревшие ключи клиенту и запросит их асинхронно у источника. Чтобы увеличить производительность кэша, используйте подзапрос с `LIMIT`, а снаружи вызывайте функцию со словарём. @@ -312,6 +316,16 @@ RANGE(MIN StartDate MAX EndDate); 1000000000 + + 0 + + 100000 + + 10 + + 60000 + + 4 ``` @@ -338,7 +352,7 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000)) ### ssd_cache {#ssd-cache} -Похож на `cache`, но хранит данные на SSD и индекс в оперативной памяти. +Похож на `cache`, но хранит данные на SSD и индекс в оперативной памяти. Все параметры, относящиеся к очереди обновлений, могут также быть применены к SSD-кэш словарям. ``` xml diff --git a/docs/ru/sql-reference/statements/create/user.md b/docs/ru/sql-reference/statements/create/user.md index 22efaa71bfc..f6248d97ba9 100644 --- a/docs/ru/sql-reference/statements/create/user.md +++ b/docs/ru/sql-reference/statements/create/user.md @@ -15,6 +15,7 @@ CREATE USER [IF NOT EXISTS | OR REPLACE] name1 [ON CLUSTER cluster_name1] [NOT IDENTIFIED | IDENTIFIED {[WITH {no_password | plaintext_password | sha256_password | sha256_hash | double_sha1_password | double_sha1_hash}] BY {'password' | 'hash'}} | {WITH ldap SERVER 'server_name'} | {WITH kerberos [REALM 'realm']}] [HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] [DEFAULT ROLE role [,...]] + [DEFAULT DATABASE database | NONE] [GRANTEES {user | role | ANY | NONE} [,...] [EXCEPT {user | role} [,...]]] [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY | WRITABLE] | PROFILE 'profile_name'] [,...] ``` From 8f328ca6c4bb07c5918e9ad0059101d5a715638e Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Fri, 10 Sep 2021 03:26:10 +0300 Subject: [PATCH 018/919] Apply suggestions from code review Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/engines/table-engines/integrations/mongodb.md | 4 ++-- docs/ru/operations/settings/settings-users.md | 2 +- .../external-dictionaries/external-dicts-dict-layout.md | 8 ++++---- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/ru/engines/table-engines/integrations/mongodb.md b/docs/ru/engines/table-engines/integrations/mongodb.md index 0fd463d511a..7a56af4c274 100644 --- a/docs/ru/engines/table-engines/integrations/mongodb.md +++ b/docs/ru/engines/table-engines/integrations/mongodb.md @@ -34,7 +34,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name ## Примеры использования {#usage-example} -Создание таблицы в ClickHouse для чтения данных из колекции MongoDB: +Создание таблицы в ClickHouse для чтения данных из коллекции MongoDB: ``` sql CREATE TABLE mongo_table @@ -44,7 +44,7 @@ CREATE TABLE mongo_table ) ENGINE = MongoDB('mongo1:27017', 'test', 'simple_table', 'testuser', 'clickhouse'); ``` -Чтение из сервера MongoDB, защищенного SSL: +Чтение с сервера MongoDB, защищенного SSL: ``` sql CREATE TABLE mongo_table_ssl diff --git a/docs/ru/operations/settings/settings-users.md b/docs/ru/operations/settings/settings-users.md index 0f85f22f7ea..4570ce38bad 100644 --- a/docs/ru/operations/settings/settings-users.md +++ b/docs/ru/operations/settings/settings-users.md @@ -28,7 +28,7 @@ toc_title: "Настройки пользователей" profile_name default - default_database>default + default diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index b61c2cbcbd7..f0b4eb614c5 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -297,13 +297,13 @@ RANGE(MIN StartDate MAX EndDate); При поиске в словаре сначала просматривается кэш. На каждый блок данных, все не найденные в кэше или устаревшие ключи запрашиваются у источника с помощью `SELECT attrs... FROM db.table WHERE id IN (k1, k2, ...)`. Затем, полученные данные записываются в кэш. -Если ключи не были найдены в словаре, то создается задание для обновления кэша, которое добавляется в очередь обновлений. Параметры очереди обновлений можно устанавливать настройками `max_update_queue_size`, `update_queue_push_timeout_milliseconds`, `query_wait_timeout_milliseconds`, `max_threads_for_updates` +Если ключи не были найдены в словаре, то для обновления кэша создается задание и добавляется в очередь обновлений. Параметры очереди обновлений можно устанавливать настройками `max_update_queue_size`, `update_queue_push_timeout_milliseconds`, `query_wait_timeout_milliseconds`, `max_threads_for_updates` Для cache-словарей может быть задано время устаревания [lifetime](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) данных в кэше. Если от загрузки данных в ячейке прошло больше времени, чем `lifetime`, то значение не используется, а ключ устаревает. Ключ будет запрошен заново при следующей необходимости его использовать. Это можно настроить с помощью `allow_read_expired_keys`. -Это наименее эффективный из всех способов размещения словарей. Скорость работы кэша очень сильно зависит от правильности настройки и сценария использования. Словарь типа cache показывает высокую производительность лишь при достаточно больших hit rate-ах (рекомендуется 99% и выше). Посмотреть средний hit rate можно в таблице [system.dictionaries](../../../operations/system-tables/dictionaries.md). +Это наименее эффективный из всех способов размещения словарей. Скорость работы кэша очень сильно зависит от правильности настройки и сценария использования. Словарь типа `cache` показывает высокую производительность лишь при достаточно большой частоте успешных обращений (рекомендуется 99% и выше). Посмотреть среднюю частоту успешных обращений (`hit rate`) можно в таблице [system.dictionaries](../../../operations/system-tables/dictionaries.md). -Если параметр `allow_read_expired_keys` выставлен на 1 (0 по умолчанию), то словарь поддерживает асинхронные обновления. Если клиент запрашивает ключи, которые находятся в кэше, но при этом некоторые из них устарели, то словарь вернет устаревшие ключи клиенту и запросит их асинхронно у источника. +Если параметр `allow_read_expired_keys` выставлен в 1 (0 по умолчанию), то словарь поддерживает асинхронные обновления. Если клиент запрашивает ключи, которые находятся в кэше, но при этом некоторые из них устарели, то словарь вернет устаревшие ключи клиенту и запросит их асинхронно у источника. Чтобы увеличить производительность кэша, используйте подзапрос с `LIMIT`, а снаружи вызывайте функцию со словарём. @@ -352,7 +352,7 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000)) ### ssd_cache {#ssd-cache} -Похож на `cache`, но хранит данные на SSD и индекс в оперативной памяти. Все параметры, относящиеся к очереди обновлений, могут также быть применены к SSD-кэш словарям. +Похож на `cache`, но хранит данные на SSD, а индекс в оперативной памяти. Все параметры, относящиеся к очереди обновлений, могут также быть применены к SSD-кэш словарям. ``` xml From 810d180ce2faf08b2e2e0eb1d1dcf00777194b31 Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 10 Sep 2021 17:21:03 +0000 Subject: [PATCH 019/919] fix --- src/Storages/FileLog/StorageFileLog.cpp | 88 ++++++++++++------------- src/Storages/FileLog/StorageFileLog.h | 5 +- 2 files changed, 45 insertions(+), 48 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index ba64a590e63..40c1a780739 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include @@ -83,6 +82,8 @@ StorageFileLog::StorageFileLog( throw Exception("The path neither a regular file, nor a directory", ErrorCodes::BAD_ARGUMENTS); } + directory_watch = std::make_unique(path); + watch_task = getContext()->getMessageBrokerSchedulePool().createTask("watchTask", [this] { watchFunc(); }); auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); @@ -207,7 +208,7 @@ void StorageFileLog::threadFunc() { auto start_time = std::chrono::steady_clock::now(); - watch_task->activateAndSchedule(); + watch_task->activate(); // Keep streaming as long as there are attached views and streaming is not cancelled while (!task->stream_cancelled) @@ -215,6 +216,8 @@ void StorageFileLog::threadFunc() if (!checkDependencies(table_id)) break; + watch_task->scheduleAfter(RESCHEDULE_MS); + LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); auto stream_is_stalled = streamToViews(); @@ -319,15 +322,15 @@ void StorageFileLog::clearInvalidFiles() /// Do not need to hold file_status lock, since it will be holded /// by caller when call this function std::vector valid_files; - for (const auto & it : file_names) + for (const auto & file_name : file_names) { - if (file_status.at(it).status == FileStatus::REMOVED) + if (file_status.at(file_name).status == FileStatus::REMOVED) { - file_status.erase(it); + file_status.erase(file_name); } else { - valid_files.push_back(it); + valid_files.push_back(file_name); } } @@ -394,52 +397,43 @@ void registerStorageFileLog(StorageFactory & factory) void StorageFileLog::watchFunc() { - FileLogDirectoryWatcher dw(path); - while (true) + auto error = directory_watch->hasError(); + if (error) + LOG_INFO(log, "Error happened during watching directory {}.", directory_watch->getPath()); + + auto events = directory_watch->getEvents(); + + for (const auto & event : events) { - sleepForMilliseconds(getPollTimeoutMillisecond()); - - auto error = dw.hasError(); - if (error) - LOG_INFO(log, "Error happened during watching directory {}.", dw.getPath()); - - auto events = dw.getEvents(); - - for (const auto & event : events) + std::lock_guard lock(status_mutex); + switch (event.type) { - std::lock_guard lock(status_mutex); - switch (event.type) - { + case Poco::DirectoryWatcher::DW_ITEM_ADDED: + LOG_TRACE(log, "New event {} watched.", event.callback); + if (std::filesystem::is_regular_file(event.path)) + { + file_status[event.path].reader = std::ifstream(event.path); + file_names.push_back(event.path); + } + break; - case Poco::DirectoryWatcher::DW_ITEM_ADDED: - LOG_TRACE(log, "New event {} watched.", event.callback); - if (std::filesystem::is_regular_file(event.path)) - { - file_status[event.path].reader = std::ifstream(event.path); - file_names.push_back(event.path); - } - break; + case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: + LOG_TRACE(log, "New event {} watched.", event.callback); + if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) + { + file_status[event.path].status = FileStatus::UPDATED; + } + break; - case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: - LOG_TRACE(log, "New event {} watched.", event.callback); - if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) - { - file_status[event.path].status = FileStatus::UPDATED; - } - break; - - case Poco::DirectoryWatcher::DW_ITEM_REMOVED: - case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: - case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: - LOG_TRACE(log, "New event {} watched.", event.callback); - if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) - { - file_status[event.path].status = FileStatus::REMOVED; - } - break; - } + case Poco::DirectoryWatcher::DW_ITEM_REMOVED: + case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: + case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: + LOG_TRACE(log, "New event {} watched.", event.callback); + if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) + { + file_status[event.path].status = FileStatus::REMOVED; + } } } } - } diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index ada9ebf4004..a3f830a086a 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -83,6 +84,8 @@ private: std::mutex status_mutex; + std::unique_ptr directory_watch = nullptr; + // Stream thread struct TaskContext { @@ -109,7 +112,7 @@ private: bool streamToViews(); bool checkDependencies(const StorageID & table_id); - [[noreturn]] void watchFunc(); + void watchFunc(); }; } From aa9951638016eeeb9b3673f5c40f5a18cd9e81e5 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 12 Sep 2021 16:25:15 +0000 Subject: [PATCH 020/919] fix fix --- src/Storages/FileLog/FileLogSource.cpp | 13 ++++- src/Storages/FileLog/StorageFileLog.cpp | 70 +++++++++++-------------- src/Storages/FileLog/StorageFileLog.h | 4 +- 3 files changed, 44 insertions(+), 43 deletions(-) diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index ecc43c579c7..61410e8488c 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -15,6 +15,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +const auto MAX_FAILED_POLL_ATTEMPTS = 10; + FileLogSource::FileLogSource( StorageFileLog & storage_, const StorageMetadataPtr & metadata_snapshot_, @@ -91,10 +93,11 @@ Chunk FileLogSource::generate() }; size_t total_rows = 0; + size_t failed_poll_attempts = 0; + Stopwatch watch; while (true) { - Stopwatch watch; size_t new_rows = 0; if (buffer->poll()) { @@ -111,8 +114,14 @@ Chunk FileLogSource::generate() { total_rows = total_rows + new_rows; } + else + { + ++failed_poll_attempts; + } - if (!buffer->hasMorePolledRecords() && ((total_rows >= max_block_size) || watch.elapsedMilliseconds() > poll_time_out)) + if (!buffer->hasMorePolledRecords() + && ((total_rows >= max_block_size) || watch.elapsedMilliseconds() > poll_time_out + || failed_poll_attempts >= MAX_FAILED_POLL_ATTEMPTS)) { break; } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 40c1a780739..c2012d40208 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -84,8 +84,6 @@ StorageFileLog::StorageFileLog( directory_watch = std::make_unique(path); - watch_task = getContext()->getMessageBrokerSchedulePool().createTask("watchTask", [this] { watchFunc(); }); - auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); task = std::make_shared(std::move(thread)); } @@ -101,17 +99,18 @@ Pipe StorageFileLog::read( { std::lock_guard lock(status_mutex); - auto modified_context = Context::createCopy(local_context); + updateFileStatus(); - clearInvalidFiles(); - - auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_names.size()); /// No files to parse - if (max_streams_number == 0) + if (file_names.empty()) { return Pipe{}; } + auto modified_context = Context::createCopy(local_context); + + auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_names.size()); + Pipes pipes; pipes.reserve(max_streams_number); for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) @@ -142,7 +141,6 @@ void StorageFileLog::shutdown() LOG_TRACE(log, "Waiting for cleanup"); task->holder->deactivate(); - watch_task->deactivate(); for (auto & file : file_status) { @@ -208,20 +206,16 @@ void StorageFileLog::threadFunc() { auto start_time = std::chrono::steady_clock::now(); - watch_task->activate(); - // Keep streaming as long as there are attached views and streaming is not cancelled while (!task->stream_cancelled) { if (!checkDependencies(table_id)) break; - watch_task->scheduleAfter(RESCHEDULE_MS); - LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); - auto stream_is_stalled = streamToViews(); - if (stream_is_stalled) + auto file_status_no_update = streamToViews(); + if (file_status_no_update) { LOG_TRACE(log, "Stream stalled. Reschedule."); break; @@ -234,6 +228,7 @@ void StorageFileLog::threadFunc() LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); break; } + updateFileStatus(); } } } @@ -242,7 +237,6 @@ void StorageFileLog::threadFunc() tryLogCurrentException(__PRETTY_FUNCTION__); } - watch_task->deactivate(); // Wait for attached views if (!task->stream_cancelled) task->holder->scheduleAfter(RESCHEDULE_MS); @@ -260,9 +254,7 @@ bool StorageFileLog::streamToViews() throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); auto metadata_snapshot = getInMemoryMetadataPtr(); - clearInvalidFiles(); - - auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_names.size()); + auto max_streams_number = std::min(filelog_settings->filelog_max_threads.value, file_names.size()); /// No files to parse if (max_streams_number == 0) { @@ -280,13 +272,14 @@ bool StorageFileLog::streamToViews() Pipes pipes; pipes.reserve(max_streams_number); + auto names = block_io.out->getHeader().getNames(); for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) { pipes.emplace_back(std::make_shared( *this, metadata_snapshot, new_context, - block_io.out->getHeader().getNames(), + names, getPollMaxBatchSize(), getPollTimeoutMillisecond(), stream_number, @@ -314,27 +307,11 @@ bool StorageFileLog::streamToViews() LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); - return true; + return updateFileStatus(); } void StorageFileLog::clearInvalidFiles() { - /// Do not need to hold file_status lock, since it will be holded - /// by caller when call this function - std::vector valid_files; - for (const auto & file_name : file_names) - { - if (file_status.at(file_name).status == FileStatus::REMOVED) - { - file_status.erase(file_name); - } - else - { - valid_files.push_back(file_name); - } - } - - file_names.swap(valid_files); } void registerStorageFileLog(StorageFactory & factory) @@ -395,7 +372,7 @@ void registerStorageFileLog(StorageFactory & factory) }); } -void StorageFileLog::watchFunc() +bool StorageFileLog::updateFileStatus() { auto error = directory_watch->hasError(); if (error) @@ -405,7 +382,6 @@ void StorageFileLog::watchFunc() for (const auto & event : events) { - std::lock_guard lock(status_mutex); switch (event.type) { case Poco::DirectoryWatcher::DW_ITEM_ADDED: @@ -435,5 +411,23 @@ void StorageFileLog::watchFunc() } } } + /// Do not need to hold file_status lock, since it will be holded + /// by caller when call this function + std::vector valid_files; + for (const auto & file_name : file_names) + { + if (file_status.at(file_name).status == FileStatus::REMOVED) + { + file_status.erase(file_name); + } + else + { + valid_files.push_back(file_name); + } + } + + file_names.swap(valid_files); + + return events.empty() || file_names.empty(); } } diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index a3f830a086a..9f856fb26ea 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -99,8 +99,6 @@ private: using TaskThread = BackgroundSchedulePool::TaskHolder; - TaskThread watch_task; - void threadFunc(); void clearInvalidFiles(); @@ -112,7 +110,7 @@ private: bool streamToViews(); bool checkDependencies(const StorageID & table_id); - void watchFunc(); + bool updateFileStatus(); }; } From bd1990a67c3fe28c478d4531b1ec114407760b21 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 13 Sep 2021 03:44:12 +0000 Subject: [PATCH 021/919] fix test and restrict path --- src/Storages/FileLog/StorageFileLog.cpp | 6 +++--- src/Storages/FileLog/StorageFileLog.h | 2 +- tests/queries/0_stateless/01645_system_table_engines.sql | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index c2012d40208..96745b5f454 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -45,13 +45,13 @@ StorageFileLog::StorageFileLog( const StorageID & table_id_, ContextPtr context_, const ColumnsDescription & columns_, - const String & path_, + const String & relative_path_, const String & format_name_, std::unique_ptr settings) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) , filelog_settings(std::move(settings)) - , path(path_) + , path(getContext()->getUserFilesPath() + "/" + relative_path_) , format_name(format_name_) , log(&Poco::Logger::get("StorageFileLog (" + table_id_.table_name + ")")) { @@ -254,7 +254,7 @@ bool StorageFileLog::streamToViews() throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); auto metadata_snapshot = getInMemoryMetadataPtr(); - auto max_streams_number = std::min(filelog_settings->filelog_max_threads.value, file_names.size()); + auto max_streams_number = std::min(filelog_settings->filelog_max_threads.value, file_names.size()); /// No files to parse if (max_streams_number == 0) { diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 9f856fb26ea..e6d466e921b 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -59,7 +59,7 @@ protected: const StorageID & table_id_, ContextPtr context_, const ColumnsDescription & columns_, - const String & path_, + const String & relative_path_, const String & format_name_, std::unique_ptr settings); diff --git a/tests/queries/0_stateless/01645_system_table_engines.sql b/tests/queries/0_stateless/01645_system_table_engines.sql index 5e8eef5508b..c2048427670 100644 --- a/tests/queries/0_stateless/01645_system_table_engines.sql +++ b/tests/queries/0_stateless/01645_system_table_engines.sql @@ -1 +1 @@ -SELECT * FROM system.table_engines WHERE name in ('MergeTree', 'ReplicatedCollapsingMergeTree') FORMAT PrettyCompactNoEscapes; +SELECT * FROM system.table_engines WHERE name in ('MergeTree', 'ReplicatedCollapsingMergeTree') ORDER BY name FORMAT PrettyCompactNoEscapes; From 9d9ba5680827ea65b3da241e9af2dc78b2a090c4 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 13 Sep 2021 04:46:47 +0000 Subject: [PATCH 022/919] add test remove unneeded code --- .../FileLog/ReadBufferFromFileLog.cpp | 4 -- .../test_storage_filelog/__init__.py | 0 .../configs/config.d/cluster.xml | 16 +++++ .../configs/user_files_path.xml | 3 + .../integration/test_storage_filelog/test.py | 67 +++++++++++++++++++ .../user_files/logs/data.csv | 10 +++ 6 files changed, 96 insertions(+), 4 deletions(-) create mode 100644 tests/integration/test_storage_filelog/__init__.py create mode 100644 tests/integration/test_storage_filelog/configs/config.d/cluster.xml create mode 100644 tests/integration/test_storage_filelog/configs/user_files_path.xml create mode 100644 tests/integration/test_storage_filelog/test.py create mode 100644 tests/integration/test_storage_filelog/user_files/logs/data.csv diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 0c164c6b070..8e1d16ba8d7 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -11,10 +11,6 @@ namespace DB { -namespace ErrorCodes -{ -} - ReadBufferFromFileLog::ReadBufferFromFileLog( StorageFileLog & storage_, size_t max_batch_size, diff --git a/tests/integration/test_storage_filelog/__init__.py b/tests/integration/test_storage_filelog/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_filelog/configs/config.d/cluster.xml b/tests/integration/test_storage_filelog/configs/config.d/cluster.xml new file mode 100644 index 00000000000..96e0e7c34c1 --- /dev/null +++ b/tests/integration/test_storage_filelog/configs/config.d/cluster.xml @@ -0,0 +1,16 @@ + + + + + + node1 + 9000 + + + node2 + 9001 + + + + + diff --git a/tests/integration/test_storage_filelog/configs/user_files_path.xml b/tests/integration/test_storage_filelog/configs/user_files_path.xml new file mode 100644 index 00000000000..b41f0f8e00d --- /dev/null +++ b/tests/integration/test_storage_filelog/configs/user_files_path.xml @@ -0,0 +1,3 @@ + + user_files + diff --git a/tests/integration/test_storage_filelog/test.py b/tests/integration/test_storage_filelog/test.py new file mode 100644 index 00000000000..9940c0afa7a --- /dev/null +++ b/tests/integration/test_storage_filelog/test.py @@ -0,0 +1,67 @@ +import pytest +import time +import shutil +import os +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance('node1', + main_configs=['configs/user_files_path.xml']) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_select(started_cluster): + try: + name = "filelog_table" + node1.query(""" + CREATE TABLE {name} (id UInt64, value UInt64) Engine=FileLog("logs", "CSV") + """.format(name=name)) + + assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "55\n" + + shutil.copy("user_files/logs/data.csv" "user_files/logs/data_copy.csv") + + assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "55\n" + + finally: + node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) + os.remove("user_files/logs/data_copy.csv") + + +def test_stream_to_view(started_cluster): + try: + name = "filelog_table" + node1.query(""" + CREATE TABLE {name} (id UInt64, value UInt64) Engine=FileLog("logs", "CSV") + """.format(name=name)) + + target_name = "mv" + node1.query(""" + CREATE Materialized View {target_name} engine=MergeTree order by id as select * from {name} + """.format(target_name = target_name, name=name)) + + time.sleep(10) + + assert node1.query("SELECT sum(id) FROM {target_name}".format(target_name=target_name)) == "55\n" + + shutil.copy("user_files/logs/data.csv" "user_files/logs/data_copy.csv") + + time.sleep(10) + + assert node1.query("SELECT sum(id) FROM {target_name}".format(target_name=target_name)) == "110\n" + + finally: + node1.query("DROP TABLE IF EXISTS {target_name}".format(target_name=target_name)) + node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) + os.remove("user_files/logs/data_copy.csv") diff --git a/tests/integration/test_storage_filelog/user_files/logs/data.csv b/tests/integration/test_storage_filelog/user_files/logs/data.csv new file mode 100644 index 00000000000..dd198b1bc93 --- /dev/null +++ b/tests/integration/test_storage_filelog/user_files/logs/data.csv @@ -0,0 +1,10 @@ +1, 1 +2, 2 +3, 3 +4, 4 +5, 5 +6, 6 +7, 7 +8, 8 +9, 9 +10, 10 From d2192902bf3c92b8fbc42c1cddac09936bf465cf Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 13 Sep 2021 06:16:43 +0000 Subject: [PATCH 023/919] try fix test --- tests/integration/test_storage_filelog/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_filelog/test.py b/tests/integration/test_storage_filelog/test.py index 9940c0afa7a..01e9e6e0d13 100644 --- a/tests/integration/test_storage_filelog/test.py +++ b/tests/integration/test_storage_filelog/test.py @@ -25,12 +25,12 @@ def test_select(started_cluster): try: name = "filelog_table" node1.query(""" - CREATE TABLE {name} (id UInt64, value UInt64) Engine=FileLog("logs", "CSV") + CREATE TABLE {name} (id UInt64, value UInt64) Engine=FileLog('logs', 'CSV') """.format(name=name)) assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "55\n" - shutil.copy("user_files/logs/data.csv" "user_files/logs/data_copy.csv") + shutil.copy("user_files/logs/data.csv", "user_files/logs/data_copy.csv") assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "55\n" @@ -43,7 +43,7 @@ def test_stream_to_view(started_cluster): try: name = "filelog_table" node1.query(""" - CREATE TABLE {name} (id UInt64, value UInt64) Engine=FileLog("logs", "CSV") + CREATE TABLE {name} (id UInt64, value UInt64) Engine=FileLog('logs', 'CSV') """.format(name=name)) target_name = "mv" From 218ac23a875c7e0cfd29d3bc45a9a2a332516802 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 13 Sep 2021 11:09:31 +0000 Subject: [PATCH 024/919] update test fix test --- .../test_storage_filelog/__init__.py | 0 .../configs/config.d/cluster.xml | 16 ----- .../configs/user_files_path.xml | 3 - .../integration/test_storage_filelog/test.py | 67 ------------------- .../user_files/logs/data.csv | 10 --- .../02023_storage_filelog.reference | 67 +++++++++++++++++++ .../0_stateless/02023_storage_filelog.sh | 50 ++++++++++++++ 7 files changed, 117 insertions(+), 96 deletions(-) delete mode 100644 tests/integration/test_storage_filelog/__init__.py delete mode 100644 tests/integration/test_storage_filelog/configs/config.d/cluster.xml delete mode 100644 tests/integration/test_storage_filelog/configs/user_files_path.xml delete mode 100644 tests/integration/test_storage_filelog/test.py delete mode 100644 tests/integration/test_storage_filelog/user_files/logs/data.csv create mode 100644 tests/queries/0_stateless/02023_storage_filelog.reference create mode 100755 tests/queries/0_stateless/02023_storage_filelog.sh diff --git a/tests/integration/test_storage_filelog/__init__.py b/tests/integration/test_storage_filelog/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_storage_filelog/configs/config.d/cluster.xml b/tests/integration/test_storage_filelog/configs/config.d/cluster.xml deleted file mode 100644 index 96e0e7c34c1..00000000000 --- a/tests/integration/test_storage_filelog/configs/config.d/cluster.xml +++ /dev/null @@ -1,16 +0,0 @@ - - - - - - node1 - 9000 - - - node2 - 9001 - - - - - diff --git a/tests/integration/test_storage_filelog/configs/user_files_path.xml b/tests/integration/test_storage_filelog/configs/user_files_path.xml deleted file mode 100644 index b41f0f8e00d..00000000000 --- a/tests/integration/test_storage_filelog/configs/user_files_path.xml +++ /dev/null @@ -1,3 +0,0 @@ - - user_files - diff --git a/tests/integration/test_storage_filelog/test.py b/tests/integration/test_storage_filelog/test.py deleted file mode 100644 index 01e9e6e0d13..00000000000 --- a/tests/integration/test_storage_filelog/test.py +++ /dev/null @@ -1,67 +0,0 @@ -import pytest -import time -import shutil -import os -from helpers.client import QueryRuntimeException -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) - -node1 = cluster.add_instance('node1', - main_configs=['configs/user_files_path.xml']) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -def test_select(started_cluster): - try: - name = "filelog_table" - node1.query(""" - CREATE TABLE {name} (id UInt64, value UInt64) Engine=FileLog('logs', 'CSV') - """.format(name=name)) - - assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "55\n" - - shutil.copy("user_files/logs/data.csv", "user_files/logs/data_copy.csv") - - assert node1.query("SELECT sum(id) FROM {name}".format(name=name)) == "55\n" - - finally: - node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) - os.remove("user_files/logs/data_copy.csv") - - -def test_stream_to_view(started_cluster): - try: - name = "filelog_table" - node1.query(""" - CREATE TABLE {name} (id UInt64, value UInt64) Engine=FileLog('logs', 'CSV') - """.format(name=name)) - - target_name = "mv" - node1.query(""" - CREATE Materialized View {target_name} engine=MergeTree order by id as select * from {name} - """.format(target_name = target_name, name=name)) - - time.sleep(10) - - assert node1.query("SELECT sum(id) FROM {target_name}".format(target_name=target_name)) == "55\n" - - shutil.copy("user_files/logs/data.csv" "user_files/logs/data_copy.csv") - - time.sleep(10) - - assert node1.query("SELECT sum(id) FROM {target_name}".format(target_name=target_name)) == "110\n" - - finally: - node1.query("DROP TABLE IF EXISTS {target_name}".format(target_name=target_name)) - node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) - os.remove("user_files/logs/data_copy.csv") diff --git a/tests/integration/test_storage_filelog/user_files/logs/data.csv b/tests/integration/test_storage_filelog/user_files/logs/data.csv deleted file mode 100644 index dd198b1bc93..00000000000 --- a/tests/integration/test_storage_filelog/user_files/logs/data.csv +++ /dev/null @@ -1,10 +0,0 @@ -1, 1 -2, 2 -3, 3 -4, 4 -5, 5 -6, 6 -7, 7 -8, 8 -9, 9 -10, 10 diff --git a/tests/queries/0_stateless/02023_storage_filelog.reference b/tests/queries/0_stateless/02023_storage_filelog.reference new file mode 100644 index 00000000000..a1ac16d1d55 --- /dev/null +++ b/tests/queries/0_stateless/02023_storage_filelog.reference @@ -0,0 +1,67 @@ +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 +100 100 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +10 10 +100 100 +1 1 +1 1 +2 2 +2 2 +3 3 +3 3 +4 4 +4 4 +5 5 +5 5 +6 6 +6 6 +7 7 +7 7 +8 8 +8 8 +9 9 +9 9 +10 10 +10 10 +100 100 +100 100 +1 1 +1 1 +2 2 +2 2 +3 3 +3 3 +4 4 +4 4 +5 5 +5 5 +6 6 +6 6 +7 7 +7 7 +8 8 +8 8 +9 9 +9 9 +10 10 +10 10 +100 100 +100 100 +111 111 diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh new file mode 100755 index 00000000000..6cd40e64274 --- /dev/null +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -0,0 +1,50 @@ +#!/usr/bin/env bash +set -eu + +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')" +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 ${user_files_path}/logs/ +echo 1, 1 >> ${user_files_path}/logs/a.txt +echo 2, 2 >> ${user_files_path}/logs/a.txt +echo 3, 3 >> ${user_files_path}/logs/a.txt +echo 4, 4 >> ${user_files_path}/logs/a.txt +echo 5, 5 >> ${user_files_path}/logs/a.txt +echo 6, 6 >> ${user_files_path}/logs/a.txt +echo 7, 7 >> ${user_files_path}/logs/a.txt +echo 8, 8 >> ${user_files_path}/logs/a.txt +echo 9, 9 >> ${user_files_path}/logs/a.txt +echo 10, 10 >> ${user_files_path}/logs/a.txt + +### 1st TEST in CLIENT mode. +${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('logs', 'CSV');" + +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" + +echo 100, 100 >> ${user_files_path}/logs/a.txt + +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" + +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/b.txt + +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" + +${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order by k as select * from file_log;" + +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/c.txt +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt + +${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" + +echo 111, 111 >> ${user_files_path}/logs/a.txt + +${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" + +rm -rf ${user_files_path}/logs From cfa92a0c1e9639e7ead70b201f2db55fbeeadc96 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 13 Sep 2021 14:50:27 +0000 Subject: [PATCH 025/919] fix fix --- .../FileLog/ReadBufferFromFileLog.cpp | 22 ++++++++++++---- src/Storages/FileLog/StorageFileLog.cpp | 25 ++++++++----------- src/Storages/FileLog/StorageFileLog.h | 2 +- .../0_stateless/02023_storage_filelog.sh | 4 +++ 4 files changed, 32 insertions(+), 21 deletions(-) diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 8e1d16ba8d7..bb43afe08df 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -103,23 +103,35 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ for (size_t i = start; i < end; ++i) { auto & file = file_status[file_names[i]]; - if (file.status == StorageFileLog::FileStatus::NO_CHANGE || file.status == StorageFileLog::FileStatus::REMOVED) + if (file.status == StorageFileLog::FileStatus::NO_CHANGE) continue; - while (read_records_size < need_records_size && file.reader.good() && !file.reader.eof()) + auto reader = std::ifstream(file_names[i]); + + reader.seekg(0, reader.end); + auto stream_end = reader.tellg(); + + reader.seekg(file.last_read_position); + + while (read_records_size < need_records_size && reader.tellg() < stream_end) { Record record; - std::getline(file.reader, record); + std::getline(reader, record); new_records.emplace_back(record); ++read_records_size; } - // Read to the end of the file - if (file.reader.eof()) + file.last_read_position = reader.tellg(); + + if (reader.tellg() == stream_end) + { file.status = StorageFileLog::FileStatus::NO_CHANGE; + } if (read_records_size == need_records_size) + { break; + } } } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 96745b5f454..91e6d045fce 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -61,7 +61,7 @@ StorageFileLog::StorageFileLog( if (std::filesystem::is_regular_file(path)) { - file_status[path].reader = std::ifstream(path); + file_status[path] = FileContext{}; file_names.push_back(path); } else if (std::filesystem::is_directory(path)) @@ -72,7 +72,7 @@ StorageFileLog::StorageFileLog( { if (dir_entry.is_regular_file()) { - file_status[dir_entry.path()].reader = std::ifstream(dir_entry.path()); + file_status[dir_entry.path()] = FileContext{}; file_names.push_back(dir_entry.path()); } } @@ -141,11 +141,6 @@ void StorageFileLog::shutdown() LOG_TRACE(log, "Waiting for cleanup"); task->holder->deactivate(); - - for (auto & file : file_status) - { - file.second.reader.close(); - } } size_t StorageFileLog::getMaxBlockSize() const @@ -374,6 +369,8 @@ void registerStorageFileLog(StorageFactory & factory) bool StorageFileLog::updateFileStatus() { + /// Do not need to hold file_status lock, since it will be holded + /// by caller when call this function auto error = directory_watch->hasError(); if (error) LOG_INFO(log, "Error happened during watching directory {}.", directory_watch->getPath()); @@ -385,18 +382,18 @@ bool StorageFileLog::updateFileStatus() switch (event.type) { case Poco::DirectoryWatcher::DW_ITEM_ADDED: - LOG_TRACE(log, "New event {} watched.", event.callback); + LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); if (std::filesystem::is_regular_file(event.path)) { - file_status[event.path].reader = std::ifstream(event.path); + file_status[event.path] = FileContext{}; file_names.push_back(event.path); } break; case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: - LOG_TRACE(log, "New event {} watched.", event.callback); - if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) - { + LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); + if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) + { file_status[event.path].status = FileStatus::UPDATED; } break; @@ -404,15 +401,13 @@ bool StorageFileLog::updateFileStatus() case Poco::DirectoryWatcher::DW_ITEM_REMOVED: case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: - LOG_TRACE(log, "New event {} watched.", event.callback); + LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) { file_status[event.path].status = FileStatus::REMOVED; } } } - /// Do not need to hold file_status lock, since it will be holded - /// by caller when call this function std::vector valid_files; for (const auto & file_name : file_names) { diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index e6d466e921b..d035c0e2903 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -74,7 +74,7 @@ private: struct FileContext { FileStatus status = FileStatus::BEGIN; - std::ifstream reader; + size_t last_read_position = 0; }; using NameToFile = std::unordered_map; diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 6cd40e64274..4987a4b526b 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -41,10 +41,14 @@ ${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/c.txt cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt +sleep 5 + ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" echo 111, 111 >> ${user_files_path}/logs/a.txt +sleep 5 + ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" rm -rf ${user_files_path}/logs From ded27ec02fb337b9dd8584f5c1bf131a8dced5e4 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 13 Sep 2021 15:59:27 +0000 Subject: [PATCH 026/919] update update --- .../0_stateless/02023_storage_filelog.reference | 12 ++++++------ tests/queries/0_stateless/02023_storage_filelog.sh | 8 ++++++-- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02023_storage_filelog.reference b/tests/queries/0_stateless/02023_storage_filelog.reference index a1ac16d1d55..f1aa3abf257 100644 --- a/tests/queries/0_stateless/02023_storage_filelog.reference +++ b/tests/queries/0_stateless/02023_storage_filelog.reference @@ -8,7 +8,7 @@ 8 8 9 9 10 10 -100 100 +100 100 1 1 2 2 3 3 @@ -19,7 +19,7 @@ 8 8 9 9 10 10 -100 100 +100 100 1 1 1 1 2 2 @@ -40,8 +40,8 @@ 9 9 10 10 10 10 -100 100 -100 100 +100 100 +100 100 1 1 1 1 2 2 @@ -62,6 +62,6 @@ 9 9 10 10 10 10 -100 100 -100 100 +100 100 +100 100 111 111 diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 4987a4b526b..3301c9c394a 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -36,19 +36,23 @@ cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/b.txt ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" +${CLICKHOUSE_CLIENT} --query "drop table if exists mv;" ${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order by k as select * from file_log;" cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/c.txt cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt -sleep 5 +sleep 10 ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" echo 111, 111 >> ${user_files_path}/logs/a.txt -sleep 5 +sleep 10 ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" +${CLICKHOUSE_CLIENT} --query "drop table file_log;" +${CLICKHOUSE_CLIENT} --query "drop table mv;" + rm -rf ${user_files_path}/logs From 6426960af99e6c5232b79ea9c2588bc0b27265a0 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Mon, 13 Sep 2021 21:06:36 +0300 Subject: [PATCH 027/919] add new column --- docs/en/operations/system-tables/replicas.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/operations/system-tables/replicas.md b/docs/en/operations/system-tables/replicas.md index e2cc607f6d8..6cdd1ecdce1 100644 --- a/docs/en/operations/system-tables/replicas.md +++ b/docs/en/operations/system-tables/replicas.md @@ -83,6 +83,7 @@ The next 4 columns have a non-zero value only where there is an active session w - `total_replicas` (`UInt8`) - The total number of known replicas of this table. - `active_replicas` (`UInt8`) - The number of replicas of this table that have a session in ZooKeeper (i.e., the number of functioning replicas). - `replica_is_active` ([Map(String, UInt8)](../../sql-reference/data-types/map.md)) — Map between replica name and is replica active. +- `last_queue_update_exception`- When the queue contains broken entries. Especially important when ClickHouse breaks backward compatibility between versions and log entries written by newer versions aren't parseable by old versions. If you request all the columns, the table may work a bit slowly, since several reads from ZooKeeper are made for each row. If you do not request the last 4 columns (log_max_index, log_pointer, total_replicas, active_replicas), the table works quickly. From 0f030f868fd5f1c2bba58f458da0f02b88017143 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Mon, 13 Sep 2021 21:17:31 +0300 Subject: [PATCH 028/919] add type --- docs/en/operations/system-tables/replicas.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/replicas.md b/docs/en/operations/system-tables/replicas.md index 6cdd1ecdce1..251c7e65537 100644 --- a/docs/en/operations/system-tables/replicas.md +++ b/docs/en/operations/system-tables/replicas.md @@ -83,7 +83,7 @@ The next 4 columns have a non-zero value only where there is an active session w - `total_replicas` (`UInt8`) - The total number of known replicas of this table. - `active_replicas` (`UInt8`) - The number of replicas of this table that have a session in ZooKeeper (i.e., the number of functioning replicas). - `replica_is_active` ([Map(String, UInt8)](../../sql-reference/data-types/map.md)) — Map between replica name and is replica active. -- `last_queue_update_exception`- When the queue contains broken entries. Especially important when ClickHouse breaks backward compatibility between versions and log entries written by newer versions aren't parseable by old versions. +- `last_queue_update_exception` (`String`) - When the queue contains broken entries. Especially important when ClickHouse breaks backward compatibility between versions and log entries written by newer versions aren't parseable by old versions. If you request all the columns, the table may work a bit slowly, since several reads from ZooKeeper are made for each row. If you do not request the last 4 columns (log_max_index, log_pointer, total_replicas, active_replicas), the table works quickly. From 94ba18e050262570c51b94699dce3f5e481a0de4 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 14 Sep 2021 03:15:05 +0000 Subject: [PATCH 029/919] fix --- src/Storages/FileLog/FileLogSource.cpp | 11 ++++++++++- src/Storages/FileLog/ReadBufferFromFileLog.cpp | 11 ++++++++++- src/Storages/FileLog/ReadBufferFromFileLog.h | 5 ++++- 3 files changed, 24 insertions(+), 3 deletions(-) diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index 61410e8488c..08a8c4b6868 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -41,7 +41,7 @@ FileLogSource::FileLogSource( Chunk FileLogSource::generate() { - if (!buffer) + if (!buffer || buffer->noRecords()) return {}; MutableColumns read_columns = non_virtual_header.cloneEmptyColumns(); @@ -110,10 +110,19 @@ Chunk FileLogSource::generate() tryLogCurrentException(__PRETTY_FUNCTION__); } } + else + { + /// No records polled, should break out early, since + /// file status can not be updated during streamToViews + break; + } + if (new_rows) { total_rows = total_rows + new_rows; } + + /// poll succeed, but parse failed else { ++failed_poll_attempts; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index bb43afe08df..bf9cd2d9fce 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -76,13 +76,17 @@ ReadBufferFromFileLog::Records ReadBufferFromFileLog::pollBatch(size_t batch_siz new_records.reserve(batch_size_); readNewRecords(new_records, batch_size); - if (new_records.size() == batch_size_) + if (new_records.size() == batch_size_ || stream_out) return new_records; Stopwatch watch; while (watch.elapsedMilliseconds() < poll_timeout && new_records.size() != batch_size_) { readNewRecords(new_records, batch_size); + /// All ifstrem reach end, no need to wait for timeout, + /// since file status can not be updated during a streamToViews + if (stream_out) + break; } return new_records; @@ -126,6 +130,11 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ if (reader.tellg() == stream_end) { file.status = StorageFileLog::FileStatus::NO_CHANGE; + /// All ifstream reach end + if (i == end - 1) + { + stream_out = true; + } } if (read_records_size == need_records_size) diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index a7447e5edcd..66a8aeaa49f 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -40,16 +40,19 @@ public: private: enum class BufferStatus { + INIT, NO_RECORD_RETURNED, POLLED_OK, }; - BufferStatus buffer_status; + BufferStatus buffer_status = BufferStatus::INIT; Poco::Logger * log; StorageFileLog & storage; + bool stream_out = false; + size_t batch_size; size_t poll_timeout; From 19ed9c6dbc93ad6712abc2b468ceddd410980a44 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 14 Sep 2021 04:50:18 +0000 Subject: [PATCH 030/919] fix --- src/Storages/FileLog/StorageFileLog.cpp | 4 ++-- tests/queries/0_stateless/02023_storage_filelog.sh | 4 ++-- tests/queries/skip_list.json | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 91e6d045fce..eb95a094168 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -392,8 +392,8 @@ bool StorageFileLog::updateFileStatus() case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) - { + if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) + { file_status[event.path].status = FileStatus::UPDATED; } break; diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 3301c9c394a..5a0333b1bc3 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -42,13 +42,13 @@ ${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/c.txt cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt -sleep 10 +sleep 5 ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" echo 111, 111 >> ${user_files_path}/logs/a.txt -sleep 10 +sleep 5 ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 0143cc78dbe..cb3be0c81b5 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -464,7 +464,6 @@ "memory_leak", "memory_limit", "polygon_dicts", // they use an explicitly specified database - "01658_read_file_to_stringcolumn", "01721_engine_file_truncate_on_insert", // It's ok to execute in parallel but not several instances of the same test. "01702_system_query_log", // It's ok to execute in parallel with oter tests but not several instances of the same test. "01748_dictionary_table_dot", // creates database @@ -513,6 +512,7 @@ "01530_drop_database_atomic_sync", /// creates database "02001_add_default_database_to_system_users", ///create user "02002_row_level_filter_bug", ///create user - "02015_system_views" + "02015_system_views", + "02023_storage_filelog" ] } From d6e16715d4b64e8e76bcd12c049d62bcab58e2e3 Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Wed, 15 Sep 2021 14:28:27 +0300 Subject: [PATCH 031/919] Update docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- .../external-dictionaries/external-dicts-dict-layout.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index f0b4eb614c5..7e12656c14a 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -299,7 +299,7 @@ RANGE(MIN StartDate MAX EndDate); Если ключи не были найдены в словаре, то для обновления кэша создается задание и добавляется в очередь обновлений. Параметры очереди обновлений можно устанавливать настройками `max_update_queue_size`, `update_queue_push_timeout_milliseconds`, `query_wait_timeout_milliseconds`, `max_threads_for_updates` -Для cache-словарей может быть задано время устаревания [lifetime](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) данных в кэше. Если от загрузки данных в ячейке прошло больше времени, чем `lifetime`, то значение не используется, а ключ устаревает. Ключ будет запрошен заново при следующей необходимости его использовать. Это можно настроить с помощью `allow_read_expired_keys`. +Для cache-словарей при помощи настройки `allow_read_expired_keys` может быть задано время устаревания [lifetime](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) данных в кэше. Если с момента загрузки данных в ячейку прошло больше времени, чем `lifetime`, то значение не используется, а ключ устаревает. Ключ будет запрошен заново при следующей необходимости его использовать. Это наименее эффективный из всех способов размещения словарей. Скорость работы кэша очень сильно зависит от правильности настройки и сценария использования. Словарь типа `cache` показывает высокую производительность лишь при достаточно большой частоте успешных обращений (рекомендуется 99% и выше). Посмотреть среднюю частоту успешных обращений (`hit rate`) можно в таблице [system.dictionaries](../../../operations/system-tables/dictionaries.md). From c428f433c3a2bfab7fc80036c8e36171ce57bec0 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Thu, 16 Sep 2021 16:38:27 +0100 Subject: [PATCH 032/919] Use do-while control loops for waiting on log processing This allows to use the same functions with very short timeouts while ensuring that the actual state is checked at least once instead of timing out before even looking at at ZK at least once. --- src/Common/ZooKeeper/ZooKeeper.cpp | 8 ++++++-- src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++----- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 220b52104b5..56b96a3a45c 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -730,7 +730,10 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & } }; - while (!condition || !condition()) + /// do-while control structure to allow using this function in non-blocking + /// fashion with a wait condition which returns false by the time this + /// method is called. + do { /// Use getData insteand of exists to avoid watch leak. impl->get(path, callback, watch); @@ -746,7 +749,8 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & if (state->event_type == Coordination::DELETED) return true; - } + } while (!condition || !condition()); + return false; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e0265ec0b09..47d5a1bc895 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5105,8 +5105,7 @@ bool StorageReplicatedMergeTree::tryWaitForReplicaToProcessLogEntry( /// Don't recheck ZooKeeper too often constexpr auto event_wait_timeout_ms = 3000; - if (!startsWith(entry.znode_name, "log-")) - throw Exception("Logical error: unexpected name of log node: " + entry.znode_name, ErrorCodes::LOGICAL_ERROR); + LOG_DEBUG(log, "Waiting for {} to process log entry", replica); { /// Take the number from the node name `log-xxxxxxxxxx`. @@ -5117,7 +5116,7 @@ bool StorageReplicatedMergeTree::tryWaitForReplicaToProcessLogEntry( /// Let's wait until entry gets into the replica queue. bool pulled_to_queue = false; - while (!stop_waiting()) + do { zkutil::EventPtr event = std::make_shared(); @@ -5132,8 +5131,9 @@ bool StorageReplicatedMergeTree::tryWaitForReplicaToProcessLogEntry( /// So log_pointer node will exist, but we will never update it because all background threads already stopped. /// It can lead to query hung because table drop query can wait for some query (alter, optimize, etc) which called this method, /// but the query will never finish because the drop already shut down the table. - event->tryWait(event_wait_timeout_ms); - } + if (!stop_waiting()) + event->tryWait(event_wait_timeout_ms); + } while (!stop_waiting()); if (!pulled_to_queue) return false; From 0381c634d4fbdbfbb1f88bdefc5a3545eda9ca89 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Thu, 16 Sep 2021 16:47:57 +0100 Subject: [PATCH 033/919] Add support for user defined identifier on log entries Sometimes we want to push a log entry once and only once. Because it is not possible to create a sequential node in ZooKeeper and store its name to a well known location in the same transaction we'll do it in the other order. First somehow generate a unique identifier, then submit a log entry with that identifier. Later, we can search through log entries using the identifier we provided to find the node. Required for part movement between shards. --- .../MergeTree/ReplicatedMergeTreeLogEntry.cpp | 10 +++ .../MergeTree/ReplicatedMergeTreeLogEntry.h | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 87 ++++++++++++++++++- 3 files changed, 95 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp index 18e90952721..89515b863ca 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.cpp @@ -26,6 +26,7 @@ enum FormatVersion : UInt8 FORMAT_WITH_DEDUPLICATE = 4, FORMAT_WITH_UUID = 5, FORMAT_WITH_DEDUPLICATE_BY_COLUMNS = 6, + FORMAT_WITH_LOG_ENTRY_ID = 7, FORMAT_LAST }; @@ -43,11 +44,17 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const if (new_part_uuid != UUIDHelpers::Nil) format_version = std::max(format_version, FORMAT_WITH_UUID); + if (!log_entry_id.empty()) + format_version = std::max(format_version, FORMAT_WITH_LOG_ENTRY_ID); + out << "format version: " << format_version << "\n" << "create_time: " << LocalDateTime(create_time ? create_time : time(nullptr)) << "\n" << "source replica: " << source_replica << '\n' << "block_id: " << escape << block_id << '\n'; + if (format_version >= FORMAT_WITH_LOG_ENTRY_ID) + out << "log_entry_id: " << escape << log_entry_id << '\n'; + switch (type) { case GET_PART: @@ -192,6 +199,9 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in) in >> "block_id: " >> escape >> block_id >> "\n"; } + if (format_version >= FORMAT_WITH_LOG_ENTRY_ID) + in >> "log_entry_id: " >> escape >> log_entry_id >> "\n"; + in >> type_str >> "\n"; bool trailing_newline_found = false; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 3752c9deb8f..901669bfbdf 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -77,6 +77,7 @@ struct ReplicatedMergeTreeLogEntryData String toString() const; String znode_name; + String log_entry_id; Type type = EMPTY; String source_replica; /// Empty string means that this entry was added to the queue immediately, and not copied from the log. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 47d5a1bc895..335302e5dda 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5107,6 +5107,7 @@ bool StorageReplicatedMergeTree::tryWaitForReplicaToProcessLogEntry( LOG_DEBUG(log, "Waiting for {} to process log entry", replica); + if (startsWith(entry.znode_name, "log-")) { /// Take the number from the node name `log-xxxxxxxxxx`. UInt64 log_index = parse(entry.znode_name.substr(entry.znode_name.size() - 10)); @@ -5137,9 +5138,76 @@ bool StorageReplicatedMergeTree::tryWaitForReplicaToProcessLogEntry( if (!pulled_to_queue) return false; - } - LOG_DEBUG(log, "Looking for node corresponding to {} in {} queue", log_node_name, replica); + LOG_DEBUG(log, "Looking for node corresponding to {} in {} queue", log_node_name, replica); + } + else if (!entry.log_entry_id.empty()) + { + /// First pass, check the table log. + /// If found in the log, wait for replica to fetch it to the queue. + /// If not found in the log, it is already in the queue. + LOG_DEBUG(log, "Looking for log entry with id `{}` in the log", entry.log_entry_id); + + String log_pointer = getZooKeeper()->get(fs::path(table_zookeeper_path) / "replicas" / replica / "log_pointer"); + + Strings log_entries = getZooKeeper()->getChildren(fs::path(table_zookeeper_path) / "log"); + UInt64 log_index = 0; + bool found = false; + + for (const String & log_entry_name : log_entries) + { + log_index = parse(log_entry_name.substr(log_entry_name.size() - 10)); + + if (!log_pointer.empty() && log_index < parse(log_pointer)) + continue; + + String log_entry_str; + Coordination::Stat log_entry_stat; + bool exists = getZooKeeper()->tryGet(fs::path(table_zookeeper_path) / "log" / log_entry_name, log_entry_str, &log_entry_stat); + ReplicatedMergeTreeLogEntryData log_entry = *ReplicatedMergeTreeLogEntry::parse(log_entry_str, log_entry_stat); + if (exists && entry.log_entry_id == log_entry.log_entry_id) + { + LOG_DEBUG(log, "Found log entry with id `{}` in the log", entry.log_entry_id); + + found = true; + log_node_name = log_entry_name; + break; + } + } + + if (found) + { + LOG_DEBUG(log, "Waiting for {} to pull {} to queue", replica, log_node_name); + + /// Let's wait until entry gets into the replica queue. + bool pulled_to_queue = false; + do + { + zkutil::EventPtr event = std::make_shared(); + + log_pointer = getZooKeeper()->get(fs::path(table_zookeeper_path) / "replicas" / replica / "log_pointer", nullptr, event); + if (!log_pointer.empty() && parse(log_pointer) > log_index) + { + pulled_to_queue = true; + break; + } + + /// Wait with timeout because we can be already shut down, but not dropped. + /// So log_pointer node will exist, but we will never update it because all background threads already stopped. + /// It can lead to query hung because table drop query can wait for some query (alter, optimize, etc) which called this method, + /// but the query will never finish because the drop already shut down the table. + if (!stop_waiting()) + event->tryWait(event_wait_timeout_ms); + } while (!stop_waiting()); + + if (!pulled_to_queue) + return false; + } + } + else + { + throw Exception("Logical error: unexpected name of log node: " + entry.znode_name, ErrorCodes::LOGICAL_ERROR); + } /** Second - find the corresponding entry in the queue of the specified replica. * Its number may not match the `log` node. Therefore, we search by comparing the content. @@ -5151,12 +5219,25 @@ bool StorageReplicatedMergeTree::tryWaitForReplicaToProcessLogEntry( for (const String & entry_name : queue_entries) { String queue_entry_str; - bool exists = getZooKeeper()->tryGet(fs::path(table_zookeeper_path) / "replicas" / replica / "queue" / entry_name, queue_entry_str); + Coordination::Stat queue_entry_stat; + bool exists = getZooKeeper()->tryGet(fs::path(table_zookeeper_path) / "replicas" / replica / "queue" / entry_name, queue_entry_str, &queue_entry_stat); if (exists && queue_entry_str == entry_str) { queue_entry_to_wait_for = entry_name; break; } + else if (!entry.log_entry_id.empty()) + { + /// Check if the id matches rather than just contents. This entry + /// might have been written by different ClickHouse versions and + /// it is hard to guarantee same text representation. + ReplicatedMergeTreeLogEntryData queue_entry = *ReplicatedMergeTreeLogEntry::parse(queue_entry_str, queue_entry_stat); + if (entry.log_entry_id == queue_entry.log_entry_id) + { + queue_entry_to_wait_for = entry_name; + break; + } + } } /// While looking for the record, it has already been executed and deleted. From 5f97cad9a4e1abf200a46a4abc2b26d6a141656a Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 17 Sep 2021 15:32:01 +0000 Subject: [PATCH 034/919] fix --- src/Storages/FileLog/StorageFileLog.cpp | 38 +++++++++++++++---------- 1 file changed, 23 insertions(+), 15 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index eb95a094168..6d380c17f38 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -61,8 +61,9 @@ StorageFileLog::StorageFileLog( if (std::filesystem::is_regular_file(path)) { - file_status[path] = FileContext{}; - file_names.push_back(path); + auto normal_path = std::filesystem::path(path).lexically_normal().native(); + file_status[normal_path] = FileContext{}; + file_names.push_back(normal_path); } else if (std::filesystem::is_directory(path)) { @@ -72,8 +73,9 @@ StorageFileLog::StorageFileLog( { if (dir_entry.is_regular_file()) { - file_status[dir_entry.path()] = FileContext{}; - file_names.push_back(dir_entry.path()); + auto normal_path = std::filesystem::path(dir_entry.path()).lexically_normal().native(); + file_status[normal_path] = FileContext{}; + file_names.push_back(normal_path); } } } @@ -381,31 +383,37 @@ bool StorageFileLog::updateFileStatus() { switch (event.type) { - case Poco::DirectoryWatcher::DW_ITEM_ADDED: + case Poco::DirectoryWatcher::DW_ITEM_ADDED: { + auto normal_path = std::filesystem::path(event.path).lexically_normal().native(); LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - if (std::filesystem::is_regular_file(event.path)) + if (std::filesystem::is_regular_file(normal_path)) { file_status[event.path] = FileContext{}; - file_names.push_back(event.path); + file_names.push_back(normal_path); } break; + } - case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: - LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) + case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: { + auto normal_path = std::filesystem::path(event.path).lexically_normal().native(); + LOG_TRACE(log, "New event {} watched, path: {}", event.callback, normal_path); + if (std::filesystem::is_regular_file(normal_path) && file_status.contains(normal_path)) { - file_status[event.path].status = FileStatus::UPDATED; + file_status[normal_path].status = FileStatus::UPDATED; } break; + } case Poco::DirectoryWatcher::DW_ITEM_REMOVED: case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: - case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: - LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - if (std::filesystem::is_regular_file(event.path) && file_status.contains(event.path)) + case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: { + auto normal_path = std::filesystem::path(event.path).lexically_normal().native(); + LOG_TRACE(log, "New event {} watched, path: {}", event.callback, normal_path); + if (std::filesystem::is_regular_file(normal_path) && file_status.contains(normal_path)) { - file_status[event.path].status = FileStatus::REMOVED; + file_status[normal_path].status = FileStatus::REMOVED; } + } } } std::vector valid_files; From 9a02061d9c77fc0b6a252abf80f7615f2b3a1a13 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Thu, 16 Sep 2021 17:03:31 +0100 Subject: [PATCH 035/919] Rewrite part movement between shards logic and add kill support See https://github.com/ClickHouse/ClickHouse/pull/24585#issuecomment-857735081 for extra context about the current implementation. --- src/Access/AccessType.h | 2 +- .../InterpreterKillQueryQuery.cpp | 67 ++ src/Parsers/ASTKillQueryQuery.cpp | 16 +- src/Parsers/ASTKillQueryQuery.h | 1 + src/Parsers/ParserKillQueryQuery.cpp | 3 + src/Storages/IStorage.h | 6 + .../PartMovesBetweenShardsOrchestrator.cpp | 697 +++++++++++++----- .../PartMovesBetweenShardsOrchestrator.h | 53 +- src/Storages/StorageReplicatedMergeTree.cpp | 12 +- src/Storages/StorageReplicatedMergeTree.h | 3 + .../StorageSystemPartMovesBetweenShards.cpp | 6 +- .../test_part_moves_between_shards/test.py | 409 +++++++++- 12 files changed, 1014 insertions(+), 261 deletions(-) diff --git a/src/Access/AccessType.h b/src/Access/AccessType.h index 8e6a8be8e36..120a97d47b7 100644 --- a/src/Access/AccessType.h +++ b/src/Access/AccessType.h @@ -106,7 +106,7 @@ enum class AccessType (anyone can kill his own queries) */\ \ M(MOVE_PARTITION_BETWEEN_SHARDS, "", GLOBAL, ALL) /* required to be able to move a part/partition to a table - identified by it's ZooKeeper path */\ + identified by its ZooKeeper path */\ \ M(CREATE_USER, "", GLOBAL, ACCESS_MANAGEMENT) \ M(ALTER_USER, "", GLOBAL, ACCESS_MANAGEMENT) \ diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index d43d697fcd5..bfff1f1cd7b 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -31,6 +31,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int ACCESS_DENIED; + extern const int NOT_IMPLEMENTED; } @@ -290,6 +291,72 @@ BlockIO InterpreterKillQueryQuery::execute() break; } + case ASTKillQueryQuery::Type::PartMoveToShard: + { + if (query.sync) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "SYNC modifier is not supported for this statement."); + + Block moves_block = getSelectResult( + "database, table, task_name, task_uuid, part_name, to_shard, state", + "system.part_moves_between_shards"); + + if (!moves_block) + return res_io; + + const ColumnString & database_col = typeid_cast(*moves_block.getByName("database").column); + const ColumnString & table_col = typeid_cast(*moves_block.getByName("table").column); + const ColumnUUID & task_uuid_col = typeid_cast(*moves_block.getByName("task_uuid").column); + + auto header = moves_block.cloneEmpty(); + header.insert(0, {ColumnString::create(), std::make_shared(), "kill_status"}); + + MutableColumns res_columns = header.cloneEmptyColumns(); + auto table_id = StorageID::createEmpty(); + AccessRightsElements required_access_rights; + auto access = getContext()->getAccess(); + bool access_denied = false; + + for (size_t i = 0; i < moves_block.rows(); ++i) + { + table_id = StorageID{database_col.getDataAt(i).toString(), table_col.getDataAt(i).toString()}; + auto task_uuid = get(task_uuid_col[i]); + + CancellationCode code = CancellationCode::Unknown; + + if (!query.test) + { + auto storage = DatabaseCatalog::instance().tryGetTable(table_id, getContext()); + if (!storage) + code = CancellationCode::NotFound; + else + { + ASTAlterCommand alter_command{}; + alter_command.type = ASTAlterCommand::MOVE_PARTITION; + alter_command.move_destination_type = DataDestinationType::SHARD; + required_access_rights = InterpreterAlterQuery::getRequiredAccessForCommand( + alter_command, table_id.database_name, table_id.table_name); + if (!access->isGranted(required_access_rights)) + { + access_denied = true; + continue; + } + code = storage->killPartMoveToShard(task_uuid); + } + } + + insertResultRow(i, code, moves_block, header, res_columns); + } + + if (res_columns[0]->empty() && access_denied) + throw Exception( + "Not allowed to kill move partition. To execute this query it's necessary to have the grant " + required_access_rights.toString(), + ErrorCodes::ACCESS_DENIED); + + + res_io.in = std::make_shared(header.cloneWithColumns(std::move(res_columns))); + + break; + } } return res_io; diff --git a/src/Parsers/ASTKillQueryQuery.cpp b/src/Parsers/ASTKillQueryQuery.cpp index 72bdd7d6b0b..71c3011dd2c 100644 --- a/src/Parsers/ASTKillQueryQuery.cpp +++ b/src/Parsers/ASTKillQueryQuery.cpp @@ -11,8 +11,20 @@ String ASTKillQueryQuery::getID(char delim) const void ASTKillQueryQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "KILL " - << (type == Type::Query ? "QUERY" : "MUTATION"); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "KILL "; + + switch (type) + { + case Type::Query: + settings.ostr << "QUERY"; + break; + case Type::Mutation: + settings.ostr << "MUTATION"; + break; + case Type::PartMoveToShard: + settings.ostr << "PART_MOVE_TO_SHARD"; + break; + } formatOnCluster(settings); diff --git a/src/Parsers/ASTKillQueryQuery.h b/src/Parsers/ASTKillQueryQuery.h index c1b3956962f..6ff12bcba93 100644 --- a/src/Parsers/ASTKillQueryQuery.h +++ b/src/Parsers/ASTKillQueryQuery.h @@ -13,6 +13,7 @@ public: { Query, /// KILL QUERY Mutation, /// KILL MUTATION + PartMoveToShard, /// KILL PART_MOVE_TO_SHARD }; Type type = Type::Query; diff --git a/src/Parsers/ParserKillQueryQuery.cpp b/src/Parsers/ParserKillQueryQuery.cpp index a195a778ed2..bc895406c9f 100644 --- a/src/Parsers/ParserKillQueryQuery.cpp +++ b/src/Parsers/ParserKillQueryQuery.cpp @@ -17,6 +17,7 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect ParserKeyword p_kill{"KILL"}; ParserKeyword p_query{"QUERY"}; ParserKeyword p_mutation{"MUTATION"}; + ParserKeyword p_part_move_to_shard{"PART_MOVE_TO_SHARD"}; ParserKeyword p_on{"ON"}; ParserKeyword p_test{"TEST"}; ParserKeyword p_sync{"SYNC"}; @@ -31,6 +32,8 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect query->type = ASTKillQueryQuery::Type::Query; else if (p_mutation.ignore(pos, expected)) query->type = ASTKillQueryQuery::Type::Mutation; + else if (p_part_move_to_shard.ignore(pos, expected)) + query->type = ASTKillQueryQuery::Type::PartMoveToShard; else return false; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 90cb963e064..9c340100469 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -460,6 +460,12 @@ public: throw Exception("Mutations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } + /// Cancel a part move to shard. + virtual CancellationCode killPartMoveToShard(const UUID & /*task_uuid*/) + { + throw Exception("Part moves between shards are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + /** If the table have to do some complicated work on startup, * that must be postponed after creation of table object * (like launching some background threads), diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp index c227febbbc2..b3a17250549 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -1,13 +1,20 @@ #include #include #include -#include #include #include #include namespace DB { + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; + extern const int TIMEOUT_EXCEEDED; +} + PartMovesBetweenShardsOrchestrator::PartMovesBetweenShardsOrchestrator(StorageReplicatedMergeTree & storage_) : storage(storage_) , zookeeper_path(storage.zookeeper_path) @@ -27,16 +34,17 @@ void PartMovesBetweenShardsOrchestrator::run() if (need_stop) return; - auto sleep_ms = 10; + /// Don't poll ZooKeeper too often. + auto sleep_ms = 3 * 1000; try { - fetchStateFromZK(); + syncStateFromZK(); + /// Schedule for immediate re-execution as likely there is more work + /// to be done. if (step()) - fetchStateFromZK(); - else - sleep_ms = 3 * 1000; + task->schedule(); } catch (...) { @@ -54,11 +62,11 @@ void PartMovesBetweenShardsOrchestrator::shutdown() LOG_TRACE(log, "PartMovesBetweenShardsOrchestrator thread finished"); } -void PartMovesBetweenShardsOrchestrator::fetchStateFromZK() +void PartMovesBetweenShardsOrchestrator::syncStateFromZK() { std::lock_guard lock(state_mutex); - entries.clear(); + std::vector new_entries; auto zk = storage.getZooKeeper(); @@ -76,8 +84,11 @@ void PartMovesBetweenShardsOrchestrator::fetchStateFromZK() e.version = stat.version; e.znode_name = task_name; - entries[task_name] = std::move(e); + new_entries.push_back(std::move(e)); } + + // Replace in-memory state. + entries = new_entries; } bool PartMovesBetweenShardsOrchestrator::step() @@ -93,7 +104,7 @@ bool PartMovesBetweenShardsOrchestrator::step() { std::lock_guard lock(state_mutex); - for (auto const & entry : entries | boost::adaptors::map_values) + for (auto const & entry : entries) { if (entry.state.value == EntryState::DONE || entry.state.value == EntryState::CANCELLED) continue; @@ -128,11 +139,21 @@ bool PartMovesBetweenShardsOrchestrator::step() throw; } + LOG_DEBUG(log, "stepEntry on task {} from state {} (rollback: {}), try: {}", + entry_to_process->znode_name, + entry_to_process->state.toString(), + entry_to_process->rollback, + entry_to_process->num_tries); + try { /// Use the same ZooKeeper connection. If we'd lost the lock then connection /// will become expired and all consequent operations will fail. - stepEntry(entry_to_process.value(), zk); + Entry new_entry = stepEntry(entry_to_process.value(), zk); + new_entry.last_exception_msg = ""; + new_entry.num_tries = 0; + new_entry.update_time = std::time(nullptr); + zk->set(new_entry.znode_path, new_entry.toString(), new_entry.version); } catch (...) { @@ -140,6 +161,7 @@ bool PartMovesBetweenShardsOrchestrator::step() Entry entry_copy = entry_to_process.value(); entry_copy.last_exception_msg = getCurrentExceptionMessage(false); + entry_copy.num_tries += 1; entry_copy.update_time = std::time(nullptr); zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); @@ -149,276 +171,537 @@ bool PartMovesBetweenShardsOrchestrator::step() return true; } -void PartMovesBetweenShardsOrchestrator::stepEntry(const Entry & entry, zkutil::ZooKeeperPtr zk) +PartMovesBetweenShardsOrchestrator::Entry PartMovesBetweenShardsOrchestrator::stepEntry(Entry entry, zkutil::ZooKeeperPtr zk) { switch (entry.state.value) { - case EntryState::DONE: - break; - + case EntryState::DONE: [[fallthrough]]; case EntryState::CANCELLED: - break; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't stepEntry after terminal state. This is a bug."); case EntryState::TODO: { - /// State transition. - Entry entry_copy = entry; - entry_copy.state = EntryState::SYNC_SOURCE; - entry_copy.update_time = std::time(nullptr); - zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + if (entry.rollback) + { + removePins(entry, zk); + entry.state = EntryState::CANCELLED; + return entry; + } + /// The forward transition happens implicitly when task is created by `StorageReplicatedMergeTree::movePartitionToShard`. + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected entry state ({}) in stepEntry. This is a bug.", entry.state.toString()); } - break; case EntryState::SYNC_SOURCE: { + if (entry.rollback) { - /// Log entry. - Coordination::Requests ops; - ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); - - ReplicatedMergeTreeLogEntryData log_entry; - log_entry.type = ReplicatedMergeTreeLogEntryData::SYNC_PINNED_PART_UUIDS; - log_entry.create_time = std::time(nullptr); - log_entry.source_replica = storage.replica_name; - ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); - ops.emplace_back(zkutil::makeCreateRequest( - zookeeper_path + "/log/log-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); - - Coordination::Responses responses; - Coordination::Error rc = zk->tryMulti(ops, responses); - zkutil::KeeperMultiException::check(rc, ops, responses); - - String log_znode_path = dynamic_cast(*responses.back()).path_created; - log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); - - /// This wait in background schedule pool is useless. It'd be - /// better to have some notification which will call `step` - /// function when all replicated will finish. TODO. - storage.waitForAllReplicasToProcessLogEntry(zookeeper_path, log_entry, -1); + entry.state = EntryState::TODO; + return entry; } - + else { - /// State transition. - Entry entry_copy = entry; - entry_copy.state = EntryState::SYNC_DESTINATION; - entry_copy.update_time = std::time(nullptr); - zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + ReplicatedMergeTreeLogEntryData sync_source_log_entry; + + String sync_source_log_entry_barrier_path = fs::path(entry.znode_path) / ("log_" + entry.state.toString()); + Coordination::Stat sync_source_log_entry_stat; + String sync_source_log_entry_str; + if (zk->tryGet(sync_source_log_entry_barrier_path, sync_source_log_entry_str, &sync_source_log_entry_stat)) + { + LOG_DEBUG(log, "Log entry was already created will check the existing one."); + + sync_source_log_entry = *ReplicatedMergeTreeLogEntry::parse(sync_source_log_entry_str, sync_source_log_entry_stat); + } + else + { + /// Log entry. + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); + + sync_source_log_entry.type = ReplicatedMergeTreeLogEntryData::SYNC_PINNED_PART_UUIDS; + sync_source_log_entry.log_entry_id = sync_source_log_entry_barrier_path; + sync_source_log_entry.create_time = std::time(nullptr); + sync_source_log_entry.source_replica = storage.replica_name; + + ops.emplace_back(zkutil::makeCreateRequest(sync_source_log_entry_barrier_path, sync_source_log_entry.toString(), -1)); + ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); + ops.emplace_back(zkutil::makeCreateRequest( + zookeeper_path + "/log/log-", sync_source_log_entry.toString(), zkutil::CreateMode::PersistentSequential)); + + Coordination::Responses responses; + Coordination::Error rc = zk->tryMulti(ops, responses); + zkutil::KeeperMultiException::check(rc, ops, responses); + + String log_znode_path = dynamic_cast(*responses.back()).path_created; + sync_source_log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + + LOG_DEBUG(log, "Pushed log entry: {}", log_znode_path); + } + + Strings unwaited = storage.tryWaitForAllReplicasToProcessLogEntry(zookeeper_path, sync_source_log_entry, 1); + if (!unwaited.empty()) + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Some replicas haven't processed event: {}, will retry later.", toString(unwaited)); + + entry.state = EntryState::SYNC_DESTINATION; + return entry; } } - break; case EntryState::SYNC_DESTINATION: { + if (entry.rollback) { - /// Log entry. - Coordination::Requests ops; - ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); - - ReplicatedMergeTreeLogEntryData log_entry; - log_entry.type = ReplicatedMergeTreeLogEntryData::SYNC_PINNED_PART_UUIDS; - log_entry.create_time = std::time(nullptr); - log_entry.source_replica = storage.replica_name; - log_entry.source_shard = zookeeper_path; - - ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/log", "", -1)); - ops.emplace_back(zkutil::makeCreateRequest( - entry.to_shard + "/log/log-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); - - Coordination::Responses responses; - Coordination::Error rc = zk->tryMulti(ops, responses); - zkutil::KeeperMultiException::check(rc, ops, responses); - - String log_znode_path = dynamic_cast(*responses.back()).path_created; - log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); - - storage.waitForAllReplicasToProcessLogEntry(entry.to_shard, log_entry, -1); - } - - { - /// State transition. Entry entry_copy = entry; - entry_copy.state = EntryState::DESTINATION_FETCH; - entry_copy.update_time = std::time(nullptr); - zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + entry_copy.state = EntryState::SYNC_SOURCE; + return entry_copy; + } + else + { + ReplicatedMergeTreeLogEntryData sync_destination_log_entry; + + String sync_destination_log_entry_barrier_path = fs::path(entry.znode_path) / ("log_" + entry.state.toString()); + Coordination::Stat sync_destination_log_entry_stat; + String sync_destination_log_entry_str; + if (zk->tryGet(sync_destination_log_entry_barrier_path, sync_destination_log_entry_str, &sync_destination_log_entry_stat)) + { + LOG_DEBUG(log, "Log entry was already created will check the existing one."); + + sync_destination_log_entry = *ReplicatedMergeTreeLogEntry::parse(sync_destination_log_entry_str, sync_destination_log_entry_stat); + } + else + { + /// Log entry. + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); + + sync_destination_log_entry.type = ReplicatedMergeTreeLogEntryData::SYNC_PINNED_PART_UUIDS; + sync_destination_log_entry.log_entry_id = sync_destination_log_entry_barrier_path; + sync_destination_log_entry.create_time = std::time(nullptr); + sync_destination_log_entry.source_replica = storage.replica_name; + sync_destination_log_entry.source_shard = zookeeper_path; + + ops.emplace_back(zkutil::makeCreateRequest(sync_destination_log_entry_barrier_path, sync_destination_log_entry.toString(), -1)); + ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/log", "", -1)); + ops.emplace_back(zkutil::makeCreateRequest( + entry.to_shard + "/log/log-", sync_destination_log_entry.toString(), zkutil::CreateMode::PersistentSequential)); + + Coordination::Responses responses; + Coordination::Error rc = zk->tryMulti(ops, responses); + zkutil::KeeperMultiException::check(rc, ops, responses); + + String log_znode_path = dynamic_cast(*responses.back()).path_created; + sync_destination_log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + + LOG_DEBUG(log, "Pushed log entry: {}", log_znode_path); + } + + Strings unwaited = storage.tryWaitForAllReplicasToProcessLogEntry(entry.to_shard, sync_destination_log_entry, 1); + if (!unwaited.empty()) + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Some replicas haven't processed event: {}, will retry later.", toString(unwaited)); + + entry.state = EntryState::DESTINATION_FETCH; + return entry; } } - break; case EntryState::DESTINATION_FETCH: { - /// Make sure table structure doesn't change when there are part movements in progress. + + if (entry.rollback) { - Coordination::Requests ops; - ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); + // TODO(nv): Do we want to cleanup fetched data on the destination? + // Maybe leave it there and make sure a background cleanup will take + // care of it sometime later. - /// Log entry. - ReplicatedMergeTreeLogEntryData log_entry; - log_entry.type = ReplicatedMergeTreeLogEntryData::CLONE_PART_FROM_SHARD; - log_entry.create_time = std::time(nullptr); - log_entry.new_part_name = entry.part_name; - log_entry.source_replica = storage.replica_name; - log_entry.source_shard = zookeeper_path; - ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/log", "", -1)); - ops.emplace_back(zkutil::makeCreateRequest( - entry.to_shard + "/log/log-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); - - Coordination::Responses responses; - Coordination::Error rc = zk->tryMulti(ops, responses); - zkutil::KeeperMultiException::check(rc, ops, responses); - - String log_znode_path = dynamic_cast(*responses.back()).path_created; - log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); - - storage.waitForAllReplicasToProcessLogEntry(entry.to_shard, log_entry, -1); + entry.state = EntryState::SYNC_DESTINATION; + return entry; } - + else { - /// State transition. - Entry entry_copy = entry; - entry_copy.state = EntryState::DESTINATION_ATTACH; - entry_copy.update_time = std::time(nullptr); - zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + /// Note: Table structure shouldn't be changed while there are part movements in progress. + + ReplicatedMergeTreeLogEntryData fetch_log_entry; + + String fetch_log_entry_barrier_path = fs::path(entry.znode_path) / ("log_" + entry.state.toString()); + Coordination::Stat fetch_log_entry_stat; + String fetch_log_entry_str; + if (zk->tryGet(fetch_log_entry_barrier_path, fetch_log_entry_str, &fetch_log_entry_stat)) + { + LOG_DEBUG(log, "Log entry was already created will check the existing one."); + + fetch_log_entry = *ReplicatedMergeTreeLogEntry::parse(fetch_log_entry_str, fetch_log_entry_stat); + } + else + { + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); + + fetch_log_entry.type = ReplicatedMergeTreeLogEntryData::CLONE_PART_FROM_SHARD; + fetch_log_entry.log_entry_id = fetch_log_entry_barrier_path; + fetch_log_entry.create_time = std::time(nullptr); + fetch_log_entry.new_part_name = entry.part_name; + fetch_log_entry.source_replica = storage.replica_name; + fetch_log_entry.source_shard = zookeeper_path; + ops.emplace_back(zkutil::makeCreateRequest(fetch_log_entry_barrier_path, fetch_log_entry.toString(), -1)); + ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/log", "", -1)); + ops.emplace_back(zkutil::makeCreateRequest( + entry.to_shard + "/log/log-", fetch_log_entry.toString(), zkutil::CreateMode::PersistentSequential)); + + Coordination::Responses responses; + Coordination::Error rc = zk->tryMulti(ops, responses); + zkutil::KeeperMultiException::check(rc, ops, responses); + + String log_znode_path = dynamic_cast(*responses.back()).path_created; + fetch_log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + + LOG_DEBUG(log, "Pushed log entry: {}", log_znode_path); + } + + Strings unwaited = storage.tryWaitForAllReplicasToProcessLogEntry(entry.to_shard, fetch_log_entry, 1); + if (!unwaited.empty()) + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Some replicas haven't processed event: {}, will retry later.", toString(unwaited)); + + entry.state = EntryState::DESTINATION_ATTACH; + return entry; } } - break; case EntryState::DESTINATION_ATTACH: { - /// There is a chance that attach on destination will fail and this task will be left in the queue forever. + String attach_log_entry_barrier_path = fs::path(entry.znode_path) / ("log_" + entry.state.toString()); + + if (entry.rollback) { + Coordination::Stat attach_log_entry_stat; + String attach_log_entry_str; + if (!zk->tryGet(attach_log_entry_barrier_path, attach_log_entry_str, &attach_log_entry_stat)) + { + LOG_DEBUG(log, "Log entry for DESTINATION_ATTACH not found. Not sending DROP_RANGE log entry."); + + // ATTACH_PART wasn't issued, nothing to revert. + entry.state = EntryState::DESTINATION_FETCH; + return entry; + } + else + { + // Need to remove ATTACH_PART from the queue or drop data. + // Similar to `StorageReplicatedMergeTree::dropPart` w/o extra + // checks as we know drop shall be possible. + ReplicatedMergeTreeLogEntryData attach_rollback_log_entry; + + String attach_rollback_log_entry_barrier_path = fs::path(entry.znode_path) / ("log_" + entry.state.toString() + "_rollback"); + Coordination::Stat attach_rollback_log_entry_stat; + String attach_rollback_log_entry_str; + if (zk->tryGet(attach_rollback_log_entry_barrier_path, attach_rollback_log_entry_str, &attach_rollback_log_entry_stat)) + { + LOG_DEBUG(log, "Log entry was already created will check the existing one."); + + attach_rollback_log_entry = *ReplicatedMergeTreeLogEntry::parse(attach_rollback_log_entry_str, attach_rollback_log_entry_stat); + } + else + { + const auto attach_log_entry = ReplicatedMergeTreeLogEntry::parse(attach_log_entry_str, attach_log_entry_stat); + + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); + + auto drop_part_info = MergeTreePartInfo::fromPartName(attach_log_entry->new_part_name, storage.format_version); + + storage.getClearBlocksInPartitionOps( + ops, *zk, drop_part_info.partition_id, drop_part_info.min_block, drop_part_info.max_block); + size_t clear_block_ops_size = ops.size(); + + attach_rollback_log_entry.type = ReplicatedMergeTreeLogEntryData::DROP_RANGE; + attach_rollback_log_entry.log_entry_id = attach_rollback_log_entry_barrier_path; + attach_rollback_log_entry.source_replica = storage.replica_name; + attach_rollback_log_entry.source_shard = zookeeper_path; + + attach_rollback_log_entry.new_part_name = getPartNamePossiblyFake(storage.format_version, drop_part_info); + attach_rollback_log_entry.create_time = time(nullptr); + + ops.emplace_back(zkutil::makeCreateRequest(attach_rollback_log_entry_barrier_path, attach_rollback_log_entry.toString(), -1)); + ops.emplace_back(zkutil::makeCreateRequest( + entry.to_shard + "/log/log-", attach_rollback_log_entry.toString(), zkutil::CreateMode::PersistentSequential)); + ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/log", "", -1)); + Coordination::Responses responses; + Coordination::Error rc = zk->tryMulti(ops, responses); + zkutil::KeeperMultiException::check(rc, ops, responses); + + String log_znode_path + = dynamic_cast(*responses[clear_block_ops_size]).path_created; + attach_rollback_log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + + LOG_DEBUG(log, "Pushed log entry: {}", log_znode_path); + } + + Strings unwaited = storage.tryWaitForAllReplicasToProcessLogEntry(entry.to_shard, attach_rollback_log_entry, 1); + if (!unwaited.empty()) + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Some replicas haven't processed event: {}, will retry later.", toString(unwaited)); + + entry.state = EntryState::DESTINATION_FETCH; + return entry; + } + } + else + { + /// There is a chance that attach on destination will fail and this task will be left in the queue forever. + Coordination::Requests ops; ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); auto part = storage.getActiveContainingPart(entry.part_name); + /// Allocating block number in other replicas zookeeper path /// TODO Maybe we can do better. - auto block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zk, "", entry.to_shard); - auto block_number = block_number_lock->getNumber(); + auto block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zk, attach_log_entry_barrier_path, entry.to_shard); - auto part_info = part->info; - part_info.min_block = block_number; - part_info.max_block = block_number; - part_info.level = 0; - part_info.mutation = 0; - - /// Attach log entry (all replicas already fetched part) ReplicatedMergeTreeLogEntryData log_entry; - log_entry.type = ReplicatedMergeTreeLogEntryData::ATTACH_PART; - log_entry.part_checksum = part->checksums.getTotalChecksumHex(); - log_entry.create_time = std::time(nullptr); - log_entry.new_part_name = part_info.getPartName(); - ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/log", "", -1)); - ops.emplace_back(zkutil::makeCreateRequest( - entry.to_shard + "/log/log-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); - Coordination::Responses responses; - Coordination::Error rc = zk->tryMulti(ops, responses); - zkutil::KeeperMultiException::check(rc, ops, responses); + if (block_number_lock) + { + auto block_number = block_number_lock->getNumber(); - String log_znode_path = dynamic_cast(*responses.back()).path_created; - log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + auto part_info = part->info; + part_info.min_block = block_number; + part_info.max_block = block_number; + part_info.level = 0; + part_info.mutation = 0; - storage.waitForAllReplicasToProcessLogEntry(entry.to_shard, log_entry, -1); - } + /// Attach log entry (all replicas already fetched part) + log_entry.type = ReplicatedMergeTreeLogEntryData::ATTACH_PART; + log_entry.log_entry_id = attach_log_entry_barrier_path; + log_entry.part_checksum = part->checksums.getTotalChecksumHex(); + log_entry.create_time = std::time(nullptr); + log_entry.new_part_name = part_info.getPartName(); - { - /// State transition. - Entry entry_copy = entry; - entry_copy.state = EntryState::SOURCE_DROP_PRE_DELAY; - entry_copy.update_time = std::time(nullptr); - zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + ops.emplace_back(zkutil::makeCreateRequest(attach_log_entry_barrier_path, log_entry.toString(), -1)); + ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/log", "", -1)); + ops.emplace_back(zkutil::makeCreateRequest( + entry.to_shard + "/log/log-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); + + Coordination::Responses responses; + Coordination::Error rc = zk->tryMulti(ops, responses); + zkutil::KeeperMultiException::check(rc, ops, responses); + + String log_znode_path = dynamic_cast(*responses.back()).path_created; + log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + + LOG_DEBUG(log, "Pushed log entry: {}", log_znode_path); + } + else + { + LOG_DEBUG(log, "Log entry was already created will check the existing one."); + + Coordination::Stat stat; + String log_entry_str = zk->get(attach_log_entry_barrier_path, &stat); + log_entry = *ReplicatedMergeTreeLogEntry::parse(log_entry_str, stat); + } + + Strings unwaited = storage.tryWaitForAllReplicasToProcessLogEntry(entry.to_shard, log_entry, 1); + if (!unwaited.empty()) + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Some replicas haven't processed event: {}, will retry later.", toString(unwaited)); + + + entry.dst_part_name = log_entry.new_part_name; + entry.state = EntryState::SOURCE_DROP_PRE_DELAY; + return entry; } } - break; case EntryState::SOURCE_DROP_PRE_DELAY: { - std::this_thread::sleep_for(std::chrono::seconds(storage.getSettings()->part_moves_between_shards_delay_seconds)); - - /// State transition. - Entry entry_copy = entry; - entry_copy.state = EntryState::SOURCE_DROP; - entry_copy.update_time = std::time(nullptr); - zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + if (entry.rollback) + { + entry.state = EntryState::DESTINATION_ATTACH; + return entry; + } + else + { + std::this_thread::sleep_for(std::chrono::seconds(storage.getSettings()->part_moves_between_shards_delay_seconds)); + entry.state = EntryState::SOURCE_DROP; + return entry; + } } - break; case EntryState::SOURCE_DROP: { + if (entry.rollback) + throw Exception(ErrorCodes::LOGICAL_ERROR, "It is not possible to rollback from this state. This is a bug."); + else { - ReplicatedMergeTreeLogEntry log_entry; - if (storage.dropPartImpl(zk, entry.part_name, log_entry, false, false)) - storage.waitForAllReplicasToProcessLogEntry(zookeeper_path, log_entry, -1); - } + // Can't use dropPartImpl directly as we need additional zk ops to remember the log entry + // for subsequent retries. - { - /// State transition. - Entry entry_copy = entry; - entry_copy.state = EntryState::SOURCE_DROP_POST_DELAY; - entry_copy.update_time = std::time(nullptr); - zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + ReplicatedMergeTreeLogEntryData source_drop_log_entry; + + String source_drop_log_entry_barrier_path = fs::path(entry.znode_path) / ("log_" + entry.state.toString()); + Coordination::Stat source_drop_log_entry_stat; + String source_drop_log_entry_str; + if (zk->tryGet(source_drop_log_entry_barrier_path, source_drop_log_entry_str, &source_drop_log_entry_stat)) + { + LOG_DEBUG(log, "Log entry was already created will check the existing one."); + + source_drop_log_entry = *ReplicatedMergeTreeLogEntry::parse(source_drop_log_entry_str, source_drop_log_entry_stat); + } + else + { + auto source_drop_part_info = MergeTreePartInfo::fromPartName(entry.part_name, storage.format_version); + + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(entry.znode_path, entry.version)); + + storage.getClearBlocksInPartitionOps(ops, *zk, source_drop_part_info.partition_id, source_drop_part_info.min_block, source_drop_part_info.max_block); + + source_drop_log_entry.type = ReplicatedMergeTreeLogEntryData::DROP_RANGE; + source_drop_log_entry.log_entry_id = source_drop_log_entry_barrier_path; + source_drop_log_entry.create_time = std::time(nullptr); + source_drop_log_entry.new_part_name = getPartNamePossiblyFake(storage.format_version, source_drop_part_info); + source_drop_log_entry.source_replica = storage.replica_name; + + ops.emplace_back(zkutil::makeCreateRequest(source_drop_log_entry_barrier_path, source_drop_log_entry.toString(), -1)); + ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/log", "", -1)); + ops.emplace_back(zkutil::makeCreateRequest( + zookeeper_path + "/log/log-", source_drop_log_entry.toString(), zkutil::CreateMode::PersistentSequential)); + + Coordination::Responses responses; + Coordination::Error rc = zk->tryMulti(ops, responses); + zkutil::KeeperMultiException::check(rc, ops, responses); + + String log_znode_path = dynamic_cast(*responses.back()).path_created; + source_drop_log_entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1); + + LOG_DEBUG(log, "Pushed log entry: {}", log_znode_path); + } + + Strings unwaited = storage.tryWaitForAllReplicasToProcessLogEntry(zookeeper_path, source_drop_log_entry, 1); + if (!unwaited.empty()) + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Some replicas haven't processed event: {}, will retry later.", toString(unwaited)); + + entry.state = EntryState::SOURCE_DROP_POST_DELAY; + return entry; } } - break; case EntryState::SOURCE_DROP_POST_DELAY: { - std::this_thread::sleep_for(std::chrono::seconds(storage.getSettings()->part_moves_between_shards_delay_seconds)); - - /// State transition. - Entry entry_copy = entry; - entry_copy.state = EntryState::REMOVE_UUID_PIN; - entry_copy.update_time = std::time(nullptr); - zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); + if (entry.rollback) + throw Exception(ErrorCodes::LOGICAL_ERROR, "It is not possible to rollback from this state. This is a bug."); + else + { + std::this_thread::sleep_for(std::chrono::seconds(storage.getSettings()->part_moves_between_shards_delay_seconds)); + entry.state = EntryState::REMOVE_UUID_PIN; + return entry; + } } - break; case EntryState::REMOVE_UUID_PIN: { + if (entry.rollback) + throw Exception(ErrorCodes::LOGICAL_ERROR, "It is not possible to rollback from this state. This is a bug."); + else { - PinnedPartUUIDs src_pins; - PinnedPartUUIDs dst_pins; + removePins(entry, zk); - { - String s = zk->get(zookeeper_path + "/pinned_part_uuids", &src_pins.stat); - src_pins.fromString(s); - } - - { - String s = zk->get(entry.to_shard + "/pinned_part_uuids", &dst_pins.stat); - dst_pins.fromString(s); - } - - src_pins.part_uuids.erase(entry.part_uuid); - dst_pins.part_uuids.erase(entry.part_uuid); - - Coordination::Requests ops; - ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/pinned_part_uuids", src_pins.toString(), src_pins.stat.version)); - ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/pinned_part_uuids", dst_pins.toString(), dst_pins.stat.version)); - - zk->multi(ops); + entry.state = EntryState::DONE; + return entry; } - - /// State transition. - Entry entry_copy = entry; - entry_copy.state = EntryState::DONE; - entry_copy.update_time = std::time(nullptr); - zk->set(entry_copy.znode_path, entry_copy.toString(), entry_copy.version); } - break; + } + + __builtin_unreachable(); +} + +void PartMovesBetweenShardsOrchestrator::removePins(const Entry & entry, zkutil::ZooKeeperPtr zk) +{ + PinnedPartUUIDs src_pins; + PinnedPartUUIDs dst_pins; + + { + String s = zk->get(zookeeper_path + "/pinned_part_uuids", &src_pins.stat); + src_pins.fromString(s); + } + + { + String s = zk->get(entry.to_shard + "/pinned_part_uuids", &dst_pins.stat); + dst_pins.fromString(s); + } + + dst_pins.part_uuids.erase(entry.part_uuid); + src_pins.part_uuids.erase(entry.part_uuid); + + Coordination::Requests ops; + ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/pinned_part_uuids", src_pins.toString(), src_pins.stat.version)); + ops.emplace_back(zkutil::makeSetRequest(entry.to_shard + "/pinned_part_uuids", dst_pins.toString(), dst_pins.stat.version)); + + zk->multi(ops); +} + +CancellationCode PartMovesBetweenShardsOrchestrator::killPartMoveToShard(const UUID & task_uuid) +{ + while (true) + { + auto entry = getEntryByUUID(task_uuid); + + // If the task is in this state or any that follows it is too late to rollback + // since we can't be sure if the source data still exists. + auto not_possible_to_rollback_after_state = EntryState(EntryState::SOURCE_DROP); + if (entry.state.value >= not_possible_to_rollback_after_state.value) + { + LOG_DEBUG(log, "Can't kill move part between shards entry {} ({}) after state {}. Current state: {}.", + toString(entry.task_uuid), entry.znode_name, not_possible_to_rollback_after_state.toString(), entry.state.toString()); + return CancellationCode::CancelCannotBeSent; + } + + LOG_TRACE(log, "Will try to mark move part between shards entry {} ({}) for rollback.", + toString(entry.task_uuid), entry.znode_name); + + auto zk = storage.getZooKeeper(); + + // State transition. + entry.rollback = true; + entry.update_time = std::time(nullptr); + entry.num_tries = 0; + entry.last_exception_msg = ""; + + auto code = zk->trySet(entry.znode_path, entry.toString(), entry.version); + if (code == Coordination::Error::ZOK) + { + // Orchestrator will process it in background. + return CancellationCode::CancelSent; + } + else if (code == Coordination::Error::ZBADVERSION) + { + /// Node was updated meanwhile. We must re-read it and repeat all the actions. + continue; + } + else + throw Coordination::Exception(code, entry.znode_path); } } -std::vector PartMovesBetweenShardsOrchestrator::getEntries() const +std::vector PartMovesBetweenShardsOrchestrator::getEntries() { + // Force sync. Also catches parsing errors. + syncStateFromZK(); + std::lock_guard lock(state_mutex); - std::vector res; + return entries; +} - for (const auto & e : entries) - res.push_back(e.second); +PartMovesBetweenShardsOrchestrator::Entry PartMovesBetweenShardsOrchestrator::getEntryByUUID(const UUID & task_uuid) +{ + /// Need latest state in case user tries to kill a move observed on a different replica. + syncStateFromZK(); - return res; + std::lock_guard lock(state_mutex); + for (auto const & entry : entries) + { + if (entry.task_uuid == task_uuid) + return entry; + } + + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Task with id {} not found", toString(task_uuid)); } String PartMovesBetweenShardsOrchestrator::Entry::toString() const @@ -431,12 +714,19 @@ String PartMovesBetweenShardsOrchestrator::Entry::toString() const json.set(JSON_KEY_PART_NAME, part_name); json.set(JSON_KEY_PART_UUID, DB::toString(part_uuid)); json.set(JSON_KEY_TO_SHARD, to_shard); + json.set(JSON_KEY_DST_PART_NAME, dst_part_name); json.set(JSON_KEY_STATE, state.toString()); + json.set(JSON_KEY_ROLLBACK, DB::toString(rollback)); json.set(JSON_KEY_LAST_EX_MSG, last_exception_msg); + json.set(JSON_KEY_NUM_TRIES, DB::toString(num_tries)); std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); - json.stringify(oss); + + // Always escape unicode to make last_exception_msg json safe. + // It may contain random binary data when exception is a parsing error + // of unexpected contents. + Poco::JSON::Stringifier::stringify(json, oss, 0, -1, Poco::JSON_WRAP_STRINGS | Poco::JSON_ESCAPE_UNICODE); return oss.str(); } @@ -452,8 +742,11 @@ void PartMovesBetweenShardsOrchestrator::Entry::fromString(const String & buf) part_name = json->getValue(JSON_KEY_PART_NAME); part_uuid = parseFromString(json->getValue(JSON_KEY_PART_UUID)); to_shard = json->getValue(JSON_KEY_TO_SHARD); + dst_part_name = json->getValue(JSON_KEY_DST_PART_NAME); state.value = EntryState::fromString(json->getValue(JSON_KEY_STATE)); + rollback = json->getValue(JSON_KEY_ROLLBACK); last_exception_msg = json->getValue(JSON_KEY_LAST_EX_MSG); + num_tries = json->getValue(JSON_KEY_NUM_TRIES); } } diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h index 9e54ae8a8ed..38b6a076748 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -18,14 +19,37 @@ namespace ErrorCodes class StorageReplicatedMergeTree; -/// Cross shard part movement workflow orchestration. +/** + * Cross shard part movement workflow orchestration. + * + * TODO(nv): + * * Usage of `format_version` when acting on the behalf of the remote shard. + * There needs to be sort of an API to coordinate with remote replicas. + * * Only one movement at a time can be coordinated. This can easily be fixed + * by cycling through different tasks and checking their status with a + * priority queue and back-off for failing tasks + * `min(backoff * num_tries, max_backoff)`. + */ class PartMovesBetweenShardsOrchestrator { public: struct EntryState { + // State transitions are linear. When a kill query is issued a rollback + // flag is set and transitions order is reversed. + // + // SOURCE_DROP is a critical state after which rollback is not possible + // and we must ensure that the task can always succeed after that. + // + // Similar for rollback. It should be always possible to rollback before + // SOURCE_DROP state and it should terminate. + // + // Note: This fragile. If you change the states please add entry to + // changelog about forward/backward compatibility. Better not to have + // any active move tasks while doing upgrade/downgrade operations. enum Value { + CANCELLED, TODO, SYNC_SOURCE, SYNC_DESTINATION, @@ -36,7 +60,6 @@ public: SOURCE_DROP_POST_DELAY, REMOVE_UUID_PIN, DONE, - CANCELLED, }; EntryState(): value(TODO) {} @@ -95,10 +118,14 @@ public: String part_name; UUID part_uuid; String to_shard; + String dst_part_name; EntryState state; + bool rollback = false; + /// Reset on successful transitions. String last_exception_msg; + UInt64 num_tries = 0; String znode_name; @@ -120,27 +147,31 @@ private: static constexpr auto JSON_KEY_PART_NAME = "part_name"; static constexpr auto JSON_KEY_PART_UUID = "part_uuid"; static constexpr auto JSON_KEY_TO_SHARD = "to_shard"; + static constexpr auto JSON_KEY_DST_PART_NAME = "dst_part_name"; static constexpr auto JSON_KEY_STATE = "state"; + static constexpr auto JSON_KEY_ROLLBACK = "rollback"; static constexpr auto JSON_KEY_LAST_EX_MSG = "last_exception"; + static constexpr auto JSON_KEY_NUM_TRIES = "num_tries"; public: - PartMovesBetweenShardsOrchestrator(StorageReplicatedMergeTree & storage_); + explicit PartMovesBetweenShardsOrchestrator(StorageReplicatedMergeTree & storage_); void start() { task->activateAndSchedule(); } void wakeup() { task->schedule(); } void shutdown(); - void fetchStateFromZK(); + CancellationCode killPartMoveToShard(const UUID & task_uuid); - /// We could have one thread per Entry and worry about concurrency issues. - /// Or we could have a single thread trying to run one step at a time. - bool step(); - - std::vector getEntries() const; + std::vector getEntries(); private: void run(); - void stepEntry(const Entry & entry, zkutil::ZooKeeperPtr zk); + bool step(); + Entry stepEntry(Entry entry, zkutil::ZooKeeperPtr zk); + + Entry getEntryByUUID(const UUID & task_uuid); + void removePins(const Entry & entry, zkutil::ZooKeeperPtr zk); + void syncStateFromZK(); private: StorageReplicatedMergeTree & storage; @@ -153,7 +184,7 @@ private: BackgroundSchedulePool::TaskHolder task; mutable std::mutex state_mutex; - std::map entries; + std::vector entries; public: String entries_znode_path; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 335302e5dda..03bef4581c1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4669,7 +4669,7 @@ void StorageReplicatedMergeTree::alter( /// If new version returns ordinary name, else returns part name containing the first and last month of the month /// NOTE: use it in pair with getFakePartCoveringAllPartsInPartition(...) -static String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info) +String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info) { if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { @@ -6568,6 +6568,7 @@ void StorageReplicatedMergeTree::movePartitionToShard( dst_pins.part_uuids.insert(part->uuid); PartMovesBetweenShardsOrchestrator::Entry part_move_entry; + part_move_entry.state = PartMovesBetweenShardsOrchestrator::EntryState::SYNC_SOURCE; part_move_entry.create_time = std::time(nullptr); part_move_entry.update_time = part_move_entry.create_time; part_move_entry.task_uuid = UUIDHelpers::generateV4(); @@ -6591,10 +6592,13 @@ void StorageReplicatedMergeTree::movePartitionToShard( String task_znode_path = dynamic_cast(*responses.back()).path_created; LOG_DEBUG(log, "Created task for part movement between shards at " + task_znode_path); - /// Force refresh local state. This will make the task immediately visible in `system.part_moves_between_shards` table. - part_moves_between_shards_orchestrator.fetchStateFromZK(); + /// TODO(nv): Nice to have support for `replication_alter_partitions_sync`. + /// For now use the system.part_moves_between_shards table for status. +} - // TODO: Add support for `replication_alter_partitions_sync`. +CancellationCode StorageReplicatedMergeTree::killPartMoveToShard(const UUID & task_uuid) +{ + return part_moves_between_shards_orchestrator.killPartMoveToShard(task_uuid); } void StorageReplicatedMergeTree::getCommitPartOps( diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index e76a54ac5ec..9eaf3c318dd 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -682,6 +682,7 @@ private: void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, ContextPtr query_context) override; void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr query_context) override; void movePartitionToShard(const ASTPtr & partition, bool move_part, const String & to, ContextPtr query_context) override; + CancellationCode killPartMoveToShard(const UUID & task_uuid) override; void fetchPartition( const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, @@ -745,6 +746,8 @@ protected: bool allow_renaming_); }; +String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info); + /** There are three places for each part, where it should be * 1. In the RAM, data_parts, all_data_parts. diff --git a/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp b/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp index 70bc473e241..c397392e9fb 100644 --- a/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp +++ b/src/Storages/System/StorageSystemPartMovesBetweenShards.cpp @@ -30,10 +30,12 @@ NamesAndTypesList StorageSystemPartMovesBetweenShards::getNamesAndTypes() { "part_name", std::make_shared() }, { "part_uuid", std::make_shared() }, { "to_shard", std::make_shared() }, + { "dst_part_name", std::make_shared() }, /// Processing status of item. { "update_time", std::make_shared() }, { "state", std::make_shared() }, + { "rollback", std::make_shared() }, { "num_tries", std::make_shared() }, { "last_exception", std::make_shared() }, }; @@ -122,11 +124,13 @@ void StorageSystemPartMovesBetweenShards::fillData(MutableColumns & res_columns, res_columns[col_num++]->insert(entry.part_name); res_columns[col_num++]->insert(entry.part_uuid); res_columns[col_num++]->insert(entry.to_shard); + res_columns[col_num++]->insert(entry.dst_part_name); /// Processing status of item. res_columns[col_num++]->insert(entry.update_time); res_columns[col_num++]->insert(entry.state.toString()); - res_columns[col_num++]->insert(0); + res_columns[col_num++]->insert(entry.rollback); + res_columns[col_num++]->insert(entry.num_tries); res_columns[col_num++]->insert(entry.last_exception_msg); } } diff --git a/tests/integration/test_part_moves_between_shards/test.py b/tests/integration/test_part_moves_between_shards/test.py index 00407f95389..ed7640e5f9e 100644 --- a/tests/integration/test_part_moves_between_shards/test.py +++ b/tests/integration/test_part_moves_between_shards/test.py @@ -1,32 +1,38 @@ -import random -import time - import pytest +import random +import threading +import time from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV +transient_ch_errors = [23, 32, 210] + cluster = ClickHouseCluster(__file__) s0r0 = cluster.add_instance( 's0r0', main_configs=['configs/remote_servers.xml', 'configs/merge_tree.xml'], + stay_alive=True, with_zookeeper=True) s0r1 = cluster.add_instance( 's0r1', main_configs=['configs/remote_servers.xml', 'configs/merge_tree.xml'], + stay_alive=True, with_zookeeper=True) s1r0 = cluster.add_instance( 's1r0', main_configs=['configs/remote_servers.xml', 'configs/merge_tree.xml'], + stay_alive=True, with_zookeeper=True) s1r1 = cluster.add_instance( 's1r1', main_configs=['configs/remote_servers.xml', 'configs/merge_tree.xml'], + stay_alive=True, with_zookeeper=True) @@ -43,12 +49,14 @@ def test_move(started_cluster): for shard_ix, rs in enumerate([[s0r0, s0r1], [s1r0, s1r1]]): for replica_ix, r in enumerate(rs): r.query(""" + DROP TABLE IF EXISTS test_move; CREATE TABLE test_move(v UInt64) ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/test_move', '{}') ORDER BY tuple() - """.format(shard_ix, replica_ix)) + """.format(shard_ix, r.name)) s0r0.query("SYSTEM STOP MERGES test_move") + s0r1.query("SYSTEM STOP MERGES test_move") s0r0.query("INSERT INTO test_move VALUES (1)") s0r0.query("INSERT INTO test_move VALUES (2)") @@ -63,14 +71,7 @@ def test_move(started_cluster): s0r0.query("SYSTEM START MERGES test_move") s0r0.query("OPTIMIZE TABLE test_move FINAL") - while True: - time.sleep(3) - - print(s0r0.query("SELECT * FROM system.part_moves_between_shards")) - - # Eventually. - if "DONE" == s0r0.query("SELECT state FROM system.part_moves_between_shards WHERE table = 'test_move'").strip(): - break + wait_for_state("DONE", s0r0, "test_move") for n in [s0r0, s0r1]: assert "1" == n.query("SELECT count() FROM test_move").strip() @@ -81,14 +82,7 @@ def test_move(started_cluster): # Move part back s1r0.query("ALTER TABLE test_move MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_0/tables/test_move'") - while True: - time.sleep(3) - - print(s1r0.query("SELECT * FROM system.part_moves_between_shards")) - - # Eventually. - if "DONE" == s1r0.query("SELECT state FROM system.part_moves_between_shards WHERE table = 'test_move'").strip(): - break + wait_for_state("DONE", s1r0, "test_move") for n in [s0r0, s0r1]: assert "2" == n.query("SELECT count() FROM test_move").strip() @@ -101,17 +95,20 @@ def test_deduplication_while_move(started_cluster): for shard_ix, rs in enumerate([[s0r0, s0r1], [s1r0, s1r1]]): for replica_ix, r in enumerate(rs): r.query(""" + DROP TABLE IF EXISTS test_deduplication; CREATE TABLE test_deduplication(v UInt64) ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/test_deduplication', '{}') ORDER BY tuple() - """.format(shard_ix, replica_ix)) + """.format(shard_ix, r.name)) r.query(""" - CREATE TABLE t_d AS test_deduplication + DROP TABLE IF EXISTS test_deduplication_d; + CREATE TABLE test_deduplication_d AS test_deduplication ENGINE Distributed('test_cluster', '', test_deduplication) """) s0r0.query("SYSTEM STOP MERGES test_deduplication") + s0r1.query("SYSTEM STOP MERGES test_deduplication") s0r0.query("INSERT INTO test_deduplication VALUES (1)") s0r0.query("INSERT INTO test_deduplication VALUES (2)") @@ -120,7 +117,8 @@ def test_deduplication_while_move(started_cluster): assert "2" == s0r0.query("SELECT count() FROM test_deduplication").strip() assert "0" == s1r0.query("SELECT count() FROM test_deduplication").strip() - s0r0.query("ALTER TABLE test_deduplication MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_1/tables/test_deduplication'") + s0r0.query( + "ALTER TABLE test_deduplication MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_1/tables/test_deduplication'") s0r0.query("SYSTEM START MERGES test_deduplication") expected = """ @@ -128,32 +126,363 @@ def test_deduplication_while_move(started_cluster): 2 """ - # Verify that we get consisntent result at all times while the part is moving from one shard to another. - while "DONE" != s0r0.query("SELECT state FROM system.part_moves_between_shards WHERE table = 'test_deduplication' ORDER BY create_time DESC LIMIT 1").strip(): + def deduplication_invariant_test(): n = random.choice(list(started_cluster.instances.values())) + assert TSV( + n.query("SELECT * FROM test_deduplication_d ORDER BY v", + settings={"allow_experimental_query_deduplication": 1}) + ) == TSV(expected) - assert TSV(n.query("SELECT * FROM t_d ORDER BY v", settings={ - "allow_experimental_query_deduplication": 1 - })) == TSV(expected) + deduplication_invariant = ConcurrentInvariant(deduplication_invariant_test) + deduplication_invariant.start() + + wait_for_state("DONE", s0r0, "test_deduplication") + + deduplication_invariant.stop_and_assert_no_exception() + + +def test_part_move_step_by_step(started_cluster): + for shard_ix, rs in enumerate([[s0r0, s0r1], [s1r0, s1r1]]): + for replica_ix, r in enumerate(rs): + r.query(""" + DROP TABLE IF EXISTS test_part_move_step_by_step; + CREATE TABLE test_part_move_step_by_step(v UInt64) + ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/test_part_move_step_by_step', '{}') + ORDER BY tuple() + """.format(shard_ix, r.name)) + + r.query(""" + DROP TABLE IF EXISTS test_part_move_step_by_step_d; + CREATE TABLE test_part_move_step_by_step_d AS test_part_move_step_by_step + ENGINE Distributed('test_cluster', currentDatabase(), test_part_move_step_by_step) + """) + + s0r0.query("SYSTEM STOP MERGES test_part_move_step_by_step") + s0r1.query("SYSTEM STOP MERGES test_part_move_step_by_step") + + s0r0.query("INSERT INTO test_part_move_step_by_step VALUES (1)") + s0r0.query("INSERT INTO test_part_move_step_by_step VALUES (2)") + s0r1.query("SYSTEM SYNC REPLICA test_part_move_step_by_step", timeout=20) + + assert "2" == s0r0.query("SELECT count() FROM test_part_move_step_by_step").strip() + assert "0" == s1r0.query("SELECT count() FROM test_part_move_step_by_step").strip() + + expected = """ +1 +2 +""" + + def deduplication_invariant_test(): + n = random.choice(list(started_cluster.instances.values())) + try: + assert TSV( + n.query("SELECT * FROM test_part_move_step_by_step_d ORDER BY v", + settings={"allow_experimental_query_deduplication": 1}) + ) == TSV(expected) + except QueryRuntimeException as e: + # ignore transient errors that are caused by us restarting nodes + if e.returncode not in transient_ch_errors: + raise e + + deduplication_invariant = ConcurrentInvariant(deduplication_invariant_test) + deduplication_invariant.start() + + # Stop a source replica to prevent SYNC_SOURCE succeeding. + s0r1.stop_clickhouse() + + s0r0.query( + "ALTER TABLE test_part_move_step_by_step MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_1/tables/test_part_move_step_by_step'") + + # Should hang on SYNC_SOURCE until all source replicas acknowledge new pinned UUIDs. + wait_for_state("SYNC_SOURCE", s0r0, "test_part_move_step_by_step", "Some replicas haven\\'t processed event") + deduplication_invariant.assert_no_exception() + + # Start all replicas in source shard but stop a replica in destination shard + # to prevent SYNC_DESTINATION succeeding. + s1r1.stop_clickhouse() + s0r1.start_clickhouse() + + # After SYNC_SOURCE step no merges will be assigned. + s0r0.query("SYSTEM START MERGES test_part_move_step_by_step; OPTIMIZE TABLE test_part_move_step_by_step;") + s0r1.query("SYSTEM START MERGES test_part_move_step_by_step; OPTIMIZE TABLE test_part_move_step_by_step;") + + wait_for_state("SYNC_DESTINATION", s0r0, "test_part_move_step_by_step", "Some replicas haven\\'t processed event") + deduplication_invariant.assert_no_exception() + + # Start previously stopped replica in destination shard to let SYNC_DESTINATION + # succeed. + # Stop the other replica in destination shard to prevent DESTINATION_FETCH succeed. + s1r0.stop_clickhouse() + s1r1.start_clickhouse() + wait_for_state("DESTINATION_FETCH", s0r0, "test_part_move_step_by_step", "Some replicas haven\\'t processed event") + deduplication_invariant.assert_no_exception() + + # Start previously stopped replica in destination shard to let DESTINATION_FETCH + # succeed. + # Stop the other replica in destination shard to prevent DESTINATION_ATTACH succeed. + s1r1.stop_clickhouse() + s1r0.start_clickhouse() + wait_for_state("DESTINATION_ATTACH", s0r0, "test_part_move_step_by_step", "Some replicas haven\\'t processed event") + deduplication_invariant.assert_no_exception() + + # Start all replicas in destination shard to let DESTINATION_ATTACH succeed. + # Stop a source replica to prevent SOURCE_DROP succeeding. + s0r0.stop_clickhouse() + s1r1.start_clickhouse() + wait_for_state("SOURCE_DROP", s0r1, "test_part_move_step_by_step", "Some replicas haven\\'t processed event") + deduplication_invariant.assert_no_exception() + + s0r0.start_clickhouse() + wait_for_state("DONE", s0r1, "test_part_move_step_by_step") + deduplication_invariant.assert_no_exception() + + # No hung tasks in replication queue. Would timeout otherwise. + for instance in started_cluster.instances.values(): + instance.query("SYSTEM SYNC REPLICA test_part_move_step_by_step") + + assert "1" == s0r0.query("SELECT count() FROM test_part_move_step_by_step").strip() + assert "1" == s1r0.query("SELECT count() FROM test_part_move_step_by_step").strip() + + deduplication_invariant.stop_and_assert_no_exception() + + +def test_part_move_step_by_step_kill(started_cluster): + for shard_ix, rs in enumerate([[s0r0, s0r1], [s1r0, s1r1]]): + for replica_ix, r in enumerate(rs): + r.query(""" + DROP TABLE IF EXISTS test_part_move_step_by_step_kill; + CREATE TABLE test_part_move_step_by_step_kill(v UInt64) + ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/test_part_move_step_by_step_kill', '{}') + ORDER BY tuple() + """.format(shard_ix, r.name)) + + r.query(""" + DROP TABLE IF EXISTS test_part_move_step_by_step_kill_d; + CREATE TABLE test_part_move_step_by_step_kill_d AS test_part_move_step_by_step_kill + ENGINE Distributed('test_cluster', currentDatabase(), test_part_move_step_by_step_kill) + """) + + s0r0.query("SYSTEM STOP MERGES test_part_move_step_by_step_kill") + s0r1.query("SYSTEM STOP MERGES test_part_move_step_by_step_kill") + + s0r0.query("INSERT INTO test_part_move_step_by_step_kill VALUES (1)") + s0r0.query("INSERT INTO test_part_move_step_by_step_kill VALUES (2)") + s0r1.query("SYSTEM SYNC REPLICA test_part_move_step_by_step_kill", timeout=20) + + assert "2" == s0r0.query("SELECT count() FROM test_part_move_step_by_step_kill").strip() + assert "0" == s1r0.query("SELECT count() FROM test_part_move_step_by_step_kill").strip() + + expected = """ +1 +2 +""" + + def deduplication_invariant_test(): + n = random.choice(list(started_cluster.instances.values())) + try: + assert TSV( + n.query("SELECT * FROM test_part_move_step_by_step_kill_d ORDER BY v", + settings={ + "allow_experimental_query_deduplication": 1 + }) + ) == TSV(expected) + except QueryRuntimeException as e: + # ignore transient errors that are caused by us restarting nodes + if e.returncode not in transient_ch_errors: + raise e + + deduplication_invariant = ConcurrentInvariant(deduplication_invariant_test) + deduplication_invariant.start() + + # Stop a source replica to prevent SYNC_SOURCE succeeding. + s0r1.stop_clickhouse() + + s0r0.query( + "ALTER TABLE test_part_move_step_by_step_kill MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_1/tables/test_part_move_step_by_step_kill'") + + # Should hang on SYNC_SOURCE until all source replicas acknowledge new pinned UUIDs. + wait_for_state("SYNC_SOURCE", s0r0, "test_part_move_step_by_step_kill", "Some replicas haven\\'t processed event") + deduplication_invariant.assert_no_exception() + + # Start all replicas in source shard but stop a replica in destination shard + # to prevent SYNC_DESTINATION succeeding. + s1r1.stop_clickhouse() + s0r1.start_clickhouse() + + # After SYNC_SOURCE step no merges will be assigned. + s0r0.query("SYSTEM START MERGES test_part_move_step_by_step_kill; OPTIMIZE TABLE test_part_move_step_by_step_kill;") + s0r1.query("SYSTEM START MERGES test_part_move_step_by_step_kill; OPTIMIZE TABLE test_part_move_step_by_step_kill;") + + wait_for_state("SYNC_DESTINATION", s0r0, "test_part_move_step_by_step_kill", + "Some replicas haven\\'t processed event") + deduplication_invariant.assert_no_exception() + + # Start previously stopped replica in destination shard to let SYNC_DESTINATION + # succeed. + # Stop the other replica in destination shard to prevent DESTINATION_FETCH succeed. + s1r0.stop_clickhouse() + s1r1.start_clickhouse() + wait_for_state("DESTINATION_FETCH", s0r0, "test_part_move_step_by_step_kill", + "Some replicas haven\\'t processed event") + + # Start previously stopped replica in destination shard to let DESTINATION_FETCH + # succeed. + # Stop the other replica in destination shard to prevent DESTINATION_ATTACH succeed. + s1r1.stop_clickhouse() + s1r0.start_clickhouse() + wait_for_state("DESTINATION_ATTACH", s0r0, "test_part_move_step_by_step_kill", + "Some replicas haven\\'t processed event") + deduplication_invariant.assert_no_exception() + + # Rollback here. + s0r0.query(""" + KILL PART_MOVE_TO_SHARD + WHERE task_uuid = (SELECT task_uuid FROM system.part_moves_between_shards WHERE table = 'test_part_move_step_by_step_kill') + """) + + wait_for_state("DESTINATION_ATTACH", s0r0, "test_part_move_step_by_step_kill", + assert_exception_msg="Some replicas haven\\'t processed event", + assert_rollback=True) + + s1r1.start_clickhouse() + + wait_for_state("CANCELLED", s0r0, "test_part_move_step_by_step_kill", assert_rollback=True) + deduplication_invariant.assert_no_exception() + + # No hung tasks in replication queue. Would timeout otherwise. + for instance in started_cluster.instances.values(): + instance.query("SYSTEM SYNC REPLICA test_part_move_step_by_step_kill") + + assert "2" == s0r0.query("SELECT count() FROM test_part_move_step_by_step_kill").strip() + assert "0" == s1r0.query("SELECT count() FROM test_part_move_step_by_step_kill").strip() + + deduplication_invariant.stop_and_assert_no_exception() def test_move_not_permitted(started_cluster): + # Verify that invariants for part compatibility are checked. + + # Tests are executed in order. Make sure cluster is up if previous test + # failed. + s0r0.start_clickhouse() + s1r0.start_clickhouse() + for ix, n in enumerate([s0r0, s1r0]): - n.query("DROP TABLE IF EXISTS not_permitted") n.query(""" - CREATE TABLE not_permitted(v_{} UInt64) - ENGINE ReplicatedMergeTree('/clickhouse/shard_{}/tables/not_permitted', 'r') - ORDER BY tuple() - """.format(ix, ix)) + DROP TABLE IF EXISTS not_permitted_columns; + + CREATE TABLE not_permitted_columns(v_{ix} UInt64) + ENGINE ReplicatedMergeTree('/clickhouse/shard_{ix}/tables/not_permitted_columns', 'r') + ORDER BY tuple(); + """.format(ix=ix)) - s0r0.query("INSERT INTO not_permitted VALUES (1)") + partition = "date" + if ix > 0: + partition = "v" - with pytest.raises(QueryRuntimeException) as exc: - s0r0.query("ALTER TABLE not_permitted MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_1/tables/not_permitted'") + n.query(""" + DROP TABLE IF EXISTS not_permitted_partition; + CREATE TABLE not_permitted_partition(date Date, v UInt64) + ENGINE ReplicatedMergeTree('/clickhouse/shard_{ix}/tables/not_permitted_partition', 'r') + PARTITION BY ({partition}) + ORDER BY tuple(); + """.format(ix=ix, partition=partition)) - assert "DB::Exception: Table columns structure in ZooKeeper is different from local table structure." in str(exc.value) + s0r0.query("INSERT INTO not_permitted_columns VALUES (1)") + s0r0.query("INSERT INTO not_permitted_partition VALUES ('2021-09-03', 1)") - with pytest.raises(QueryRuntimeException) as exc: - s0r0.query("ALTER TABLE not_permitted MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_0/tables/not_permitted'") + with pytest.raises(QueryRuntimeException, match="DB::Exception: Source and destination are the same"): + s0r0.query("ALTER TABLE not_permitted_columns MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_0/tables/not_permitted_columns'") - assert "DB::Exception: Source and destination are the same" in str(exc.value) + with pytest.raises(QueryRuntimeException, match="DB::Exception: Table columns structure in ZooKeeper is different from local table structure."): + s0r0.query("ALTER TABLE not_permitted_columns MOVE PART 'all_0_0_0' TO SHARD '/clickhouse/shard_1/tables/not_permitted_columns'") + + with pytest.raises(QueryRuntimeException, match="DB::Exception: Existing table metadata in ZooKeeper differs in partition key expression."): + s0r0.query("ALTER TABLE not_permitted_partition MOVE PART '20210903_0_0_0' TO SHARD '/clickhouse/shard_1/tables/not_permitted_partition'") + + +def wait_for_state(desired_state, instance, test_table, assert_exception_msg=None, assert_rollback=False): + last_debug_print_time = time.time() + + print("Waiting to reach state: {}".format(desired_state)) + if assert_exception_msg: + print(" with exception contents: {}".format(assert_exception_msg)) + if assert_rollback: + print(" and rollback: {}".format(assert_rollback)) + + while True: + tasks = TSV.toMat(instance.query( + "SELECT state, num_tries, last_exception, rollback FROM system.part_moves_between_shards WHERE table = '{}'".format( + test_table))) + assert len(tasks) == 1, "only one task expected in this test" + + if time.time() - last_debug_print_time > 30: + last_debug_print_time = time.time() + print("Current state: ", tasks) + + [state, num_tries, last_exception, rollback] = tasks[0] + + if state == desired_state: + if assert_exception_msg and int(num_tries) < 3: + # Let the task be retried a few times when expecting an exception + # to make sure the exception is persistent and the code doesn't + # accidentally continue to run when we expect it not to. + continue + + if assert_exception_msg: + assert assert_exception_msg in last_exception + + if assert_rollback: + assert int(rollback) == 1, "rollback bit isn't set" + + break + elif state in ["DONE", "CANCELLED"]: + raise Exception("Reached terminal state {}, but was waiting for {}".format(state, desired_state)) + + time.sleep(0.1) + + +class ConcurrentInvariant: + def __init__(self, invariant_test, loop_sleep=0.1): + self.invariant_test = invariant_test + self.loop_sleep = loop_sleep + + self.started = False + self.exiting = False + self.exception = None + self.thread = threading.Thread(target=self._loop) + + def start(self): + if self.started: + raise Exception('invariant thread already started') + + self.started = True + self.thread.start() + + def stop_and_assert_no_exception(self): + self._assert_started() + + self.exiting = True + self.thread.join() + + if self.exception: + raise self.exception + + def assert_no_exception(self): + self._assert_started() + + if self.exception: + raise self.exception + + def _loop(self): + try: + while not self.exiting: + self.invariant_test() + time.sleep(self.loop_sleep) + except Exception as e: + self.exiting = True + self.exception = e + + def _assert_started(self): + if not self.started: + raise Exception('invariant thread not started, forgot to call start?') From 4fbc7c3f31ef97f4a017847336caeb58ff5c6e9f Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 18 Sep 2021 03:01:25 +0000 Subject: [PATCH 036/919] remove unused code --- src/Storages/FileLog/StorageFileLog.cpp | 4 ---- src/Storages/FileLog/StorageFileLog.h | 2 -- tests/queries/0_stateless/02023_storage_filelog.sh | 1 - 3 files changed, 7 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 6d380c17f38..10d838c84ec 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -307,10 +307,6 @@ bool StorageFileLog::streamToViews() return updateFileStatus(); } -void StorageFileLog::clearInvalidFiles() -{ -} - void registerStorageFileLog(StorageFactory & factory) { auto creator_fn = [](const StorageFactory::Arguments & args) diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index d035c0e2903..7df66b5c54f 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -101,8 +101,6 @@ private: void threadFunc(); - void clearInvalidFiles(); - size_t getPollMaxBatchSize() const; size_t getMaxBlockSize() const; size_t getPollTimeoutMillisecond() const; diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 5a0333b1bc3..8b072dd65df 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -22,7 +22,6 @@ echo 8, 8 >> ${user_files_path}/logs/a.txt echo 9, 9 >> ${user_files_path}/logs/a.txt echo 10, 10 >> ${user_files_path}/logs/a.txt -### 1st TEST in CLIENT mode. ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" ${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('logs', 'CSV');" From 11ae94b8d37f83a8b20407012f5e8f82f37d2f10 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 17 Sep 2021 18:27:43 +0300 Subject: [PATCH 037/919] Async read for remote fs --- src/Disks/DiskWebServer.cpp | 13 ++- src/Disks/HDFS/DiskHDFS.cpp | 7 +- src/Disks/IDiskRemote.h | 4 +- src/Disks/ReadIndirectBufferFromRemoteFS.cpp | 88 +++--------------- src/Disks/ReadIndirectBufferFromRemoteFS.h | 23 ++--- src/Disks/S3/DiskS3.cpp | 30 +++++-- src/Disks/S3/DiskS3.h | 4 +- src/Disks/S3/registerDiskS3.cpp | 3 +- src/IO/ReadBufferFromRemoteFS.cpp | 94 ++++++++++++++++++++ src/IO/ReadBufferFromRemoteFS.h | 42 +++++++++ src/IO/SeekableReadBuffer.h | 1 + src/IO/ThreadPoolRemoteFSReader.cpp | 70 +++++++++++++++ src/IO/ThreadPoolRemoteFSReader.h | 30 +++++++ 13 files changed, 293 insertions(+), 116 deletions(-) create mode 100644 src/IO/ReadBufferFromRemoteFS.cpp create mode 100644 src/IO/ReadBufferFromRemoteFS.h create mode 100644 src/IO/ThreadPoolRemoteFSReader.cpp create mode 100644 src/IO/ThreadPoolRemoteFSReader.h diff --git a/src/Disks/DiskWebServer.cpp b/src/Disks/DiskWebServer.cpp index 93d5593f0f2..ba6a7ed9168 100644 --- a/src/Disks/DiskWebServer.cpp +++ b/src/Disks/DiskWebServer.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include @@ -105,22 +106,20 @@ private: }; -class ReadBufferFromWebServer final : public ReadIndirectBufferFromRemoteFS +class ReadBufferFromWebServer final : public ReadBufferFromRemoteFS { public: ReadBufferFromWebServer( - const String & uri_, - RemoteMetadata metadata_, - ContextPtr context_, - size_t buf_size_) - : ReadIndirectBufferFromRemoteFS(metadata_) + const String & uri_, RemoteMetadata metadata_, + ContextPtr context_, size_t buf_size_) + : ReadBufferFromRemoteFS(metadata_) , uri(uri_) , context(context_) , buf_size(buf_size_) { } - std::unique_ptr createReadBuffer(const String & path) override + SeekableReadBufferPtr createReadBuffer(const String & path) const override { return std::make_unique(fs::path(uri) / path, context, buf_size); } diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index 5b15ee69cb8..3f0fa4cc263 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -48,7 +49,7 @@ private: /// Reads data from HDFS using stored paths in metadata. -class ReadIndirectBufferFromHDFS final : public ReadIndirectBufferFromRemoteFS +class ReadIndirectBufferFromHDFS final : public ReadBufferFromRemoteFS { public: ReadIndirectBufferFromHDFS( @@ -56,7 +57,7 @@ public: const String & hdfs_uri_, DiskHDFS::Metadata metadata_, size_t buf_size_) - : ReadIndirectBufferFromRemoteFS(metadata_) + : ReadBufferFromRemoteFS(metadata_) , config(config_) , buf_size(buf_size_) { @@ -65,7 +66,7 @@ public: hdfs_uri = hdfs_uri_.substr(0, begin_of_path); } - std::unique_ptr createReadBuffer(const String & path) override + SeekableReadBufferPtr createReadBuffer(const String & path) const override { return std::make_unique(hdfs_uri, hdfs_directory + path, config, buf_size); } diff --git a/src/Disks/IDiskRemote.h b/src/Disks/IDiskRemote.h index 5333b14a12d..fb837316327 100644 --- a/src/Disks/IDiskRemote.h +++ b/src/Disks/IDiskRemote.h @@ -5,8 +5,8 @@ #endif #include -#include "Disks/DiskFactory.h" -#include "Disks/Executor.h" +#include +#include #include #include #include diff --git a/src/Disks/ReadIndirectBufferFromRemoteFS.cpp b/src/Disks/ReadIndirectBufferFromRemoteFS.cpp index 3bc22167f50..fdbb3a3c8f4 100644 --- a/src/Disks/ReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/ReadIndirectBufferFromRemoteFS.cpp @@ -1,9 +1,5 @@ #include "ReadIndirectBufferFromRemoteFS.h" -#include -#include -#include - namespace DB { @@ -14,16 +10,13 @@ namespace ErrorCodes } -template -ReadIndirectBufferFromRemoteFS::ReadIndirectBufferFromRemoteFS( - RemoteMetadata metadata_) - : metadata(std::move(metadata_)) +ReadIndirectBufferFromRemoteFS::ReadIndirectBufferFromRemoteFS( + ReadBufferFromRemoteFSImpl impl_) : impl(std::move(impl_)) { } -template -off_t ReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence) +off_t ReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence) { if (whence == SEEK_CUR) { @@ -41,7 +34,8 @@ off_t ReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence) else if (whence == SEEK_SET) { /// If position within current working buffer - shift pos. - if (!working_buffer.empty() && size_t(offset_) >= absolute_position - working_buffer.size() + if (!working_buffer.empty() + && size_t(offset_) >= absolute_position - working_buffer.size() && size_t(offset_) < absolute_position) { pos = working_buffer.end() - (absolute_position - offset_); @@ -55,69 +49,21 @@ off_t ReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence) else throw Exception("Only SEEK_SET or SEEK_CUR modes are allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); - current_buf = initialize(); + impl->seek(absolute_position, SEEK_SET); pos = working_buffer.end(); return absolute_position; } -template -std::unique_ptr ReadIndirectBufferFromRemoteFS::initialize() -{ - size_t offset = absolute_position; - for (size_t i = 0; i < metadata.remote_fs_objects.size(); ++i) - { - current_buf_idx = i; - const auto & [file_path, size] = metadata.remote_fs_objects[i]; - if (size > offset) - { - auto buf = createReadBuffer(file_path); - buf->seek(offset, SEEK_SET); - return buf; - } - offset -= size; - } - return nullptr; -} - - -template -bool ReadIndirectBufferFromRemoteFS::nextImpl() -{ - /// Find first available buffer that fits to given offset. - if (!current_buf) - current_buf = initialize(); - - /// If current buffer has remaining data - use it. - if (current_buf) - { - bool result = nextAndShiftPosition(); - if (result) - return true; - } - - /// If there is no available buffers - nothing to read. - if (current_buf_idx + 1 >= metadata.remote_fs_objects.size()) - return false; - - ++current_buf_idx; - const auto & path = metadata.remote_fs_objects[current_buf_idx].first; - - current_buf = createReadBuffer(path); - - return nextAndShiftPosition(); -} - -template -bool ReadIndirectBufferFromRemoteFS::nextAndShiftPosition() +bool ReadIndirectBufferFromRemoteFS::nextImpl() { /// Transfer current position and working_buffer to actual ReadBuffer - swap(*current_buf); + swap(*impl); /// Position and working_buffer will be updated in next() call - auto result = current_buf->next(); + auto result = impl->next(); /// and assigned to current buffer. - swap(*current_buf); + swap(*impl); /// absolute position is shifted by a data size that was read in next() call above. if (result) @@ -126,18 +72,4 @@ bool ReadIndirectBufferFromRemoteFS::nextAndShiftPosition() return result; } - -#if USE_AWS_S3 -template -class ReadIndirectBufferFromRemoteFS; -#endif - -#if USE_HDFS -template -class ReadIndirectBufferFromRemoteFS; -#endif - -template -class ReadIndirectBufferFromRemoteFS; - } diff --git a/src/Disks/ReadIndirectBufferFromRemoteFS.h b/src/Disks/ReadIndirectBufferFromRemoteFS.h index d3cc6aeddd0..e15a843ae66 100644 --- a/src/Disks/ReadIndirectBufferFromRemoteFS.h +++ b/src/Disks/ReadIndirectBufferFromRemoteFS.h @@ -5,6 +5,7 @@ #endif #include +#include #include #include @@ -12,36 +13,24 @@ namespace DB { -/// Reads data from S3/HDFS using stored paths in metadata. -template +/// Reads data from S3/HDFS/Web using stored paths in metadata. class ReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase { public: - explicit ReadIndirectBufferFromRemoteFS(RemoteMetadata metadata_); + explicit ReadIndirectBufferFromRemoteFS(ReadBufferFromRemoteFSImpl impl_); off_t seek(off_t offset_, int whence) override; off_t getPosition() override { return absolute_position - available(); } - String getFileName() const override { return metadata.metadata_file_path; } - - virtual std::unique_ptr createReadBuffer(const String & path) = 0; - -protected: - RemoteMetadata metadata; + String getFileName() const override { return impl->getFileName(); } private: - std::unique_ptr initialize(); - - bool nextAndShiftPosition(); - bool nextImpl() override; + ReadBufferFromRemoteFSImpl impl; + size_t absolute_position = 0; - - size_t current_buf_idx = 0; - - std::unique_ptr current_buf; }; } diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 30cd314702b..aae5bb8fc26 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -27,6 +27,9 @@ #include #include #include +#include +#include +#include #include // Y_IGNORE #include // Y_IGNORE @@ -127,7 +130,7 @@ void throwIfError(const Aws::Utils::Outcome & response) } /// Reads data from S3 using stored paths in metadata. -class ReadIndirectBufferFromS3 final : public ReadIndirectBufferFromRemoteFS +class ReadIndirectBufferFromS3 final : public ReadBufferFromRemoteFS { public: ReadIndirectBufferFromS3( @@ -136,7 +139,7 @@ public: DiskS3::Metadata metadata_, size_t max_single_read_retries_, size_t buf_size_) - : ReadIndirectBufferFromRemoteFS(metadata_) + : ReadBufferFromRemoteFS(metadata_) , client_ptr(std::move(client_ptr_)) , bucket(bucket_) , max_single_read_retries(max_single_read_retries_) @@ -144,7 +147,7 @@ public: { } - std::unique_ptr createReadBuffer(const String & path) override + SeekableReadBufferPtr createReadBuffer(const String & path) const override { return std::make_unique(client_ptr, bucket, fs::path(metadata.remote_fs_root_path) / path, max_single_read_retries, buf_size); } @@ -229,9 +232,20 @@ std::unique_ptr DiskS3::readFile(const String & path, co LOG_TRACE(log, "Read from file by path: {}. Existing S3 objects: {}", backQuote(metadata_path + path), metadata.remote_fs_objects.size()); - auto reader = std::make_unique( - settings->client, bucket, metadata, settings->s3_max_single_read_retries, read_settings.remote_fs_buffer_size); - return std::make_unique(std::move(reader), settings->min_bytes_for_seek); + auto s3_impl = std::make_unique( + settings->client, bucket, metadata, + settings->s3_max_single_read_retries, read_settings.remote_fs_buffer_size); + + if (settings->async_read) + { + static AsynchronousReaderPtr reader = std::make_shared(16, 1000000); + return std::make_unique(reader, read_settings.priority, std::move(s3_impl)); + //return std::make_unique(std::move(buf), settings->min_bytes_for_seek); + } + else + { + return std::make_unique(std::move(s3_impl), settings->min_bytes_for_seek); + } } std::unique_ptr DiskS3::writeFile(const String & path, size_t buf_size, WriteMode mode) @@ -1051,7 +1065,8 @@ DiskS3Settings::DiskS3Settings( bool send_metadata_, int thread_pool_size_, int list_object_keys_size_, - int objects_chunk_size_to_delete_) + int objects_chunk_size_to_delete_, + bool async_read_) : client(client_) , s3_max_single_read_retries(s3_max_single_read_retries_) , s3_min_upload_part_size(s3_min_upload_part_size_) @@ -1061,6 +1076,7 @@ DiskS3Settings::DiskS3Settings( , thread_pool_size(thread_pool_size_) , list_object_keys_size(list_object_keys_size_) , objects_chunk_size_to_delete(objects_chunk_size_to_delete_) + , async_read(async_read_) { } diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index e2141b6196e..fafd115946d 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -36,7 +36,8 @@ struct DiskS3Settings bool send_metadata_, int thread_pool_size_, int list_object_keys_size_, - int objects_chunk_size_to_delete_); + int objects_chunk_size_to_delete_, + bool async_read); std::shared_ptr client; size_t s3_max_single_read_retries; @@ -47,6 +48,7 @@ struct DiskS3Settings int thread_pool_size; int list_object_keys_size; int objects_chunk_size_to_delete; + bool async_read; }; diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index decf454d257..aab97d10760 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -161,7 +161,8 @@ std::unique_ptr getSettings(const Poco::Util::AbstractConfigurat config.getBool(config_prefix + ".send_metadata", false), config.getInt(config_prefix + ".thread_pool_size", 16), config.getInt(config_prefix + ".list_object_keys_size", 1000), - config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000)); + config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000), + config.getBool(config_prefix + ".async_read", false)); } } diff --git a/src/IO/ReadBufferFromRemoteFS.cpp b/src/IO/ReadBufferFromRemoteFS.cpp new file mode 100644 index 00000000000..353b3431bf1 --- /dev/null +++ b/src/IO/ReadBufferFromRemoteFS.cpp @@ -0,0 +1,94 @@ +#include "ReadBufferFromRemoteFS.h" + +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_SEEK_THROUGH_FILE; +} + + +ReadBufferFromRemoteFS::ReadBufferFromRemoteFS(const RemoteMetadata & metadata_) + : metadata(std::move(metadata_)) +{ +} + + +SeekableReadBufferPtr ReadBufferFromRemoteFS::initialize() +{ + auto current_buf_offset = absolute_position; + for (size_t i = 0; i < metadata.remote_fs_objects.size(); ++i) + { + current_buf_idx = i; + const auto & [file_path, size] = metadata.remote_fs_objects[i]; + + if (size > current_buf_offset) + { + auto buf = createReadBuffer(file_path); + buf->seek(current_buf_offset, SEEK_SET); + return buf; + } + + current_buf_offset -= size; + } + return nullptr; +} + + +bool ReadBufferFromRemoteFS::nextImpl() +{ + /// Find first available buffer that fits to given offset. + if (!current_buf) + current_buf = initialize(); + + /// If current buffer has remaining data - use it. + if (current_buf) + { + if (readImpl()) + return true; + } + + /// If there is no available buffers - nothing to read. + if (current_buf_idx + 1 >= metadata.remote_fs_objects.size()) + return false; + + ++current_buf_idx; + + const auto & path = metadata.remote_fs_objects[current_buf_idx].first; + current_buf = createReadBuffer(path); + + return readImpl(); +} + + +bool ReadBufferFromRemoteFS::readImpl() +{ + /// Transfer current position and working_buffer to actual ReadBuffer + swap(*current_buf); + /// Position and working_buffer will be updated in next() call + auto result = current_buf->next(); + /// Assign result to current buffer. + swap(*current_buf); + + return result; +} + + +off_t ReadBufferFromRemoteFS::seek(off_t offset_, int whence) +{ + if (whence != SEEK_SET) + throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only SEEK_SET is allowed"); + + absolute_position = offset_; + current_buf = initialize(); + return absolute_position; +} + + +} diff --git a/src/IO/ReadBufferFromRemoteFS.h b/src/IO/ReadBufferFromRemoteFS.h new file mode 100644 index 00000000000..468a9c0f3f6 --- /dev/null +++ b/src/IO/ReadBufferFromRemoteFS.h @@ -0,0 +1,42 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class ReadBufferFromRemoteFS : public ReadBufferFromFileBase +{ +public: + explicit ReadBufferFromRemoteFS(const RemoteMetadata & metadata_); + + off_t seek(off_t offset, int whence) override; + + off_t getPosition() override { return absolute_position - available(); } + + String getFileName() const override { return metadata.metadata_file_path; } + + bool readNext() { return nextImpl(); } + +protected: + virtual SeekableReadBufferPtr createReadBuffer(const String & path) const = 0; + RemoteMetadata metadata; + +private: + bool nextImpl() override; + + SeekableReadBufferPtr initialize(); + + bool readImpl(); + + SeekableReadBufferPtr current_buf; + + size_t current_buf_idx = 0; + size_t absolute_position = 0; +}; + +using ReadBufferFromRemoteFSImpl = std::shared_ptr; + +} diff --git a/src/IO/SeekableReadBuffer.h b/src/IO/SeekableReadBuffer.h index 97620f0c03c..400fb99db16 100644 --- a/src/IO/SeekableReadBuffer.h +++ b/src/IO/SeekableReadBuffer.h @@ -33,4 +33,5 @@ public: virtual off_t getPosition() = 0; }; +using SeekableReadBufferPtr = std::shared_ptr; } diff --git a/src/IO/ThreadPoolRemoteFSReader.cpp b/src/IO/ThreadPoolRemoteFSReader.cpp new file mode 100644 index 00000000000..c218ce160d0 --- /dev/null +++ b/src/IO/ThreadPoolRemoteFSReader.cpp @@ -0,0 +1,70 @@ +#include + +#include +#include +#include +#include +#include +#include + +#include +#include + +#include +#include + + +namespace ProfileEvents +{ + extern const Event RemoteVFSReadMicroseconds; + extern const Event RemoteVFSReadBytes; +} + +namespace CurrentMetrics +{ + extern const Metric Read; +} + +namespace DB +{ + + +ThreadPoolRemoteFSReader::ThreadPoolRemoteFSReader(size_t pool_size, size_t queue_size_) + : pool(pool_size, pool_size, queue_size_) +{ +} + + +std::future ThreadPoolRemoteFSReader::submit(Request request) +{ + auto task = std::make_shared>([request] + { + std::cerr << "\n\nTask is execited!!!\n\n"; + + setThreadName("ThreadPoolRead"); + CurrentMetrics::Increment metric_increment{CurrentMetrics::Read}; + Stopwatch watch(CLOCK_MONOTONIC); + + size_t bytes_read = 0; + auto * remote_fs_fd = assert_cast(request.descriptor.get()); + auto * remote_fs_buf = dynamic_cast(remote_fs_fd->impl.get()); + auto result = remote_fs_buf->readNext(); + if (result) + bytes_read = remote_fs_buf->buffer().size(); + + std::cerr << "Read " << bytes_read << " bytes.\n"; + watch.stop(); + + ProfileEvents::increment(ProfileEvents::RemoteVFSReadMicroseconds, watch.elapsedMicroseconds()); + ProfileEvents::increment(ProfileEvents::RemoteVFSReadBytes, bytes_read); + + return bytes_read; + }); + + auto future = task->get_future(); + + /// ThreadPool is using "bigger is higher priority" instead of "smaller is more priority". + pool.scheduleOrThrow([task]{ (*task)(); }, -request.priority); + return future; +} +} diff --git a/src/IO/ThreadPoolRemoteFSReader.h b/src/IO/ThreadPoolRemoteFSReader.h new file mode 100644 index 00000000000..ef05fdbe3ff --- /dev/null +++ b/src/IO/ThreadPoolRemoteFSReader.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +class ThreadPoolRemoteFSReader : public IAsynchronousReader +{ + +private: + ThreadPool pool; + +public: + ThreadPoolRemoteFSReader(size_t pool_size, size_t queue_size_); + + std::future submit(Request request) override; + + struct RemoteFSFileDescriptor : IFileDescriptor + { + ReadBufferPtr impl; + }; + +}; + +} From f312760dcaf6f6b487d73ec4e6d7cff93e9a1332 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 20 Sep 2021 16:59:44 +0300 Subject: [PATCH 038/919] Better --- src/Core/Settings.h | 1 + ...chronousReadIndirectBufferFromRemoteFS.cpp | 171 ++++++++++++++++++ ...ynchronousReadIndirectBufferFromRemoteFS.h | 51 ++++++ src/Disks/S3/DiskS3.cpp | 7 +- src/Disks/S3/DiskS3.h | 4 +- src/Disks/S3/registerDiskS3.cpp | 3 +- src/IO/ReadSettings.h | 12 +- src/IO/ThreadPoolRemoteFSReader.cpp | 2 - src/IO/createReadBufferFromFileBase.cpp | 10 +- src/Interpreters/Context.cpp | 11 +- 10 files changed, 252 insertions(+), 20 deletions(-) create mode 100644 src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp create mode 100644 src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0e29168f906..8c4c6327c52 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -503,6 +503,7 @@ class IColumn; M(ShortCircuitFunctionEvaluation, short_circuit_function_evaluation, ShortCircuitFunctionEvaluation::ENABLE, "Setting for short-circuit function evaluation configuration. Possible values: 'enable' - use short-circuit function evaluation for functions that are suitable for it, 'disable' - disable short-circuit function evaluation, 'force_enable' - use short-circuit function evaluation for all functions.", 0) \ \ M(String, local_filesystem_read_method, "pread", "Method of reading data from local filesystem, one of: read, pread, mmap, pread_threadpool.", 0) \ + M(String, remote_filesystem_read_method, "read", "Method of reading data from remote filesystem, one of: read, read_threadpool.", 0) \ M(Bool, local_filesystem_read_prefetch, false, "Should use prefetching when reading data from local filesystem.", 0) \ M(Bool, remote_filesystem_read_prefetch, true, "Should use prefetching when reading data from remote filesystem.", 0) \ M(Int64, read_priority, 0, "Priority to read data from local filesystem. Only supported for 'pread_threadpool' method.", 0) \ diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp new file mode 100644 index 00000000000..beb69939e5f --- /dev/null +++ b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -0,0 +1,171 @@ +#include "AsynchronousReadIndirectBufferFromRemoteFS.h" + +#include +#include +#include +#include +#include + + +namespace CurrentMetrics +{ + extern const Metric AsynchronousReadWait; +} +namespace ProfileEvents +{ + extern const Event AsynchronousReadWaitMicroseconds; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_SEEK_THROUGH_FILE; +} + + +AsynchronousReadIndirectBufferFromRemoteFS::AsynchronousReadIndirectBufferFromRemoteFS( + AsynchronousReaderPtr reader_, Int32 priority_, ReadBufferFromRemoteFSImpl impl_) + : reader(reader_), priority(priority_), impl(impl_) +{ +} + + +std::future AsynchronousReadIndirectBufferFromRemoteFS::read() +{ + IAsynchronousReader::Request request; + + auto remote_fd = std::make_shared(); + remote_fd->impl = impl; + swap(*impl); + request.descriptor = std::move(remote_fd); + request.offset = absolute_position; + request.priority = priority; + + return reader->submit(request); +} + + +void AsynchronousReadIndirectBufferFromRemoteFS::prefetch() +{ + if (prefetch_future.valid()) + { + std::cerr << "Prefetch, but not needed." << "\n"; + return; + } + + std::cerr << fmt::format("Prefetch. Internal buffer size: {}, " + "prefetch buffer size: {}, " + "impl interanl buffer size: unknown", + internal_buffer.size()); + + prefetch_future = read(); +} + + +bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() +{ + size_t size = 0; + std::cerr << fmt::format("NextImpl. Offset: {}, absolute_pos: {}", offset(), absolute_position) << std::endl; + + if (prefetch_future.valid()) + { + std::cerr << "Future is VALID!\n"; + + Stopwatch watch; + CurrentMetrics::Increment metric_increment{CurrentMetrics::AsynchronousReadWait}; + + size = prefetch_future.get(); + + watch.stop(); + ProfileEvents::increment(ProfileEvents::AsynchronousReadWaitMicroseconds, watch.elapsedMicroseconds()); + + swap(*impl); + prefetch_future = {}; + } + else + { + std::cerr << "Future is NOT VALID!\n"; + + size = read().get(); + swap(*impl); + prefetch_future = {}; + } + + if (size) + { + absolute_position += size; + return true; + } + + return false; +} + + +off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence) +{ + if (whence == SEEK_CUR) + { + /// If position within current working buffer - shift pos. + if (!working_buffer.empty() && static_cast(getPosition() + offset_) < absolute_position) + { + pos += offset_; + return getPosition(); + } + else + { + absolute_position += offset_; + } + } + else if (whence == SEEK_SET) + { + /// If position is within current working buffer - shift pos. + if (!working_buffer.empty() + && static_cast(offset_) >= absolute_position - working_buffer.size() + && size_t(offset_) < absolute_position) + { + pos = working_buffer.end() - (absolute_position - offset_); + assert(pos >= working_buffer.begin()); + assert(pos <= working_buffer.end()); + return getPosition(); + } + else + { + absolute_position = offset_; + } + } + else + throw Exception("Only SEEK_SET or SEEK_CUR modes are allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); + + if (prefetch_future.valid()) + { + std::cerr << "Ignoring prefetched data" << "\n"; + prefetch_future.wait(); + prefetch_future = {}; + } + + std::cerr << "Seek with new absolute_position: " << absolute_position << std::endl; + impl->seek(absolute_position, SEEK_SET); + pos = working_buffer.end(); + + return absolute_position; +} + + +void AsynchronousReadIndirectBufferFromRemoteFS::finalize() +{ + if (prefetch_future.valid()) + { + prefetch_future.wait(); + prefetch_future = {}; + } +} + + +AsynchronousReadIndirectBufferFromRemoteFS::~AsynchronousReadIndirectBufferFromRemoteFS() +{ + finalize(); +} + +} diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h new file mode 100644 index 00000000000..eb62300ebec --- /dev/null +++ b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -0,0 +1,51 @@ +#pragma once + +#if !defined(ARCADIA_BUILD) +#include +#endif + +#include +#include +#include +#include +#include + + +namespace DB +{ + +/// Reads data from S3/HDFS using stored paths in metadata. +class AsynchronousReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase +{ +public: + explicit AsynchronousReadIndirectBufferFromRemoteFS( + AsynchronousReaderPtr reader_, Int32 priority_, ReadBufferFromRemoteFSImpl impl_); + + ~AsynchronousReadIndirectBufferFromRemoteFS() override; + + off_t seek(off_t offset_, int whence) override; + + off_t getPosition() override { return absolute_position - available(); } + + String getFileName() const override { return metadata_file_path; } + + void prefetch() override; + +private: + bool nextImpl() override; + + void finalize(); + + std::future read(); + + String metadata_file_path; + size_t absolute_position = 0; + + AsynchronousReaderPtr reader; + Int32 priority; + ReadBufferFromRemoteFSImpl impl; + + std::future prefetch_future; +}; + +} diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index aae5bb8fc26..93174d808c3 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -236,8 +236,9 @@ std::unique_ptr DiskS3::readFile(const String & path, co settings->client, bucket, metadata, settings->s3_max_single_read_retries, read_settings.remote_fs_buffer_size); - if (settings->async_read) + if (read_settings.remote_fs_method == RemoteFSReadMethod::read_threadpool) { + std::cerr << "\n\ncreating read buffer with thtread pool\n\n"; static AsynchronousReaderPtr reader = std::make_shared(16, 1000000); return std::make_unique(reader, read_settings.priority, std::move(s3_impl)); //return std::make_unique(std::move(buf), settings->min_bytes_for_seek); @@ -1065,8 +1066,7 @@ DiskS3Settings::DiskS3Settings( bool send_metadata_, int thread_pool_size_, int list_object_keys_size_, - int objects_chunk_size_to_delete_, - bool async_read_) + int objects_chunk_size_to_delete_) : client(client_) , s3_max_single_read_retries(s3_max_single_read_retries_) , s3_min_upload_part_size(s3_min_upload_part_size_) @@ -1076,7 +1076,6 @@ DiskS3Settings::DiskS3Settings( , thread_pool_size(thread_pool_size_) , list_object_keys_size(list_object_keys_size_) , objects_chunk_size_to_delete(objects_chunk_size_to_delete_) - , async_read(async_read_) { } diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index fafd115946d..e2141b6196e 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -36,8 +36,7 @@ struct DiskS3Settings bool send_metadata_, int thread_pool_size_, int list_object_keys_size_, - int objects_chunk_size_to_delete_, - bool async_read); + int objects_chunk_size_to_delete_); std::shared_ptr client; size_t s3_max_single_read_retries; @@ -48,7 +47,6 @@ struct DiskS3Settings int thread_pool_size; int list_object_keys_size; int objects_chunk_size_to_delete; - bool async_read; }; diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index aab97d10760..decf454d257 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -161,8 +161,7 @@ std::unique_ptr getSettings(const Poco::Util::AbstractConfigurat config.getBool(config_prefix + ".send_metadata", false), config.getInt(config_prefix + ".thread_pool_size", 16), config.getInt(config_prefix + ".list_object_keys_size", 1000), - config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000), - config.getBool(config_prefix + ".async_read", false)); + config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000)); } } diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index 100041d3dec..642da1cff37 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -6,7 +6,7 @@ namespace DB { -enum class ReadMethod +enum class LocalFSReadMethod { /** * Simple synchronous reads with 'read'. @@ -43,12 +43,20 @@ enum class ReadMethod pread_fake_async }; +enum class RemoteFSReadMethod +{ + read, + read_threadpool, +}; + class MMappedFileCache; struct ReadSettings { /// Method to use reading from local filesystem. - ReadMethod local_fs_method = ReadMethod::pread; + LocalFSReadMethod local_fs_method = LocalFSReadMethod::pread; + /// Method to use reading from remote filesystem. + RemoteFSReadMethod remote_fs_method = RemoteFSReadMethod::read; size_t local_fs_buffer_size = DBMS_DEFAULT_BUFFER_SIZE; size_t remote_fs_buffer_size = DBMS_DEFAULT_BUFFER_SIZE; diff --git a/src/IO/ThreadPoolRemoteFSReader.cpp b/src/IO/ThreadPoolRemoteFSReader.cpp index c218ce160d0..706408e818d 100644 --- a/src/IO/ThreadPoolRemoteFSReader.cpp +++ b/src/IO/ThreadPoolRemoteFSReader.cpp @@ -39,8 +39,6 @@ std::future ThreadPoolRemoteFSReader::submit(Reques { auto task = std::make_shared>([request] { - std::cerr << "\n\nTask is execited!!!\n\n"; - setThreadName("ThreadPoolRead"); CurrentMetrics::Increment metric_increment{CurrentMetrics::Read}; Stopwatch watch(CLOCK_MONOTONIC); diff --git a/src/IO/createReadBufferFromFileBase.cpp b/src/IO/createReadBufferFromFileBase.cpp index f828199a2b4..dd667e03205 100644 --- a/src/IO/createReadBufferFromFileBase.cpp +++ b/src/IO/createReadBufferFromFileBase.cpp @@ -34,7 +34,7 @@ std::unique_ptr createReadBufferFromFileBase( size_t alignment) { if (!existing_memory - && settings.local_fs_method == ReadMethod::mmap + && settings.local_fs_method == LocalFSReadMethod::mmap && settings.mmap_threshold && settings.mmap_cache && estimated_size >= settings.mmap_threshold) @@ -56,21 +56,21 @@ std::unique_ptr createReadBufferFromFileBase( { std::unique_ptr res; - if (settings.local_fs_method == ReadMethod::read) + if (settings.local_fs_method == LocalFSReadMethod::read) { res = std::make_unique(filename, buffer_size, actual_flags, existing_memory, alignment); } - else if (settings.local_fs_method == ReadMethod::pread || settings.local_fs_method == ReadMethod::mmap) + else if (settings.local_fs_method == LocalFSReadMethod::pread || settings.local_fs_method == LocalFSReadMethod::mmap) { res = std::make_unique(filename, buffer_size, actual_flags, existing_memory, alignment); } - else if (settings.local_fs_method == ReadMethod::pread_fake_async) + else if (settings.local_fs_method == LocalFSReadMethod::pread_fake_async) { static AsynchronousReaderPtr reader = std::make_shared(); res = std::make_unique( reader, settings.priority, filename, buffer_size, actual_flags, existing_memory, alignment); } - else if (settings.local_fs_method == ReadMethod::pread_threadpool) + else if (settings.local_fs_method == LocalFSReadMethod::pread_threadpool) { static AsynchronousReaderPtr reader = std::make_shared(16, 1000000); res = std::make_unique( diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index be1cb21bbc3..67f5d9e35ff 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2803,10 +2803,17 @@ ReadSettings Context::getReadSettings() const std::string_view read_method_str = settings.local_filesystem_read_method.value; - if (auto opt_method = magic_enum::enum_cast(read_method_str)) + if (auto opt_method = magic_enum::enum_cast(read_method_str)) res.local_fs_method = *opt_method; else - throw Exception(ErrorCodes::UNKNOWN_READ_METHOD, "Unknown read method '{}'", read_method_str); + throw Exception(ErrorCodes::UNKNOWN_READ_METHOD, "Unknown read method '{}' for local filesystem", read_method_str); + + read_method_str = settings.remote_filesystem_read_method.value; + + if (auto opt_method = magic_enum::enum_cast(read_method_str)) + res.remote_fs_method = *opt_method; + else + throw Exception(ErrorCodes::UNKNOWN_READ_METHOD, "Unknown read method '{}' for remote filesystem", read_method_str); res.local_fs_prefetch = settings.local_filesystem_read_prefetch; res.remote_fs_prefetch = settings.remote_filesystem_read_prefetch; From 0104cb1c8e9f1760a7d3a702b60256c0ecbf92c7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 20 Sep 2021 21:18:08 +0300 Subject: [PATCH 039/919] Better --- src/Common/ProfileEvents.cpp | 3 + ...chronousReadIndirectBufferFromRemoteFS.cpp | 62 ++++----- ...ynchronousReadIndirectBufferFromRemoteFS.h | 12 +- src/Disks/DiskWebServer.cpp | 16 ++- src/Disks/HDFS/DiskHDFS.cpp | 17 ++- src/Disks/ReadIndirectBufferFromRemoteFS.cpp | 23 ++-- src/Disks/ReadIndirectBufferFromRemoteFS.h | 10 +- src/Disks/S3/DiskS3.cpp | 8 +- src/IO/ReadBufferFromRemoteFS.cpp | 12 +- src/IO/ReadBufferFromRemoteFS.h | 5 + src/IO/SeekAvoidingReadBuffer.h | 2 + src/IO/ThreadPoolRemoteFSReader.cpp | 16 +-- .../configs/config.d/async_read.xml | 7 + tests/integration/test_merge_tree_s3/test.py | 122 ++++++++++-------- 14 files changed, 177 insertions(+), 138 deletions(-) create mode 100644 tests/integration/test_merge_tree_s3/configs/config.d/async_read.xml diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index fa23f4df533..cba194fe8d6 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -249,6 +249,9 @@ M(S3WriteRequestsRedirects, "Number of redirects in POST, DELETE, PUT and PATCH requests to S3 storage.") \ M(QueryMemoryLimitExceeded, "Number of times when memory limit exceeded for query.") \ \ + M(RemoteFSReadMicroseconds, "Time of reading from remote filesystem.") \ + M(RemoteFSReadBytes, "Read bytes from remote filesystem.") \ + \ M(SleepFunctionCalls, "Number of times a sleep function (sleep, sleepEachRow) has been called.") \ M(SleepFunctionMicroseconds, "Time spent sleeping due to a sleep function call.") \ \ diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp index beb69939e5f..27a088e927b 100644 --- a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -32,15 +32,16 @@ AsynchronousReadIndirectBufferFromRemoteFS::AsynchronousReadIndirectBufferFromRe } -std::future AsynchronousReadIndirectBufferFromRemoteFS::read() +std::future AsynchronousReadIndirectBufferFromRemoteFS::readNext() { IAsynchronousReader::Request request; auto remote_fd = std::make_shared(); remote_fd->impl = impl; - swap(*impl); + impl->position() = position(); + assert(!impl->hasPendingData()); + request.descriptor = std::move(remote_fd); - request.offset = absolute_position; request.priority = priority; return reader->submit(request); @@ -50,56 +51,37 @@ std::future AsynchronousReadIndirectBufferFromRemot void AsynchronousReadIndirectBufferFromRemoteFS::prefetch() { if (prefetch_future.valid()) - { - std::cerr << "Prefetch, but not needed." << "\n"; return; - } - - std::cerr << fmt::format("Prefetch. Internal buffer size: {}, " - "prefetch buffer size: {}, " - "impl interanl buffer size: unknown", - internal_buffer.size()); - - prefetch_future = read(); + prefetch_future = readNext(); } bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() { size_t size = 0; - std::cerr << fmt::format("NextImpl. Offset: {}, absolute_pos: {}", offset(), absolute_position) << std::endl; if (prefetch_future.valid()) { - std::cerr << "Future is VALID!\n"; - - Stopwatch watch; + std::cerr << "Having prefetched data\n"; CurrentMetrics::Increment metric_increment{CurrentMetrics::AsynchronousReadWait}; - + Stopwatch watch; size = prefetch_future.get(); - watch.stop(); ProfileEvents::increment(ProfileEvents::AsynchronousReadWaitMicroseconds, watch.elapsedMicroseconds()); - - swap(*impl); - prefetch_future = {}; } else { - std::cerr << "Future is NOT VALID!\n"; - - size = read().get(); - swap(*impl); - prefetch_future = {}; + std::cerr << "No prefetched data\n"; + size = readNext().get(); } if (size) { - absolute_position += size; - return true; + BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); + prefetch_future = {}; } - return false; + return size; } @@ -108,31 +90,33 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence if (whence == SEEK_CUR) { /// If position within current working buffer - shift pos. - if (!working_buffer.empty() && static_cast(getPosition() + offset_) < absolute_position) + if (!working_buffer.empty() && static_cast(getPosition() + offset_) < impl->absolute_position) { pos += offset_; return getPosition(); } else { - absolute_position += offset_; + impl->absolute_position += offset_; } } else if (whence == SEEK_SET) { /// If position is within current working buffer - shift pos. if (!working_buffer.empty() - && static_cast(offset_) >= absolute_position - working_buffer.size() - && size_t(offset_) < absolute_position) + && static_cast(offset_) >= impl->absolute_position - working_buffer.size() + && size_t(offset_) < impl->absolute_position) { - pos = working_buffer.end() - (absolute_position - offset_); + pos = working_buffer.end() - (impl->absolute_position - offset_); + assert(pos >= working_buffer.begin()); assert(pos <= working_buffer.end()); + return getPosition(); } else { - absolute_position = offset_; + impl->absolute_position = offset_; } } else @@ -142,14 +126,14 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence { std::cerr << "Ignoring prefetched data" << "\n"; prefetch_future.wait(); + impl->reset(); /// Clean the buffer, we do no need it. prefetch_future = {}; } - std::cerr << "Seek with new absolute_position: " << absolute_position << std::endl; - impl->seek(absolute_position, SEEK_SET); + impl->seek(impl->absolute_position, SEEK_SET); pos = working_buffer.end(); - return absolute_position; + return impl->absolute_position; } diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h index eb62300ebec..8ce9292e487 100644 --- a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -14,7 +14,7 @@ namespace DB { -/// Reads data from S3/HDFS using stored paths in metadata. +/// Reads data from S3/HDFS/Web using stored paths in metadata. class AsynchronousReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase { public: @@ -25,9 +25,9 @@ public: off_t seek(off_t offset_, int whence) override; - off_t getPosition() override { return absolute_position - available(); } + off_t getPosition() override { return impl->absolute_position - available(); } - String getFileName() const override { return metadata_file_path; } + String getFileName() const override { return impl->getFileName(); } void prefetch() override; @@ -36,15 +36,11 @@ private: void finalize(); - std::future read(); - - String metadata_file_path; - size_t absolute_position = 0; + std::future readNext(); AsynchronousReaderPtr reader; Int32 priority; ReadBufferFromRemoteFSImpl impl; - std::future prefetch_future; }; diff --git a/src/Disks/DiskWebServer.cpp b/src/Disks/DiskWebServer.cpp index ba6a7ed9168..a42ca1b3350 100644 --- a/src/Disks/DiskWebServer.cpp +++ b/src/Disks/DiskWebServer.cpp @@ -12,6 +12,8 @@ #include #include #include +#include +#include #include @@ -189,8 +191,18 @@ std::unique_ptr DiskWebServer::readFile(const String & p RemoteMetadata meta(path, remote_path); meta.remote_fs_objects.emplace_back(std::make_pair(remote_path, iter->second.size)); - auto reader = std::make_unique(url, meta, getContext(), read_settings.remote_fs_buffer_size); - return std::make_unique(std::move(reader), min_bytes_for_seek); + auto web_impl = std::make_unique(url, meta, getContext(), read_settings.remote_fs_buffer_size); + if (read_settings.remote_fs_method == RemoteFSReadMethod::read_threadpool) + { + static AsynchronousReaderPtr reader = std::make_shared(16, 1000000); + auto buf = std::make_unique(reader, read_settings.priority, std::move(web_impl)); + return std::make_unique(std::move(buf), min_bytes_for_seek); + } + else + { + auto buf = std::make_unique(std::move(web_impl)); + return std::make_unique(std::move(buf), min_bytes_for_seek); + } } diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index 3f0fa4cc263..2e0799a22ab 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -4,6 +4,8 @@ #include #include #include +#include +#include #include #include #include @@ -102,8 +104,19 @@ std::unique_ptr DiskHDFS::readFile(const String & path, "Read from file by path: {}. Existing HDFS objects: {}", backQuote(metadata_path + path), metadata.remote_fs_objects.size()); - auto reader = std::make_unique(config, remote_fs_root_path, metadata, read_settings.remote_fs_buffer_size); - return std::make_unique(std::move(reader), settings->min_bytes_for_seek); + auto hdfs_impl = std::make_unique(config, remote_fs_root_path, metadata, read_settings.remote_fs_buffer_size); + + if (read_settings.remote_fs_method == RemoteFSReadMethod::read_threadpool) + { + static AsynchronousReaderPtr reader = std::make_shared(16, 1000000); + auto buf = std::make_unique(reader, read_settings.priority, std::move(hdfs_impl)); + return std::make_unique(std::move(buf), settings->min_bytes_for_seek); + } + else + { + auto buf = std::make_unique(std::move(hdfs_impl)); + return std::make_unique(std::move(buf), settings->min_bytes_for_seek); + } } diff --git a/src/Disks/ReadIndirectBufferFromRemoteFS.cpp b/src/Disks/ReadIndirectBufferFromRemoteFS.cpp index fdbb3a3c8f4..2e0573fecf9 100644 --- a/src/Disks/ReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/ReadIndirectBufferFromRemoteFS.cpp @@ -10,8 +10,7 @@ namespace ErrorCodes } -ReadIndirectBufferFromRemoteFS::ReadIndirectBufferFromRemoteFS( - ReadBufferFromRemoteFSImpl impl_) : impl(std::move(impl_)) +ReadIndirectBufferFromRemoteFS::ReadIndirectBufferFromRemoteFS(ImplPtr impl_) : impl(std::move(impl_)) { } @@ -21,38 +20,38 @@ off_t ReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence) if (whence == SEEK_CUR) { /// If position within current working buffer - shift pos. - if (!working_buffer.empty() && size_t(getPosition() + offset_) < absolute_position) + if (!working_buffer.empty() && size_t(getPosition() + offset_) < impl->absolute_position) { pos += offset_; return getPosition(); } else { - absolute_position += offset_; + impl->absolute_position += offset_; } } else if (whence == SEEK_SET) { /// If position within current working buffer - shift pos. if (!working_buffer.empty() - && size_t(offset_) >= absolute_position - working_buffer.size() - && size_t(offset_) < absolute_position) + && size_t(offset_) >= impl->absolute_position - working_buffer.size() + && size_t(offset_) < impl->absolute_position) { - pos = working_buffer.end() - (absolute_position - offset_); + pos = working_buffer.end() - (impl->absolute_position - offset_); return getPosition(); } else { - absolute_position = offset_; + impl->absolute_position = offset_; } } else throw Exception("Only SEEK_SET or SEEK_CUR modes are allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); - impl->seek(absolute_position, SEEK_SET); + impl->seek(impl->absolute_position, SEEK_SET); pos = working_buffer.end(); - return absolute_position; + return impl->absolute_position; } @@ -65,10 +64,6 @@ bool ReadIndirectBufferFromRemoteFS::nextImpl() /// and assigned to current buffer. swap(*impl); - /// absolute position is shifted by a data size that was read in next() call above. - if (result) - absolute_position += working_buffer.size(); - return result; } diff --git a/src/Disks/ReadIndirectBufferFromRemoteFS.h b/src/Disks/ReadIndirectBufferFromRemoteFS.h index e15a843ae66..48af28f67f9 100644 --- a/src/Disks/ReadIndirectBufferFromRemoteFS.h +++ b/src/Disks/ReadIndirectBufferFromRemoteFS.h @@ -16,21 +16,21 @@ namespace DB /// Reads data from S3/HDFS/Web using stored paths in metadata. class ReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase { +using ImplPtr = std::unique_ptr; + public: - explicit ReadIndirectBufferFromRemoteFS(ReadBufferFromRemoteFSImpl impl_); + explicit ReadIndirectBufferFromRemoteFS(ImplPtr impl_); off_t seek(off_t offset_, int whence) override; - off_t getPosition() override { return absolute_position - available(); } + off_t getPosition() override { return impl->absolute_position - available(); } String getFileName() const override { return impl->getFileName(); } private: bool nextImpl() override; - ReadBufferFromRemoteFSImpl impl; - - size_t absolute_position = 0; + ImplPtr impl; }; } diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 93174d808c3..1f15db9460e 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -238,14 +238,14 @@ std::unique_ptr DiskS3::readFile(const String & path, co if (read_settings.remote_fs_method == RemoteFSReadMethod::read_threadpool) { - std::cerr << "\n\ncreating read buffer with thtread pool\n\n"; static AsynchronousReaderPtr reader = std::make_shared(16, 1000000); - return std::make_unique(reader, read_settings.priority, std::move(s3_impl)); - //return std::make_unique(std::move(buf), settings->min_bytes_for_seek); + auto buf = std::make_unique(reader, read_settings.priority, std::move(s3_impl)); + return std::make_unique(std::move(buf), settings->min_bytes_for_seek); } else { - return std::make_unique(std::move(s3_impl), settings->min_bytes_for_seek); + auto buf = std::make_unique(std::move(s3_impl)); + return std::make_unique(std::move(buf), settings->min_bytes_for_seek); } } diff --git a/src/IO/ReadBufferFromRemoteFS.cpp b/src/IO/ReadBufferFromRemoteFS.cpp index 353b3431bf1..fdebd736e49 100644 --- a/src/IO/ReadBufferFromRemoteFS.cpp +++ b/src/IO/ReadBufferFromRemoteFS.cpp @@ -76,6 +76,10 @@ bool ReadBufferFromRemoteFS::readImpl() /// Assign result to current buffer. swap(*current_buf); + /// absolute position is shifted by a data size that was read in next() call above. + if (result) + absolute_position += working_buffer.size(); + return result; } @@ -84,11 +88,17 @@ off_t ReadBufferFromRemoteFS::seek(off_t offset_, int whence) { if (whence != SEEK_SET) throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only SEEK_SET is allowed"); + /// We already made a seek and adjusted position in ReadIndirectBufferFromRemoteFS. + assert(offset_ == static_cast(absolute_position)); - absolute_position = offset_; current_buf = initialize(); return absolute_position; } +void ReadBufferFromRemoteFS::reset() +{ + set(nullptr, 0); +} + } diff --git a/src/IO/ReadBufferFromRemoteFS.h b/src/IO/ReadBufferFromRemoteFS.h index 468a9c0f3f6..09bbd7ccb1d 100644 --- a/src/IO/ReadBufferFromRemoteFS.h +++ b/src/IO/ReadBufferFromRemoteFS.h @@ -9,6 +9,9 @@ namespace DB class ReadBufferFromRemoteFS : public ReadBufferFromFileBase { +friend class ReadIndirectBufferFromRemoteFS; +friend class AsynchronousReadIndirectBufferFromRemoteFS; + public: explicit ReadBufferFromRemoteFS(const RemoteMetadata & metadata_); @@ -20,6 +23,8 @@ public: bool readNext() { return nextImpl(); } + void reset(); + protected: virtual SeekableReadBufferPtr createReadBuffer(const String & path) const = 0; RemoteMetadata metadata; diff --git a/src/IO/SeekAvoidingReadBuffer.h b/src/IO/SeekAvoidingReadBuffer.h index 5896efc4462..6662675cdfc 100644 --- a/src/IO/SeekAvoidingReadBuffer.h +++ b/src/IO/SeekAvoidingReadBuffer.h @@ -17,6 +17,8 @@ public: off_t seek(off_t off, int whence) override; + void prefetch() override { impl->prefetch(); } + private: UInt64 min_bytes_for_seek; /// Minimum positive seek offset which shall be executed using seek operation. }; diff --git a/src/IO/ThreadPoolRemoteFSReader.cpp b/src/IO/ThreadPoolRemoteFSReader.cpp index 706408e818d..6e4a66f2205 100644 --- a/src/IO/ThreadPoolRemoteFSReader.cpp +++ b/src/IO/ThreadPoolRemoteFSReader.cpp @@ -16,8 +16,8 @@ namespace ProfileEvents { - extern const Event RemoteVFSReadMicroseconds; - extern const Event RemoteVFSReadBytes; + extern const Event RemoteFSReadMicroseconds; + extern const Event RemoteFSReadBytes; } namespace CurrentMetrics @@ -41,20 +41,18 @@ std::future ThreadPoolRemoteFSReader::submit(Reques { setThreadName("ThreadPoolRead"); CurrentMetrics::Increment metric_increment{CurrentMetrics::Read}; - Stopwatch watch(CLOCK_MONOTONIC); - - size_t bytes_read = 0; auto * remote_fs_fd = assert_cast(request.descriptor.get()); auto * remote_fs_buf = dynamic_cast(remote_fs_fd->impl.get()); + + Stopwatch watch(CLOCK_MONOTONIC); + size_t bytes_read = 0; auto result = remote_fs_buf->readNext(); if (result) bytes_read = remote_fs_buf->buffer().size(); - - std::cerr << "Read " << bytes_read << " bytes.\n"; watch.stop(); - ProfileEvents::increment(ProfileEvents::RemoteVFSReadMicroseconds, watch.elapsedMicroseconds()); - ProfileEvents::increment(ProfileEvents::RemoteVFSReadBytes, bytes_read); + ProfileEvents::increment(ProfileEvents::RemoteFSReadMicroseconds, watch.elapsedMicroseconds()); + ProfileEvents::increment(ProfileEvents::RemoteFSReadBytes, bytes_read); return bytes_read; }); diff --git a/tests/integration/test_merge_tree_s3/configs/config.d/async_read.xml b/tests/integration/test_merge_tree_s3/configs/config.d/async_read.xml new file mode 100644 index 00000000000..4449d83779a --- /dev/null +++ b/tests/integration/test_merge_tree_s3/configs/config.d/async_read.xml @@ -0,0 +1,7 @@ + + + + read_threadpool + + + diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index be97b538ea3..acbeca5ca71 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -51,6 +51,11 @@ def cluster(): main_configs=["configs/config.d/storage_conf.xml", "configs/config.d/bg_processing_pool_conf.xml"], with_minio=True) + cluster.add_instance("node_async_read", + main_configs=["configs/config.d/storage_conf.xml", + "configs/config.d/bg_processing_pool_conf.xml"], + user_configs=["configs/config.d/async_read.xml"], + with_minio=True) logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") @@ -78,8 +83,7 @@ def generate_values(date_str, count, sign=1): return ",".join(["('{}',{},'{}')".format(x, y, z) for x, y, z in data]) -def create_table(cluster, table_name, **additional_settings): - node = cluster.instances["node"] +def create_table(node, table_name, **additional_settings): settings = { "storage_policy": "s3", "old_parts_lifetime": 0, @@ -98,6 +102,7 @@ def create_table(cluster, table_name, **additional_settings): ORDER BY (dt, id) SETTINGS {",".join((k+"="+repr(v) for k, v in settings.items()))}""" + node.query(f"DROP TABLE IF EXISTS {table_name}") node.query(create_table_statement) @@ -141,9 +146,10 @@ def wait_for_delete_s3_objects(cluster, expected, timeout=30): @pytest.fixture(autouse=True) -def drop_table(cluster): +@pytest.mark.parametrize("node_name", ["node", "node_async_read"]) +def drop_table(cluster, node_name): yield - node = cluster.instances["node"] + node = cluster.instances[node_name] minio = cluster.minio_client node.query("DROP TABLE IF EXISTS s3_test NO DELAY") @@ -155,18 +161,22 @@ def drop_table(cluster): for obj in list(minio.list_objects(cluster.minio_bucket, 'data/')): minio.remove_object(cluster.minio_bucket, obj.object_name) +def assert_with_read_method(node, query, read_method, expected): + assert node.query("SET remote_filesystem_read_method = '{}';{}".format(read_method, query)) == expected + @pytest.mark.parametrize( - "min_rows_for_wide_part,files_per_part", + "min_rows_for_wide_part,files_per_part,node_name", [ - (0, FILES_OVERHEAD_PER_PART_WIDE), - (8192, FILES_OVERHEAD_PER_PART_COMPACT) + (0, FILES_OVERHEAD_PER_PART_WIDE, "node"), + (8192, FILES_OVERHEAD_PER_PART_COMPACT, "node"), + (0, FILES_OVERHEAD_PER_PART_WIDE, "node_async_read"), + (8192, FILES_OVERHEAD_PER_PART_COMPACT, "node_async_read") ] ) -def test_simple_insert_select(cluster, min_rows_for_wide_part, files_per_part): - create_table(cluster, "s3_test", min_rows_for_wide_part=min_rows_for_wide_part) - - node = cluster.instances["node"] +def test_simple_insert_select(cluster, min_rows_for_wide_part, files_per_part, node_name): + node = cluster.instances[node_name] + create_table(node, "s3_test", min_rows_for_wide_part=min_rows_for_wide_part) minio = cluster.minio_client values1 = generate_values('2020-01-03', 4096) @@ -179,21 +189,24 @@ def test_simple_insert_select(cluster, min_rows_for_wide_part, files_per_part): assert node.query("SELECT * FROM s3_test ORDER BY dt, id FORMAT Values") == values1 + "," + values2 assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD + files_per_part * 2 - assert node.query("SELECT count(*) FROM s3_test where id = 1 FORMAT Values") == "(2)" + assert node.query("SELECT count(*) FROM s3_test where id = 1 FORMAT Values") == "(2)" @pytest.mark.parametrize( - "merge_vertical", [False, True] -) -def test_insert_same_partition_and_merge(cluster, merge_vertical): + "merge_vertical,node_name", [ + (True, "node"), + (False, "node"), + (True, "node_async_read"), + (False, "node_async_read") +]) +def test_insert_same_partition_and_merge(cluster, merge_vertical, node_name): settings = {} if merge_vertical: settings['vertical_merge_algorithm_min_rows_to_activate'] = 0 settings['vertical_merge_algorithm_min_columns_to_activate'] = 0 - create_table(cluster, "s3_test", **settings) - - node = cluster.instances["node"] + node = cluster.instances[node_name] + create_table(node, "s3_test", **settings) minio = cluster.minio_client node.query("SYSTEM STOP MERGES s3_test") @@ -226,10 +239,10 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical): wait_for_delete_s3_objects(cluster, FILES_OVERHEAD_PER_PART_WIDE + FILES_OVERHEAD) -def test_alter_table_columns(cluster): - create_table(cluster, "s3_test") - - node = cluster.instances["node"] +@pytest.mark.parametrize("node_name", ["node", "node_async_read"]) +def test_alter_table_columns(cluster, node_name): + node = cluster.instances[node_name] + create_table(node, "s3_test") minio = cluster.minio_client node.query("INSERT INTO s3_test VALUES {}".format(generate_values('2020-01-03', 4096))) @@ -255,10 +268,10 @@ def test_alter_table_columns(cluster): wait_for_delete_s3_objects(cluster, FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE + 2) -def test_attach_detach_partition(cluster): - create_table(cluster, "s3_test") - - node = cluster.instances["node"] +@pytest.mark.parametrize("node_name", ["node", "node_async_read"]) +def test_attach_detach_partition(cluster, node_name): + node = cluster.instances[node_name] + create_table(node, "s3_test") minio = cluster.minio_client node.query("INSERT INTO s3_test VALUES {}".format(generate_values('2020-01-03', 4096))) @@ -287,10 +300,10 @@ def test_attach_detach_partition(cluster): assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD -def test_move_partition_to_another_disk(cluster): - create_table(cluster, "s3_test") - - node = cluster.instances["node"] +@pytest.mark.parametrize("node_name", ["node", "node_async_read"]) +def test_move_partition_to_another_disk(cluster, node_name): + node = cluster.instances[node_name] + create_table(node, "s3_test") minio = cluster.minio_client node.query("INSERT INTO s3_test VALUES {}".format(generate_values('2020-01-03', 4096))) @@ -309,10 +322,10 @@ def test_move_partition_to_another_disk(cluster): list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2 -def test_table_manipulations(cluster): - create_table(cluster, "s3_test") - - node = cluster.instances["node"] +@pytest.mark.parametrize("node_name", ["node"]) +def test_table_manipulations(cluster, node_name): + node = cluster.instances[node_name] + create_table(node, "s3_test") minio = cluster.minio_client node.query("INSERT INTO s3_test VALUES {}".format(generate_values('2020-01-03', 4096))) @@ -337,10 +350,10 @@ def test_table_manipulations(cluster): assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD -def test_move_replace_partition_to_another_table(cluster): - create_table(cluster, "s3_test") - - node = cluster.instances["node"] +@pytest.mark.parametrize("node_name", ["node", "node_async_read"]) +def test_move_replace_partition_to_another_table(cluster, node_name): + node = cluster.instances[node_name] + create_table(node, "s3_test") minio = cluster.minio_client node.query("INSERT INTO s3_test VALUES {}".format(generate_values('2020-01-03', 4096))) @@ -352,7 +365,7 @@ def test_move_replace_partition_to_another_table(cluster): assert len( list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 4 - create_table(cluster, "s3_clone") + create_table(node, "s3_clone") node.query("ALTER TABLE s3_test MOVE PARTITION '2020-01-03' TO TABLE s3_clone") node.query("ALTER TABLE s3_test MOVE PARTITION '2020-01-05' TO TABLE s3_clone") @@ -403,10 +416,10 @@ def test_move_replace_partition_to_another_table(cluster): minio.remove_object(cluster.minio_bucket, obj.object_name) -def test_freeze_unfreeze(cluster): - create_table(cluster, "s3_test") - - node = cluster.instances["node"] +@pytest.mark.parametrize("node_name", ["node"]) +def test_freeze_unfreeze(cluster, node_name): + node = cluster.instances[node_name] + create_table(node, "s3_test") minio = cluster.minio_client node.query("INSERT INTO s3_test VALUES {}".format(generate_values('2020-01-03', 4096))) @@ -428,9 +441,10 @@ def test_freeze_unfreeze(cluster): list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD -def test_s3_disk_apply_new_settings(cluster): - create_table(cluster, "s3_test") - node = cluster.instances["node"] +@pytest.mark.parametrize("node_name", ["node"]) +def test_s3_disk_apply_new_settings(cluster, node_name): + node = cluster.instances[node_name] + create_table(node, "s3_test") def get_s3_requests(): node.query("SYSTEM FLUSH LOGS") @@ -453,10 +467,10 @@ def test_s3_disk_apply_new_settings(cluster): assert get_s3_requests() - s3_requests_before == s3_requests_to_write_partition * 3 -def test_s3_disk_restart_during_load(cluster): - create_table(cluster, "s3_test") - - node = cluster.instances["node"] +@pytest.mark.parametrize("node_name", ["node"]) +def test_s3_disk_restart_during_load(cluster, node_name): + node = cluster.instances[node_name] + create_table(node, "s3_test") node.query("INSERT INTO s3_test VALUES {}".format(generate_values('2020-01-04', 1024 * 1024))) node.query("INSERT INTO s3_test VALUES {}".format(generate_values('2020-01-05', 1024 * 1024, -1))) @@ -488,11 +502,11 @@ def test_s3_disk_restart_during_load(cluster): thread.join() -def test_s3_disk_reads_on_unstable_connection(cluster): - create_table(cluster, "s3_test", storage_policy='unstable_s3') - node = cluster.instances["node"] +@pytest.mark.parametrize("node_name", ["node", "node_async_read"]) +def test_s3_disk_reads_on_unstable_connection(cluster, node_name): + node = cluster.instances[node_name] + create_table(node, "s3_test", storage_policy='unstable_s3') node.query("INSERT INTO s3_test SELECT today(), *, toString(*) FROM system.numbers LIMIT 9000000") for i in range(30): print(f"Read sequence {i}") assert node.query("SELECT sum(id) FROM s3_test").splitlines() == ["40499995500000"] - From cacc81ca615e93c365f16d4dd6d4a76f879cb3b5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 Sep 2021 02:25:50 +0300 Subject: [PATCH 040/919] Fix --- src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp | 3 ++- src/Disks/ReadIndirectBufferFromRemoteFS.cpp | 3 +++ src/IO/ReadBufferFromRemoteFS.cpp | 5 +---- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp index 27a088e927b..1802ecd4661 100644 --- a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -78,9 +78,10 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() if (size) { BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); - prefetch_future = {}; + impl->absolute_position += working_buffer.size(); } + prefetch_future = {}; return size; } diff --git a/src/Disks/ReadIndirectBufferFromRemoteFS.cpp b/src/Disks/ReadIndirectBufferFromRemoteFS.cpp index 2e0573fecf9..f03c84952db 100644 --- a/src/Disks/ReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/ReadIndirectBufferFromRemoteFS.cpp @@ -64,6 +64,9 @@ bool ReadIndirectBufferFromRemoteFS::nextImpl() /// and assigned to current buffer. swap(*impl); + if (result) + impl->absolute_position += working_buffer.size(); + return result; } diff --git a/src/IO/ReadBufferFromRemoteFS.cpp b/src/IO/ReadBufferFromRemoteFS.cpp index fdebd736e49..2730b90bec5 100644 --- a/src/IO/ReadBufferFromRemoteFS.cpp +++ b/src/IO/ReadBufferFromRemoteFS.cpp @@ -75,10 +75,7 @@ bool ReadBufferFromRemoteFS::readImpl() auto result = current_buf->next(); /// Assign result to current buffer. swap(*current_buf); - - /// absolute position is shifted by a data size that was read in next() call above. - if (result) - absolute_position += working_buffer.size(); + /// Absolute position is updated by *IndirectBufferFromRemoteFS only. return result; } From 40ee75fe97211eca6524724c83f8ad5bd69c2cbe Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 Sep 2021 10:00:56 +0300 Subject: [PATCH 041/919] Common thread pool for disks --- src/Disks/DiskWebServer.cpp | 3 +-- src/Disks/HDFS/DiskHDFS.cpp | 3 +-- src/Disks/IDiskRemote.cpp | 8 ++++++++ src/Disks/IDiskRemote.h | 6 ++++++ src/Disks/S3/DiskS3.cpp | 3 +-- src/IO/ReadBufferFromRemoteFS.cpp | 2 +- 6 files changed, 18 insertions(+), 7 deletions(-) diff --git a/src/Disks/DiskWebServer.cpp b/src/Disks/DiskWebServer.cpp index a42ca1b3350..863947ecd69 100644 --- a/src/Disks/DiskWebServer.cpp +++ b/src/Disks/DiskWebServer.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include @@ -194,7 +193,7 @@ std::unique_ptr DiskWebServer::readFile(const String & p auto web_impl = std::make_unique(url, meta, getContext(), read_settings.remote_fs_buffer_size); if (read_settings.remote_fs_method == RemoteFSReadMethod::read_threadpool) { - static AsynchronousReaderPtr reader = std::make_shared(16, 1000000); + auto reader = IDiskRemote::getThreadPoolReader(); auto buf = std::make_unique(reader, read_settings.priority, std::move(web_impl)); return std::make_unique(std::move(buf), min_bytes_for_seek); } diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index 2e0799a22ab..50a8806fa7b 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include @@ -108,7 +107,7 @@ std::unique_ptr DiskHDFS::readFile(const String & path, if (read_settings.remote_fs_method == RemoteFSReadMethod::read_threadpool) { - static AsynchronousReaderPtr reader = std::make_shared(16, 1000000); + auto reader = getThreadPoolReader(); auto buf = std::make_unique(reader, read_settings.priority, std::move(hdfs_impl)); return std::make_unique(std::move(buf), settings->min_bytes_for_seek); } diff --git a/src/Disks/IDiskRemote.cpp b/src/Disks/IDiskRemote.cpp index b9bce147cfd..d09fe1e8880 100644 --- a/src/Disks/IDiskRemote.cpp +++ b/src/Disks/IDiskRemote.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB @@ -496,4 +497,11 @@ String IDiskRemote::getUniqueId(const String & path) const return id; } + +AsynchronousReaderPtr IDiskRemote::getThreadPoolReader() +{ + static AsynchronousReaderPtr reader = std::make_shared(16, 1000000); + return reader; +} + } diff --git a/src/Disks/IDiskRemote.h b/src/Disks/IDiskRemote.h index fb837316327..74661d96fb6 100644 --- a/src/Disks/IDiskRemote.h +++ b/src/Disks/IDiskRemote.h @@ -35,6 +35,10 @@ protected: using RemoteFSPathKeeperPtr = std::shared_ptr; +class IAsynchronousReader; +using AsynchronousReaderPtr = std::shared_ptr; + + /// Base Disk class for remote FS's, which are not posix-compatible (DiskS3 and DiskHDFS) class IDiskRemote : public IDisk { @@ -127,6 +131,8 @@ public: virtual RemoteFSPathKeeperPtr createFSPathKeeper() const = 0; + static AsynchronousReaderPtr getThreadPoolReader(); + protected: Poco::Logger * log; const String name; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 1f15db9460e..4128381b071 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -29,7 +29,6 @@ #include #include #include -#include #include // Y_IGNORE #include // Y_IGNORE @@ -238,7 +237,7 @@ std::unique_ptr DiskS3::readFile(const String & path, co if (read_settings.remote_fs_method == RemoteFSReadMethod::read_threadpool) { - static AsynchronousReaderPtr reader = std::make_shared(16, 1000000); + auto reader = getThreadPoolReader(); auto buf = std::make_unique(reader, read_settings.priority, std::move(s3_impl)); return std::make_unique(std::move(buf), settings->min_bytes_for_seek); } diff --git a/src/IO/ReadBufferFromRemoteFS.cpp b/src/IO/ReadBufferFromRemoteFS.cpp index 2730b90bec5..88b3fffcf8b 100644 --- a/src/IO/ReadBufferFromRemoteFS.cpp +++ b/src/IO/ReadBufferFromRemoteFS.cpp @@ -81,7 +81,7 @@ bool ReadBufferFromRemoteFS::readImpl() } -off_t ReadBufferFromRemoteFS::seek(off_t offset_, int whence) +off_t ReadBufferFromRemoteFS::seek([[maybe_unused]] off_t offset_, int whence) { if (whence != SEEK_SET) throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only SEEK_SET is allowed"); From e301457e91878fabfc9ac030d62e2d37a79c14ab Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 Sep 2021 15:40:48 +0300 Subject: [PATCH 042/919] Fix prefetch mistake --- ...AsynchronousReadIndirectBufferFromRemoteFS.cpp | 15 +++++++++++---- src/Disks/DiskRestartProxy.cpp | 2 ++ src/IO/ReadBufferFromRemoteFS.h | 2 ++ src/IO/ThreadPoolRemoteFSReader.h | 2 +- tests/integration/test_merge_tree_s3/test.py | 5 +---- 5 files changed, 17 insertions(+), 9 deletions(-) diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp index 1802ecd4661..ada29155b3f 100644 --- a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -38,8 +38,6 @@ std::future AsynchronousReadIndirectBufferFromRemot auto remote_fd = std::make_shared(); remote_fd->impl = impl; - impl->position() = position(); - assert(!impl->hasPendingData()); request.descriptor = std::move(remote_fd); request.priority = priority; @@ -52,6 +50,12 @@ void AsynchronousReadIndirectBufferFromRemoteFS::prefetch() { if (prefetch_future.valid()) return; + + if (impl->initialized()) + { + impl->position() = impl->buffer().end(); /// May be should try to do this differently. + assert(!impl->hasPendingData()); + } prefetch_future = readNext(); } @@ -62,7 +66,6 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() if (prefetch_future.valid()) { - std::cerr << "Having prefetched data\n"; CurrentMetrics::Increment metric_increment{CurrentMetrics::AsynchronousReadWait}; Stopwatch watch; size = prefetch_future.get(); @@ -71,7 +74,11 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() } else { - std::cerr << "No prefetched data\n"; + if (impl->initialized()) + { + impl->position() = position(); + assert(!impl->hasPendingData()); + } size = readNext().get(); } diff --git a/src/Disks/DiskRestartProxy.cpp b/src/Disks/DiskRestartProxy.cpp index 8f12452e2ed..0d762f6fa9b 100644 --- a/src/Disks/DiskRestartProxy.cpp +++ b/src/Disks/DiskRestartProxy.cpp @@ -20,6 +20,8 @@ public: RestartAwareReadBuffer(const DiskRestartProxy & disk, std::unique_ptr impl_) : ReadBufferFromFileDecorator(std::move(impl_)), lock(disk.mutex) { } + void prefetch() override { impl->prefetch(); } + private: ReadLock lock; }; diff --git a/src/IO/ReadBufferFromRemoteFS.h b/src/IO/ReadBufferFromRemoteFS.h index 09bbd7ccb1d..842755945a8 100644 --- a/src/IO/ReadBufferFromRemoteFS.h +++ b/src/IO/ReadBufferFromRemoteFS.h @@ -25,6 +25,8 @@ public: void reset(); + bool initialized() const { return current_buf != nullptr; } + protected: virtual SeekableReadBufferPtr createReadBuffer(const String & path) const = 0; RemoteMetadata metadata; diff --git a/src/IO/ThreadPoolRemoteFSReader.h b/src/IO/ThreadPoolRemoteFSReader.h index ef05fdbe3ff..cb2c4240a20 100644 --- a/src/IO/ThreadPoolRemoteFSReader.h +++ b/src/IO/ThreadPoolRemoteFSReader.h @@ -20,7 +20,7 @@ public: std::future submit(Request request) override; - struct RemoteFSFileDescriptor : IFileDescriptor + struct RemoteFSFileDescriptor : public IFileDescriptor { ReadBufferPtr impl; }; diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index acbeca5ca71..a9dc63cc8a9 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -161,9 +161,6 @@ def drop_table(cluster, node_name): for obj in list(minio.list_objects(cluster.minio_bucket, 'data/')): minio.remove_object(cluster.minio_bucket, obj.object_name) -def assert_with_read_method(node, query, read_method, expected): - assert node.query("SET remote_filesystem_read_method = '{}';{}".format(read_method, query)) == expected - @pytest.mark.parametrize( "min_rows_for_wide_part,files_per_part,node_name", @@ -189,7 +186,7 @@ def test_simple_insert_select(cluster, min_rows_for_wide_part, files_per_part, n assert node.query("SELECT * FROM s3_test ORDER BY dt, id FORMAT Values") == values1 + "," + values2 assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD + files_per_part * 2 - assert node.query("SELECT count(*) FROM s3_test where id = 1 FORMAT Values") == "(2)" + assert node.query("SELECT count(*) FROM s3_test where id = 1 FORMAT Values") == "(2)" @pytest.mark.parametrize( From 008c3c812b09486894b1f6f617cff4de52364ce1 Mon Sep 17 00:00:00 2001 From: Kevin Michel Date: Mon, 20 Sep 2021 19:08:59 +0200 Subject: [PATCH 043/919] Allow non-replicated ALTER TABLE FETCH/ATTACH in Replicated databases `ALTER TABLE ... FETCH` and `ALTER TABLE ... ATTACH` queries were disabled in the Replicated database engine, because it could cause accidental duplication of data. This enables these queries but without replicating them. In the case of `FETCH`, the part will only be fetched on the server where the query is issued. Similarly, in the case of `ATTACH`, the attached part only needs to be available on the server where the query is issued. If the table itself is using one of the Replicated MergeTree engines, the attached data is then replicated by the table engine itself, without intervention of the database engine. This change is meant to help with live backup/restore when using the Replicated database engine, using FREEZE for backup and ATTACH for restore. --- .../en/engines/database-engines/replicated.md | 2 + src/Interpreters/InterpreterAlterQuery.cpp | 4 +- src/Parsers/ASTAlterQuery.cpp | 11 +++++ src/Parsers/ASTAlterQuery.h | 4 ++ .../test_replicated_database/test.py | 49 ++++++++++++++++++- 5 files changed, 68 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/database-engines/replicated.md b/docs/en/engines/database-engines/replicated.md index 5ce73604207..67671e11cab 100644 --- a/docs/en/engines/database-engines/replicated.md +++ b/docs/en/engines/database-engines/replicated.md @@ -35,6 +35,8 @@ The [system.clusters](../../operations/system-tables/clusters.md) system table c When creating a new replica of the database, this replica creates tables by itself. If the replica has been unavailable for a long time and has lagged behind the replication log — it checks its local metadata with the current metadata in ZooKeeper, moves the extra tables with data to a separate non-replicated database (so as not to accidentally delete anything superfluous), creates the missing tables, updates the table names if they have been renamed. The data is replicated at the `ReplicatedMergeTree` level, i.e. if the table is not replicated, the data will not be replicated (the database is responsible only for metadata). +[`ALTER TABLE FETCH`](../../sql-reference/statements/alter/partition.md) and [`ALTER TABLE ATTACH`](../../sql-reference/statements/alter/partition.md) queries are allowed but not replicated. The database engine will only add the partition/part to the current replica. However, if the table itself uses a Replicated table engine, then the data will be replicated after using `ATTACH`. + ## Usage Example {#usage-example} Creating a cluster with three hosts: diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 08b9846b325..e2b2d487be5 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -55,7 +55,9 @@ BlockIO InterpreterAlterQuery::execute() DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name); if (typeid_cast(database.get()) - && !getContext()->getClientInfo().is_replicated_database_internal) + && !getContext()->getClientInfo().is_replicated_database_internal + && !alter.isAttachAlter() + && !alter.isFetchAlter()) { auto guard = DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name); guard->releaseTableLock(); diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 4b70f25f5c5..140ef972eeb 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -450,6 +450,17 @@ bool ASTAlterQuery::isFreezeAlter() const || isOneCommandTypeOnly(ASTAlterCommand::UNFREEZE_PARTITION) || isOneCommandTypeOnly(ASTAlterCommand::UNFREEZE_ALL); } +bool ASTAlterQuery::isAttachAlter() const +{ + return isOneCommandTypeOnly(ASTAlterCommand::ATTACH_PARTITION); +} + +bool ASTAlterQuery::isFetchAlter() const +{ + return isOneCommandTypeOnly(ASTAlterCommand::FETCH_PARTITION); +} + + /** Get the text that identifies this element. */ String ASTAlterQuery::getID(char delim) const { diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index a5410e797ce..432f0c887fd 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -218,6 +218,10 @@ public: bool isFreezeAlter() const; + bool isAttachAlter() const; + + bool isFetchAlter() const; + String getID(char) const override; ASTPtr clone() const override; diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 450caafb9ba..ff400f4466b 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1,3 +1,5 @@ +import os +import shutil import time import re import pytest @@ -16,7 +18,7 @@ snapshot_recovering_node = cluster.add_instance('snapshot_recovering_node', main all_nodes = [main_node, dummy_node, competing_node, snapshotting_node, snapshot_recovering_node] -uuid_regex = re.compile("[0-9a-f]{8}\-[0-9a-f]{4}\-[0-9a-f]{4}\-[0-9a-f]{4}\-[0-9a-f]{12}") +uuid_regex = re.compile("[0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12}") def assert_create_query(nodes, table_name, expected): replace_uuid = lambda x: re.sub(uuid_regex, "uuid", x) query = "show create table {}".format(table_name) @@ -100,6 +102,51 @@ def test_simple_alter_table(started_cluster, engine): assert_create_query([main_node, dummy_node, competing_node], name, expected) +def get_table_uuid(database, name): + return main_node.query(f"SELECT uuid FROM system.tables WHERE database = '{database}' and name = '{name}'").strip() + + +@pytest.fixture(scope="module", name="attachable_part") +def fixture_attachable_part(started_cluster): + main_node.query(f"CREATE DATABASE testdb_attach_atomic ENGINE = Atomic") + main_node.query(f"CREATE TABLE testdb_attach_atomic.test (CounterID UInt32) ENGINE = MergeTree ORDER BY (CounterID)") + main_node.query(f"INSERT INTO testdb_attach_atomic.test VALUES (123)") + main_node.query(f"ALTER TABLE testdb_attach_atomic.test FREEZE WITH NAME 'test_attach'") + table_uuid = get_table_uuid("testdb_attach_atomic", "test") + return os.path.join(main_node.path, f"database/shadow/test_attach/store/{table_uuid[:3]}/{table_uuid}/all_1_1_0") + + + +@pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) +def test_alter_attach(started_cluster, attachable_part, engine): + name = "alter_attach_test_{}".format(engine) + main_node.query(f"CREATE TABLE testdb.{name} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)") + table_uuid = get_table_uuid("testdb", name) + # Provide and attach a part to the main node + shutil.copytree( + attachable_part, os.path.join(main_node.path, f"database/store/{table_uuid[:3]}/{table_uuid}/detached/all_1_1_0") + ) + main_node.query(f"ALTER TABLE testdb.{name} ATTACH PART 'all_1_1_0'") + # On the main node, data is attached + assert main_node.query(f"SELECT CounterID FROM testdb.{name}") == "123\n" + # On the other node, data is replicated only if using a Replicated table engine + if engine == "ReplicatedMergeTree": + assert dummy_node.query(f"SELECT CounterID FROM testdb.{name}") == "123\n" + else: + assert dummy_node.query(f"SELECT CounterID FROM testdb.{name}") == "" + + +def test_alter_fetch(started_cluster): + main_node.query("CREATE TABLE testdb.fetch_source (CounterID UInt32) ENGINE = ReplicatedMergeTree ORDER BY (CounterID)") + main_node.query("CREATE TABLE testdb.fetch_target (CounterID UInt32) ENGINE = ReplicatedMergeTree ORDER BY (CounterID)") + main_node.query("INSERT INTO testdb.fetch_source VALUES (123)") + table_uuid = get_table_uuid("testdb", "fetch_source") + main_node.query(f"ALTER TABLE testdb.fetch_target FETCH PART 'all_0_0_0' FROM '/clickhouse/tables/{table_uuid}/{{shard}}' ") + detached_parts_query = "SELECT name FROM system.detached_parts WHERE database='testdb' AND table='fetch_target'" + assert main_node.query(detached_parts_query) == "all_0_0_0\n" + assert dummy_node.query(detached_parts_query) == "" + + def test_alters_from_different_replicas(started_cluster): # test_alters_from_different_replicas competing_node.query("CREATE DATABASE IF NOT EXISTS testdb ENGINE = Replicated('/clickhouse/databases/test1', 'shard1', 'replica3');") From e3fa2e36346f86a6bac909aa05ab3782aabc9b12 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 21 Sep 2021 16:11:35 +0000 Subject: [PATCH 044/919] fix fix update tests update tests --- .../FileLog/FileLogDirectoryWatcher.cpp | 21 +++-- .../FileLog/FileLogDirectoryWatcher.h | 16 +++- src/Storages/FileLog/FileLogSettings.h | 6 +- src/Storages/FileLog/FileLogSource.cpp | 4 +- .../FileLog/ReadBufferFromFileLog.cpp | 10 +- src/Storages/FileLog/StorageFileLog.cpp | 93 +++++++++++-------- src/Storages/FileLog/StorageFileLog.h | 8 +- .../0_stateless/02023_storage_filelog.sh | 18 +++- 8 files changed, 111 insertions(+), 65 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index bc731500d61..b27b4719933 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -3,8 +3,11 @@ #include FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_) - : path(path_), dw(std::make_shared(path)) + : path(path_), dw(std::make_shared(path)), log(&Poco::Logger::get("DirectoryIterator (" + path + ")")) { + /// DW_ITEM_MOVED_FROM and DW_ITEM_MOVED_TO events will only be reported on Linux. + /// On other platforms, a file rename or move operation will be reported via a + /// DW_ITEM_REMOVED and a DW_ITEM_ADDED event. The order of these two events is not defined. dw->itemAdded += Poco::delegate(this, &FileLogDirectoryWatcher::onItemAdded); dw->itemRemoved += Poco::delegate(this, &FileLogDirectoryWatcher::onItemRemoved); dw->itemModified += Poco::delegate(this, &FileLogDirectoryWatcher::onItemModified); @@ -12,7 +15,7 @@ FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_) dw->itemMovedTo += Poco::delegate(this, &FileLogDirectoryWatcher::onItemMovedTo); } -FileLogDirectoryWatcher::Events FileLogDirectoryWatcher::getEvents() +FileLogDirectoryWatcher::Events FileLogDirectoryWatcher::getEventsAndReset() { std::lock_guard lock(mutex); Events res; @@ -20,9 +23,12 @@ FileLogDirectoryWatcher::Events FileLogDirectoryWatcher::getEvents() return res; } -bool FileLogDirectoryWatcher::hasError() const +FileLogDirectoryWatcher::Error FileLogDirectoryWatcher::getErrorAndReset() { - return error; + std::lock_guard lock(mutex); + Error old_error = error; + error = {}; + return old_error; } const std::string & FileLogDirectoryWatcher::getPath() const @@ -82,7 +88,10 @@ void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::Direct events.emplace_back(de); } -void FileLogDirectoryWatcher::onError(const Poco::Exception &) +void FileLogDirectoryWatcher::onError(const Poco::Exception & e) { - error = true; + std::lock_guard lock(mutex); + LOG_ERROR(log, "Error happened during watching directory {}: {}", path, error.error_msg); + error.has_error = true; + error.error_msg = e.message(); } diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index 651d1d9db2d..d21d3d1e1ce 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -4,6 +4,8 @@ #include #include +#include + #include #include @@ -19,12 +21,18 @@ public: using Events = std::vector; + struct Error + { + bool has_error = false; + std::string error_msg = {}; + }; + explicit FileLogDirectoryWatcher(const std::string & path_); ~FileLogDirectoryWatcher() = default; - Events getEvents(); + Events getEventsAndReset(); - bool hasError() const; + Error getErrorAndReset(); const std::string & getPath() const; @@ -40,9 +48,11 @@ private: const std::string path; std::shared_ptr dw; + Poco::Logger * log; + std::mutex mutex; Events events; - bool error = false; + Error error; }; diff --git a/src/Storages/FileLog/FileLogSettings.h b/src/Storages/FileLog/FileLogSettings.h index cfea9d1e195..ba76b4d02da 100644 --- a/src/Storages/FileLog/FileLogSettings.h +++ b/src/Storages/FileLog/FileLogSettings.h @@ -12,10 +12,8 @@ class ASTStorage; #define FILELOG_RELATED_SETTINGS(M) \ /* default is stream_poll_timeout_ms */ \ M(Milliseconds, filelog_poll_timeout_ms, 0, "Timeout for single poll from FileLog.", 0) \ - /* default is min(max_block_size, kafka_max_block_size)*/ \ - M(UInt64, filelog_poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single Kafka poll.", 0) \ - /* default is = max_insert_block_size / kafka_num_consumers */ \ - M(UInt64, filelog_max_block_size, 0, "Number of row collected by poll(s) for flushing data from Kafka.", 0) \ + M(UInt64, filelog_poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single filelog poll.", 0) \ + M(UInt64, filelog_max_block_size, 0, "Number of row collected by poll(s) for flushing data from filelog.", 0) \ M(UInt64, filelog_max_threads, 8, "Number of max threads to parse files, default is 8", 0) #define LIST_OF_FILELOG_SETTINGS(M) \ diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index 08a8c4b6868..c1feb721af0 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -121,9 +121,7 @@ Chunk FileLogSource::generate() { total_rows = total_rows + new_rows; } - - /// poll succeed, but parse failed - else + else /// poll succeed, but parse failed { ++failed_poll_attempts; } diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index bf9cd2d9fce..83ad412f202 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -40,7 +40,6 @@ void ReadBufferFromFileLog::cleanUnprocessed() bool ReadBufferFromFileLog::poll() { - if (hasMorePolledRecords()) { allowed = true; @@ -98,7 +97,7 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ size_t read_records_size = 0; const auto & file_names = storage.getFileNames(); - auto & file_status = storage.getFileStatus(); + auto & file_status = storage.getFileStatuses(); size_t files_per_stream = file_names.size() / max_streams_number; size_t start = stream_number * files_per_stream; @@ -113,19 +112,24 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ auto reader = std::ifstream(file_names[i]); reader.seekg(0, reader.end); + assert(reader.good()); + auto stream_end = reader.tellg(); + assert(reader.good()); reader.seekg(file.last_read_position); + assert(reader.good()); + Record record; while (read_records_size < need_records_size && reader.tellg() < stream_end) { - Record record; std::getline(reader, record); new_records.emplace_back(record); ++read_records_size; } file.last_read_position = reader.tellg(); + assert(reader.good()); if (reader.tellg() == stream_end) { diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 10d838c84ec..fa638c2a884 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -38,6 +38,7 @@ namespace ErrorCodes namespace { const auto RESCHEDULE_MS = 500; + const auto BACKOFF_TRESHOLD = 32000; const auto MAX_THREAD_WORK_DURATION_MS = 60000; // once per minute leave do reschedule (we can't lock threads in pool forever) } @@ -54,40 +55,48 @@ StorageFileLog::StorageFileLog( , path(getContext()->getUserFilesPath() + "/" + relative_path_) , format_name(format_name_) , log(&Poco::Logger::get("StorageFileLog (" + table_id_.table_name + ")")) + , milliseconds_to_wait(RESCHEDULE_MS) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); setInMemoryMetadata(storage_metadata); - if (std::filesystem::is_regular_file(path)) + try { - auto normal_path = std::filesystem::path(path).lexically_normal().native(); - file_status[normal_path] = FileContext{}; - file_names.push_back(normal_path); - } - else if (std::filesystem::is_directory(path)) - { - path_is_directory = true; - /// Just consider file with depth 1 - for (const auto & dir_entry : std::filesystem::directory_iterator{path}) + if (std::filesystem::is_regular_file(path)) { - if (dir_entry.is_regular_file()) + auto normal_path = std::filesystem::path(path).lexically_normal().native(); + file_statuses[normal_path] = FileContext{}; + file_names.push_back(normal_path); + } + else if (std::filesystem::is_directory(path)) + { + path_is_directory = true; + /// Just consider file with depth 1 + for (const auto & dir_entry : std::filesystem::directory_iterator{path}) { - auto normal_path = std::filesystem::path(dir_entry.path()).lexically_normal().native(); - file_status[normal_path] = FileContext{}; - file_names.push_back(normal_path); + if (dir_entry.is_regular_file()) + { + auto normal_path = std::filesystem::path(dir_entry.path()).lexically_normal().native(); + file_statuses[normal_path] = FileContext{}; + file_names.push_back(normal_path); + } } } + else + { + throw Exception("The path neither a regular file, nor a directory", ErrorCodes::BAD_ARGUMENTS); + } + + directory_watch = std::make_unique(path); + + auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); + task = std::make_shared(std::move(thread)); } - else + catch (...) { - throw Exception("The path neither a regular file, nor a directory", ErrorCodes::BAD_ARGUMENTS); + tryLogCurrentException(__PRETTY_FUNCTION__); } - - directory_watch = std::make_unique(path); - - auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); - task = std::make_shared(std::move(thread)); } Pipe StorageFileLog::read( @@ -101,7 +110,7 @@ Pipe StorageFileLog::read( { std::lock_guard lock(status_mutex); - updateFileStatus(); + updateFileStatuses(); /// No files to parse if (file_names.empty()) @@ -155,7 +164,6 @@ size_t StorageFileLog::getPollMaxBatchSize() const { size_t batch_size = filelog_settings->filelog_poll_max_batch_size.changed ? filelog_settings->filelog_poll_max_batch_size.value : getContext()->getSettingsRef().max_block_size.value; - return std::min(batch_size, getMaxBlockSize()); } @@ -211,12 +219,17 @@ void StorageFileLog::threadFunc() LOG_DEBUG(log, "Started streaming to {} attached views", dependencies_count); - auto file_status_no_update = streamToViews(); - if (file_status_no_update) + if (streamToViews()) { LOG_TRACE(log, "Stream stalled. Reschedule."); + if (milliseconds_to_wait < BACKOFF_TRESHOLD) + milliseconds_to_wait *= 2; break; } + else + { + milliseconds_to_wait = RESCHEDULE_MS; + } auto ts = std::chrono::steady_clock::now(); auto duration = std::chrono::duration_cast(ts-start_time); @@ -225,7 +238,7 @@ void StorageFileLog::threadFunc() LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); break; } - updateFileStatus(); + updateFileStatuses(); } } } @@ -236,7 +249,7 @@ void StorageFileLog::threadFunc() // Wait for attached views if (!task->stream_cancelled) - task->holder->scheduleAfter(RESCHEDULE_MS); + task->holder->scheduleAfter(milliseconds_to_wait); } @@ -304,7 +317,7 @@ bool StorageFileLog::streamToViews() LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); - return updateFileStatus(); + return updateFileStatuses(); } void registerStorageFileLog(StorageFactory & factory) @@ -365,15 +378,15 @@ void registerStorageFileLog(StorageFactory & factory) }); } -bool StorageFileLog::updateFileStatus() +bool StorageFileLog::updateFileStatuses() { /// Do not need to hold file_status lock, since it will be holded /// by caller when call this function - auto error = directory_watch->hasError(); - if (error) - LOG_INFO(log, "Error happened during watching directory {}.", directory_watch->getPath()); + auto error = directory_watch->getErrorAndReset(); + if (error.has_error) + LOG_ERROR(log, "Error happened during watching directory {}: {}", directory_watch->getPath(), error.error_msg); - auto events = directory_watch->getEvents(); + auto events = directory_watch->getEventsAndReset(); for (const auto & event : events) { @@ -384,7 +397,7 @@ bool StorageFileLog::updateFileStatus() LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); if (std::filesystem::is_regular_file(normal_path)) { - file_status[event.path] = FileContext{}; + file_statuses[event.path] = FileContext{}; file_names.push_back(normal_path); } break; @@ -393,9 +406,9 @@ bool StorageFileLog::updateFileStatus() case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: { auto normal_path = std::filesystem::path(event.path).lexically_normal().native(); LOG_TRACE(log, "New event {} watched, path: {}", event.callback, normal_path); - if (std::filesystem::is_regular_file(normal_path) && file_status.contains(normal_path)) + if (std::filesystem::is_regular_file(normal_path) && file_statuses.contains(normal_path)) { - file_status[normal_path].status = FileStatus::UPDATED; + file_statuses[normal_path].status = FileStatus::UPDATED; } break; } @@ -405,9 +418,9 @@ bool StorageFileLog::updateFileStatus() case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: { auto normal_path = std::filesystem::path(event.path).lexically_normal().native(); LOG_TRACE(log, "New event {} watched, path: {}", event.callback, normal_path); - if (std::filesystem::is_regular_file(normal_path) && file_status.contains(normal_path)) + if (std::filesystem::is_regular_file(normal_path) && file_statuses.contains(normal_path)) { - file_status[normal_path].status = FileStatus::REMOVED; + file_statuses[normal_path].status = FileStatus::REMOVED; } } } @@ -415,9 +428,9 @@ bool StorageFileLog::updateFileStatus() std::vector valid_files; for (const auto & file_name : file_names) { - if (file_status.at(file_name).status == FileStatus::REMOVED) + if (file_statuses.at(file_name).status == FileStatus::REMOVED) { - file_status.erase(file_name); + file_statuses.erase(file_name); } else { diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 7df66b5c54f..ff94b0a548c 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -52,7 +52,7 @@ public: const auto & getFormatName() const { return format_name; } auto & getFileNames() { return file_names; } - auto & getFileStatus() { return file_status; } + auto & getFileStatuses() { return file_statuses; } protected: StorageFileLog( @@ -78,7 +78,7 @@ private: }; using NameToFile = std::unordered_map; - NameToFile file_status; + NameToFile file_statuses; std::vector file_names; @@ -86,6 +86,8 @@ private: std::unique_ptr directory_watch = nullptr; + uint64_t milliseconds_to_wait; + // Stream thread struct TaskContext { @@ -108,7 +110,7 @@ private: bool streamToViews(); bool checkDependencies(const StorageID & table_id); - bool updateFileStatus(); + bool updateFileStatuses(); }; } diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 8b072dd65df..d6a9d77f796 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -33,6 +33,9 @@ ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/b.txt +# touch does not change file content, no event +touch ${user_files_path}/logs/a.txt + ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" ${CLICKHOUSE_CLIENT} --query "drop table if exists mv;" @@ -40,14 +43,23 @@ ${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/c.txt cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt +# MOVE_FILE_FROM and MOVE_FILE_TO monitored +mv ${user_files_path}/logs/a.txt ${user_files_path}/logs/e.txt +rm ${user_files_path}/logs/e.txt -sleep 5 +for i in 1 2 3 4 5 +do + sleep 1 +done ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" -echo 111, 111 >> ${user_files_path}/logs/a.txt +echo 111, 111 >> ${user_files_path}/logs/c.txt -sleep 5 +for i in 1 2 3 4 5 +do + sleep 1 +done ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" From cd6f8078b5f6ee57f790646cddf339cfad4e88d7 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 22 Sep 2021 02:07:59 +0000 Subject: [PATCH 045/919] refactor some code --- src/Storages/FileLog/StorageFileLog.cpp | 63 +++++++++++++------------ src/Storages/FileLog/StorageFileLog.h | 2 + 2 files changed, 36 insertions(+), 29 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index fa638c2a884..9711b8dbfc4 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -63,35 +63,7 @@ StorageFileLog::StorageFileLog( try { - if (std::filesystem::is_regular_file(path)) - { - auto normal_path = std::filesystem::path(path).lexically_normal().native(); - file_statuses[normal_path] = FileContext{}; - file_names.push_back(normal_path); - } - else if (std::filesystem::is_directory(path)) - { - path_is_directory = true; - /// Just consider file with depth 1 - for (const auto & dir_entry : std::filesystem::directory_iterator{path}) - { - if (dir_entry.is_regular_file()) - { - auto normal_path = std::filesystem::path(dir_entry.path()).lexically_normal().native(); - file_statuses[normal_path] = FileContext{}; - file_names.push_back(normal_path); - } - } - } - else - { - throw Exception("The path neither a regular file, nor a directory", ErrorCodes::BAD_ARGUMENTS); - } - - directory_watch = std::make_unique(path); - - auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); - task = std::make_shared(std::move(thread)); + init(); } catch (...) { @@ -99,6 +71,39 @@ StorageFileLog::StorageFileLog( } } +void StorageFileLog::init() +{ + if (std::filesystem::is_regular_file(path)) + { + auto normal_path = std::filesystem::path(path).lexically_normal().native(); + file_statuses[normal_path] = FileContext{}; + file_names.push_back(normal_path); + } + else if (std::filesystem::is_directory(path)) + { + path_is_directory = true; + /// Just consider file with depth 1 + for (const auto & dir_entry : std::filesystem::directory_iterator{path}) + { + if (dir_entry.is_regular_file()) + { + auto normal_path = std::filesystem::path(dir_entry.path()).lexically_normal().native(); + file_statuses[normal_path] = FileContext{}; + file_names.push_back(normal_path); + } + } + } + else + { + throw Exception("The path neither a regular file, nor a directory", ErrorCodes::BAD_ARGUMENTS); + } + + directory_watch = std::make_unique(path); + + auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); + task = std::make_shared(std::move(thread)); +} + Pipe StorageFileLog::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index ff94b0a548c..5825f3b5094 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -101,6 +101,8 @@ private: using TaskThread = BackgroundSchedulePool::TaskHolder; + void init(); + void threadFunc(); size_t getPollMaxBatchSize() const; From f637f4f2d167c6d7a120c01b7c98043b1b078124 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 22 Sep 2021 02:52:34 +0000 Subject: [PATCH 046/919] remove unused variable update test update --- src/Storages/FileLog/StorageFileLog.cpp | 1 - src/Storages/FileLog/StorageFileLog.h | 1 - tests/queries/0_stateless/02023_storage_filelog.sh | 12 ++++++++---- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 9711b8dbfc4..0404442c1ee 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -81,7 +81,6 @@ void StorageFileLog::init() } else if (std::filesystem::is_directory(path)) { - path_is_directory = true; /// Just consider file with depth 1 for (const auto & dir_entry : std::filesystem::directory_iterator{path}) { diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 5825f3b5094..795180ba9eb 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -66,7 +66,6 @@ protected: private: std::unique_ptr filelog_settings; const String path; - bool path_is_directory = false; const String format_name; Poco::Logger * log; diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index d6a9d77f796..908d77f5c26 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -47,18 +47,22 @@ cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt mv ${user_files_path}/logs/a.txt ${user_files_path}/logs/e.txt rm ${user_files_path}/logs/e.txt -for i in 1 2 3 4 5 +# the max value of reschedule time is 32s, so 40s should +# always be enough to finish streamToViews +for _ in {1..400} do - sleep 1 + sleep 0.1 done ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" echo 111, 111 >> ${user_files_path}/logs/c.txt -for i in 1 2 3 4 5 +# the max value of reschedule time is 32s, so 40s should +# always be enough to finish streamToViews +for _ in {1..400} do - sleep 1 + sleep 0.1 done ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" From e3ae32f7635c82a441ef5af55cd787c2302765f0 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Thu, 23 Sep 2021 19:39:15 +0300 Subject: [PATCH 047/919] Allow nulls first --- src/Storages/MergeTree/KeyCondition.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 425860509be..e23be998115 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -333,8 +333,9 @@ const KeyCondition::AtomMap KeyCondition::atom_map [] (RPNElement & out, const Field &) { out.function = RPNElement::FUNCTION_IS_NULL; - // When using NULL_LAST, isNull means [+Inf, +Inf] - out.range = Range(Field(POSITIVE_INFINITY)); + // isNull means +Inf (NULLS_LAST) or -Inf (NULLS_FIRST), + // which is eqivalent to not in Range (-Inf, +Inf) + out.range = Range(); return true; } } @@ -1988,7 +1989,10 @@ BoolMask KeyCondition::checkInHyperrectangle( /// No need to apply monotonic functions as nulls are kept. bool intersects = element.range.intersectsRange(*key_range); bool contains = element.range.containsRange(*key_range); + rpn_stack.emplace_back(intersects, !contains); + if (element.function == RPNElement::FUNCTION_IS_NULL) + rpn_stack.back() = !rpn_stack.back(); } else if ( element.function == RPNElement::FUNCTION_IN_SET From 9456c674864172c1125341a0421c92bd3c234ccb Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 22 Sep 2021 07:47:36 +0000 Subject: [PATCH 048/919] fix fix test fix fix fix exception handle --- .../FileLog/ReadBufferFromFileLog.cpp | 65 +++++++++++++++---- src/Storages/FileLog/StorageFileLog.cpp | 2 +- src/Storages/FileLog/StorageFileLog.h | 3 +- .../0_stateless/02023_storage_filelog.sh | 7 +- 4 files changed, 60 insertions(+), 17 deletions(-) diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 83ad412f202..6d3a637a6bb 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -10,6 +10,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_READ_FROM_ISTREAM; +} ReadBufferFromFileLog::ReadBufferFromFileLog( StorageFileLog & storage_, @@ -97,7 +101,7 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ size_t read_records_size = 0; const auto & file_names = storage.getFileNames(); - auto & file_status = storage.getFileStatuses(); + auto & file_statuses = storage.getFileStatuses(); size_t files_per_stream = file_names.size() / max_streams_number; size_t start = stream_number * files_per_stream; @@ -105,40 +109,75 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ for (size_t i = start; i < end; ++i) { - auto & file = file_status[file_names[i]]; + auto & file = file_statuses[file_names[i]]; if (file.status == StorageFileLog::FileStatus::NO_CHANGE) continue; auto reader = std::ifstream(file_names[i]); + /// check if ifstream is good. For example, if the file deleted during streamToViews, + /// this will return false because file does not exist anymore. + if (!reader.good()) + { + throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); + } + reader.seekg(0, reader.end); - assert(reader.good()); + /// Exception may happen in seekg and tellg, then badbit will be set + if (!reader.good()) + { + throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); + } auto stream_end = reader.tellg(); - assert(reader.good()); + if (!reader.good()) + { + throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); + } + + /// file may broken(for example truncate), mark this file to BROKEN, + /// should be removed in next updateFileStatuses call + if (file.last_read_position > static_cast(stream_end)) + { + throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); + } reader.seekg(file.last_read_position); - assert(reader.good()); + if (!reader.good()) + { + throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); + } Record record; while (read_records_size < need_records_size && reader.tellg() < stream_end) { + if (!reader.good()) + { + throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); + } std::getline(reader, record); new_records.emplace_back(record); ++read_records_size; } - file.last_read_position = reader.tellg(); - assert(reader.good()); + auto current_position = reader.tellg(); + if (!reader.good()) + { + throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); + } - if (reader.tellg() == stream_end) + file.last_read_position = current_position; + + /// stream reach to end + if (current_position == stream_end && file.status != StorageFileLog::FileStatus::BROKEN) { file.status = StorageFileLog::FileStatus::NO_CHANGE; - /// All ifstream reach end - if (i == end - 1) - { - stream_out = true; - } + } + + /// All ifstream reach end or broken + if (i == end - 1 && (file.status == StorageFileLog::FileStatus::NO_CHANGE || file.status == StorageFileLog::FileStatus::BEGIN)) + { + stream_out = true; } if (read_records_size == need_records_size) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 0404442c1ee..09a90fd7155 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -432,7 +432,7 @@ bool StorageFileLog::updateFileStatuses() std::vector valid_files; for (const auto & file_name : file_names) { - if (file_statuses.at(file_name).status == FileStatus::REMOVED) + if (file_statuses.at(file_name).status == FileStatus::REMOVED || file_statuses.at(file_name).status == FileStatus::BROKEN) { file_statuses.erase(file_name); } diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 795180ba9eb..d149fb9c548 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -28,7 +28,8 @@ public: BEGIN, NO_CHANGE, UPDATED, - REMOVED + REMOVED, + BROKEN }; using Files = std::vector; diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 908d77f5c26..97a147479ac 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -1,4 +1,6 @@ #!/usr/bin/env bash +# Tags: long, no-parallel + set -eu CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) @@ -49,7 +51,7 @@ rm ${user_files_path}/logs/e.txt # the max value of reschedule time is 32s, so 40s should # always be enough to finish streamToViews -for _ in {1..400} +for _ in {1..150} do sleep 0.1 done @@ -57,10 +59,11 @@ done ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" echo 111, 111 >> ${user_files_path}/logs/c.txt +truncate ${user_files_path}/logs/d.txt --size 0 # the max value of reschedule time is 32s, so 40s should # always be enough to finish streamToViews -for _ in {1..400} +for _ in {1..150} do sleep 0.1 done From c1ea44b9c2df004fd43b6b1434fa713f6595ab0e Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Sep 2021 13:38:08 +0300 Subject: [PATCH 049/919] Fixes --- ...chronousReadIndirectBufferFromRemoteFS.cpp | 29 +++++++++++-------- src/IO/ReadBufferFromRemoteFS.cpp | 7 ++--- src/IO/ReadBufferFromS3.cpp | 3 +- 3 files changed, 22 insertions(+), 17 deletions(-) diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp index ada29155b3f..a42e15325d8 100644 --- a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -36,6 +36,13 @@ std::future AsynchronousReadIndirectBufferFromRemot { IAsynchronousReader::Request request; + impl->set(impl->buffer().begin(), impl->buffer().size()); + if (impl->initialized()) + { + impl->position() = impl->buffer().end(); + assert(!impl->hasPendingData()); + } + auto remote_fd = std::make_shared(); remote_fd->impl = impl; @@ -51,11 +58,6 @@ void AsynchronousReadIndirectBufferFromRemoteFS::prefetch() if (prefetch_future.valid()) return; - if (impl->initialized()) - { - impl->position() = impl->buffer().end(); /// May be should try to do this differently. - assert(!impl->hasPendingData()); - } prefetch_future = readNext(); } @@ -74,17 +76,19 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() } else { - if (impl->initialized()) - { - impl->position() = position(); - assert(!impl->hasPendingData()); - } + // if (impl->initialized()) + // { + // impl->position() = position(); + // assert(!impl->hasPendingData()); + // } size = readNext().get(); } if (size) { - BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); + //set(working_buffer.begin(), working_buffer.size()); + swap(*impl); + // BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); impl->absolute_position += working_buffer.size(); } @@ -138,8 +142,9 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence prefetch_future = {}; } - impl->seek(impl->absolute_position, SEEK_SET); + // impl->seek(impl->absolute_position, SEEK_SET); pos = working_buffer.end(); + impl->reset(); return impl->absolute_position; } diff --git a/src/IO/ReadBufferFromRemoteFS.cpp b/src/IO/ReadBufferFromRemoteFS.cpp index 88b3fffcf8b..b52104790af 100644 --- a/src/IO/ReadBufferFromRemoteFS.cpp +++ b/src/IO/ReadBufferFromRemoteFS.cpp @@ -85,17 +85,16 @@ off_t ReadBufferFromRemoteFS::seek([[maybe_unused]] off_t offset_, int whence) { if (whence != SEEK_SET) throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only SEEK_SET is allowed"); - /// We already made a seek and adjusted position in ReadIndirectBufferFromRemoteFS. - assert(offset_ == static_cast(absolute_position)); - current_buf = initialize(); + // current_buf = initialize(); return absolute_position; } void ReadBufferFromRemoteFS::reset() { - set(nullptr, 0); + current_buf.reset(); + // set(nullptr, 0); } } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 9485bec5679..737090c4bce 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -48,7 +48,8 @@ bool ReadBufferFromS3::nextImpl() if (impl) { /// `impl` has been initialized earlier and now we're at the end of the current portion of data. - impl->position() = position(); + // impl->position() = position(); + impl->set(working_buffer.begin(), working_buffer.size()); assert(!impl->hasPendingData()); } else From aa3f4003c82b80331468e789e31dfa7cd7ad0420 Mon Sep 17 00:00:00 2001 From: Kevin Michel Date: Fri, 24 Sep 2021 12:51:52 +0200 Subject: [PATCH 050/919] Fix hanging DDL queries on Replicated database There was a race condition when issuing a DDL query on a replica just after a new replica was added. If the DDL query is issued after the new replica adds itself to the list of replicas, but before the new replica has finished its recovery, then the first replica adds the new replica to the list of replicas to wait to confirm the query was replicated. Meanwhile, the new replica is still in recovery and applies queries from the /metadata snapshot. When it's done, it bumps its log_ptr without marking the corresponding log entries (if any) as finished. The first replica then waits until distributed_ddl_task_timeout expires and wrongly assumes the query was not replicated. The issue is fixed by remembering the max_log_ptr at the exact point where the replica adds itself to the list of replicas, then mark as finished all queries that happened between that max_log_ptr and the max_log_ptr of the metadata snapshot used in recovery. The bug was randomly observed during a downstream test. It can be reproduced more easily by inserting a sleep of a few seconds at the end of createReplicaNodesInZooKeeper, enough to have time to issue a DDL query on the first replica. --- src/Databases/DatabaseReplicated.cpp | 43 +++++++++++++++++++++++++--- src/Databases/DatabaseReplicated.h | 1 + 2 files changed, 40 insertions(+), 4 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index da03eb6aba6..42389941c98 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -298,10 +298,30 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt /// Write host name to replica_path, it will protect from multiple replicas with the same name auto host_id = getHostID(getContext(), db_uuid); - Coordination::Requests ops; - ops.emplace_back(zkutil::makeCreateRequest(replica_path, host_id, zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/log_ptr", "0", zkutil::CreateMode::Persistent)); - current_zookeeper->multi(ops); + for (int attempts = 10; attempts > 0; --attempts) + { + Coordination::Stat stat; + String max_log_ptr_str = current_zookeeper->get(zookeeper_path + "/max_log_ptr", &stat); + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCreateRequest(replica_path, host_id, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/log_ptr", "0", zkutil::CreateMode::Persistent)); + /// In addition to creating the replica nodes, we record the max_log_ptr at the instant where + /// we declared ourself as an existing replica. We'll need this during recoverLostReplica to + /// notify other nodes that issued new queries while this node was recovering. + ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/max_log_ptr", stat.version)); + Coordination::Responses responses; + const auto code = current_zookeeper->tryMulti(ops, responses); + if (code == Coordination::Error::ZOK) + { + max_log_ptr_at_creation = parse(max_log_ptr_str); + break; + } + else if (code == Coordination::Error::ZNODEEXISTS || attempts == 1) + { + /// If its our last attempt, or if the replica already exists, fail immediately. + zkutil::KeeperMultiException::check(code, ops, responses); + } + } createEmptyLogEntry(current_zookeeper); } @@ -613,6 +633,21 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep InterpreterCreateQuery(query_ast, create_query_context).execute(); } + if (max_log_ptr_at_creation != 0) + { + /// If the replica is new and some of the queries applied during recovery + /// where issued after the replica was created, then other nodes might be + /// waiting for this node to notify them that the query was applied. + for (UInt32 ptr = max_log_ptr_at_creation; ptr <= max_log_ptr; ++ptr) + { + auto entry_name = DDLTaskBase::getLogEntryName(ptr); + auto path = fs::path(zookeeper_path) / "log" / entry_name / "finished" / getFullReplicaName(); + auto status = ExecutionStatus(0).serializeText(); + auto res = current_zookeeper->tryCreate(path, status, zkutil::CreateMode::Persistent); + if (res == Coordination::Error::ZOK) + LOG_INFO(log, "Marked recovered {} as finished", entry_name); + } + } current_zookeeper->set(replica_path + "/log_ptr", toString(max_log_ptr)); } diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 1e0daeed07e..849c33dfbcc 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -90,6 +90,7 @@ private: std::atomic_bool is_readonly = true; std::unique_ptr ddl_worker; + UInt32 max_log_ptr_at_creation = 0; mutable ClusterPtr cluster; }; From 6219d541a5d2b8ba00e2ac02ee614b73561e51f8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 24 Sep 2021 15:12:11 +0300 Subject: [PATCH 051/919] Finally fixed --- ...chronousReadIndirectBufferFromRemoteFS.cpp | 21 ++++++------------- src/Disks/S3/DiskS3.cpp | 13 ++++++++---- src/IO/ReadBufferFromS3.cpp | 18 ++++++++++++---- src/IO/ReadBufferFromS3.h | 4 +++- 4 files changed, 32 insertions(+), 24 deletions(-) diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp index a42e15325d8..c754bfc004c 100644 --- a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -36,14 +36,10 @@ std::future AsynchronousReadIndirectBufferFromRemot { IAsynchronousReader::Request request; - impl->set(impl->buffer().begin(), impl->buffer().size()); - if (impl->initialized()) - { - impl->position() = impl->buffer().end(); - assert(!impl->hasPendingData()); - } - auto remote_fd = std::make_shared(); + + /// Resize buffer to 0 and move pos to start. + impl->set(impl->buffer().begin(), impl->buffer().size()); remote_fd->impl = impl; request.descriptor = std::move(remote_fd); @@ -76,20 +72,16 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() } else { - // if (impl->initialized()) - // { - // impl->position() = position(); - // assert(!impl->hasPendingData()); - // } size = readNext().get(); } if (size) { - //set(working_buffer.begin(), working_buffer.size()); + size_t offset = pos - working_buffer.begin(); + assert(offset >= 0); swap(*impl); - // BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); impl->absolute_position += working_buffer.size(); + position() = working_buffer.begin() + offset; } prefetch_future = {}; @@ -142,7 +134,6 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence prefetch_future = {}; } - // impl->seek(impl->absolute_position, SEEK_SET); pos = working_buffer.end(); impl->reset(); diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 4128381b071..41fb6ceb3bb 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -137,18 +137,20 @@ public: const String & bucket_, DiskS3::Metadata metadata_, size_t max_single_read_retries_, - size_t buf_size_) + size_t buf_size_, + bool threadpool_read_ = false) : ReadBufferFromRemoteFS(metadata_) , client_ptr(std::move(client_ptr_)) , bucket(bucket_) , max_single_read_retries(max_single_read_retries_) , buf_size(buf_size_) + , threadpool_read(threadpool_read_) { } SeekableReadBufferPtr createReadBuffer(const String & path) const override { - return std::make_unique(client_ptr, bucket, fs::path(metadata.remote_fs_root_path) / path, max_single_read_retries, buf_size); + return std::make_unique(client_ptr, bucket, fs::path(metadata.remote_fs_root_path) / path, max_single_read_retries, buf_size, threadpool_read); } private: @@ -156,6 +158,7 @@ private: const String & bucket; UInt64 max_single_read_retries; size_t buf_size; + bool threadpool_read; }; DiskS3::DiskS3( @@ -231,11 +234,13 @@ std::unique_ptr DiskS3::readFile(const String & path, co LOG_TRACE(log, "Read from file by path: {}. Existing S3 objects: {}", backQuote(metadata_path + path), metadata.remote_fs_objects.size()); + bool threadpool_read = read_settings.remote_fs_method == RemoteFSReadMethod::read_threadpool; + auto s3_impl = std::make_unique( settings->client, bucket, metadata, - settings->s3_max_single_read_retries, read_settings.remote_fs_buffer_size); + settings->s3_max_single_read_retries, read_settings.remote_fs_buffer_size, threadpool_read); - if (read_settings.remote_fs_method == RemoteFSReadMethod::read_threadpool) + if (threadpool_read) { auto reader = getThreadPoolReader(); auto buf = std::make_unique(reader, read_settings.priority, std::move(s3_impl)); diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 737090c4bce..cc00635b5bd 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -31,13 +31,15 @@ namespace ErrorCodes ReadBufferFromS3::ReadBufferFromS3( - std::shared_ptr client_ptr_, const String & bucket_, const String & key_, UInt64 max_single_read_retries_, size_t buffer_size_) + std::shared_ptr client_ptr_, const String & bucket_, const String & key_, + UInt64 max_single_read_retries_, size_t buffer_size_, bool use_external_buffer_) : SeekableReadBuffer(nullptr, 0) , client_ptr(std::move(client_ptr_)) , bucket(bucket_) , key(key_) , max_single_read_retries(max_single_read_retries_) , buffer_size(buffer_size_) + , use_external_buffer(use_external_buffer_) { } @@ -45,11 +47,18 @@ bool ReadBufferFromS3::nextImpl() { bool next_result = false; + /// `impl` has been initialized earlier and now we're at the end of the current portion of data. if (impl) { - /// `impl` has been initialized earlier and now we're at the end of the current portion of data. - // impl->position() = position(); - impl->set(working_buffer.begin(), working_buffer.size()); + if (use_external_buffer) + { + impl->set(working_buffer.begin(), working_buffer.size()); + } + else + { + impl->position() = position(); + } + assert(!impl->hasPendingData()); } else @@ -94,6 +103,7 @@ bool ReadBufferFromS3::nextImpl() } } + std::cerr << "s3 buffer size: " << impl->buffer().size() << std::endl; if (!next_result) return false; diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index e24d571b557..3fa515da707 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -43,7 +43,8 @@ public: const String & bucket_, const String & key_, UInt64 max_single_read_retries_, - size_t buffer_size_); + size_t buffer_size_, + bool use_external_buffer = false); bool nextImpl() override; @@ -52,6 +53,7 @@ public: private: std::unique_ptr initialize(); + bool use_external_buffer; }; } From 0009cab96299eef8520424c2ac0b7e17229b972d Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 24 Sep 2021 16:44:22 +0000 Subject: [PATCH 052/919] add virtual columns --- src/Storages/FileLog/FileLogSource.cpp | 89 ++++--------------- src/Storages/FileLog/FileLogSource.h | 4 +- .../FileLog/ReadBufferFromFileLog.cpp | 9 +- src/Storages/FileLog/ReadBufferFromFileLog.h | 11 ++- src/Storages/FileLog/StorageFileLog.cpp | 66 ++++++++++---- src/Storages/FileLog/StorageFileLog.h | 4 + 6 files changed, 91 insertions(+), 92 deletions(-) diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index c1feb721af0..c272df1c2b2 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -21,20 +22,19 @@ FileLogSource::FileLogSource( StorageFileLog & storage_, const StorageMetadataPtr & metadata_snapshot_, const ContextPtr & context_, - const Names & columns, size_t max_block_size_, size_t poll_time_out_, size_t stream_number_, size_t max_streams_number_) - : SourceWithProgress(metadata_snapshot_->getSampleBlockForColumns(columns, storage_.getVirtuals(), storage_.getStorageID())) + : SourceWithProgress(metadata_snapshot_->getSampleBlockWithVirtuals(storage_.getVirtuals())) , storage(storage_) , metadata_snapshot(metadata_snapshot_) , context(context_) - , column_names(columns) , max_block_size(max_block_size_) , poll_time_out(poll_time_out_) , non_virtual_header(metadata_snapshot_->getSampleBlockNonMaterialized()) - , column_names_and_types(metadata_snapshot_->getColumns().getByNames(ColumnsDescription::All, columns, true)) + , virtual_header( + metadata_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames(), storage.getVirtuals(), storage.getStorageID())) { buffer = std::make_unique(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_); } @@ -44,53 +44,12 @@ Chunk FileLogSource::generate() if (!buffer || buffer->noRecords()) return {}; - MutableColumns read_columns = non_virtual_header.cloneEmptyColumns(); + 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); - InputPort port(input_format->getPort().getHeader(), input_format.get()); - connect(input_format->getPort(), port); - port.setNeeded(); - - std::optional exception_message; - auto read_file_log = [&] { - size_t new_rows = 0; - while (true) - { - auto status = input_format->prepare(); - - switch (status) - { - case IProcessor::Status::Ready: - input_format->work(); - break; - - case IProcessor::Status::Finished: - input_format->resetParser(); - return new_rows; - - case IProcessor::Status::PortFull: - { - auto chunk = port.pull(); - - auto chunk_rows = chunk.getNumRows(); - new_rows += chunk_rows; - - auto columns = chunk.detachColumns(); - for (size_t i = 0, s = columns.size(); i < s; ++i) - { - read_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); - } - break; - } - case IProcessor::Status::NeedData: - case IProcessor::Status::Async: - case IProcessor::Status::ExpandPipeline: - throw Exception("Source processor returned status " + IProcessor::statusToName(status), ErrorCodes::LOGICAL_ERROR); - } - } - }; + StreamingFormatExecutor executor(non_virtual_header, input_format); size_t total_rows = 0; size_t failed_poll_attempts = 0; @@ -100,25 +59,17 @@ Chunk FileLogSource::generate() { size_t new_rows = 0; if (buffer->poll()) - { - try - { - new_rows = read_file_log(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - else - { - /// No records polled, should break out early, since - /// file status can not be updated during streamToViews - break; - } + new_rows = executor.execute(); if (new_rows) { + auto file_name = buffer->getFileName(); + auto offset = buffer->getOffset(); + for (size_t i = 0; i < new_rows; ++i) + { + virtual_columns[0]->insert(file_name); + virtual_columns[1]->insert(offset); + } total_rows = total_rows + new_rows; } else /// poll succeed, but parse failed @@ -137,13 +88,11 @@ Chunk FileLogSource::generate() if (total_rows == 0) return {}; - Columns result_columns; - result_columns.reserve(column_names_and_types.size()); + auto result_columns = executor.getResultColumns(); - for (const auto & elem : column_names_and_types) + for (auto & column : virtual_columns) { - auto index = non_virtual_header.getPositionByName(elem.getNameInStorage()); - result_columns.emplace_back(std::move(read_columns[index])); + result_columns.emplace_back(std::move(column)); } return Chunk(std::move(result_columns), total_rows); diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index 1dc55f2c0c2..7f7d75bfaad 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -19,7 +19,6 @@ public: StorageFileLog & storage_, const StorageMetadataPtr & metadata_snapshot_, const ContextPtr & context_, - const Names & columns, size_t max_block_size_, size_t poll_time_out_, size_t stream_number_, @@ -36,7 +35,6 @@ private: StorageFileLog & storage; StorageMetadataPtr metadata_snapshot; ContextPtr context; - Names column_names; UInt64 max_block_size; size_t poll_time_out; @@ -44,7 +42,7 @@ private: std::unique_ptr buffer; Block non_virtual_header; - const NamesAndTypesList column_names_and_types; + Block virtual_header; }; } diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 6d3a637a6bb..ad23edac2d4 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -155,7 +155,10 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ { throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); } - std::getline(reader, record); + UInt64 start_offset = reader.tellg(); + std::getline(reader, record.data); + record.file_name = file_names[i]; + record.offset = start_offset; new_records.emplace_back(record); ++read_records_size; } @@ -192,8 +195,8 @@ bool ReadBufferFromFileLog::nextImpl() if (!allowed || !hasMorePolledRecords()) return false; - auto * new_position = const_cast(current->data()); - BufferBase::set(new_position, current->size(), 0); + auto * new_position = const_cast(current->data.data()); + BufferBase::set(new_position, current->data.size(), 0); allowed = false; ++current; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index 66a8aeaa49f..747a5f316fe 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -37,6 +37,9 @@ public: bool noRecords() { return buffer_status == BufferStatus::NO_RECORD_RETURNED; } + auto getFileName() const { return current != records.begin() ? (current - 1)->file_name : ""; } + auto getOffset() const { return current != records.begin() ? (current - 1)->offset : 0; } + private: enum class BufferStatus { @@ -63,7 +66,13 @@ private: bool allowed = true; - using Record = std::string; + using RecordData = std::string; + struct Record + { + RecordData data; + std::string file_name; + UInt64 offset; + }; using Records = std::vector; Records records; diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 09a90fd7155..04b76404f56 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -1,3 +1,5 @@ +#include +#include #include #include #include @@ -8,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -15,15 +18,10 @@ #include #include #include -#include #include #include -#include #include #include -#include - -#include namespace DB { @@ -39,7 +37,7 @@ namespace { const auto RESCHEDULE_MS = 500; const auto BACKOFF_TRESHOLD = 32000; - const auto MAX_THREAD_WORK_DURATION_MS = 60000; // once per minute leave do reschedule (we can't lock threads in pool forever) + const auto MAX_THREAD_WORK_DURATION_MS = 60000; } StorageFileLog::StorageFileLog( @@ -134,28 +132,45 @@ Pipe StorageFileLog::read( *this, metadata_snapshot, modified_context, - column_names, getMaxBlockSize(), getPollTimeoutMillisecond(), stream_number, max_streams_number)); } - return Pipe::unitePipes(std::move(pipes)); + auto pipe = Pipe::unitePipes(std::move(pipes)); + + auto convert_actions_dag = ActionsDAG::makeConvertingActions( + pipe.getHeader().getColumnsWithTypeAndName(), + metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()).getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto actions = std::make_shared( + convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); + + pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); }); + + return pipe; } void StorageFileLog::startup() { - task->holder->activateAndSchedule(); + if (task) + { + task->holder->activateAndSchedule(); + } } void StorageFileLog::shutdown() { - task->stream_cancelled = true; + if (task) + { + task->stream_cancelled = true; - LOG_TRACE(log, "Waiting for cleanup"); - task->holder->deactivate(); + LOG_TRACE(log, "Waiting for cleanup"); + task->holder->deactivate(); + } } size_t StorageFileLog::getMaxBlockSize() const @@ -286,22 +301,31 @@ bool StorageFileLog::streamToViews() Pipes pipes; pipes.reserve(max_streams_number); - auto names = block_io.out->getHeader().getNames(); for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) { pipes.emplace_back(std::make_shared( *this, metadata_snapshot, new_context, - names, getPollMaxBatchSize(), getPollTimeoutMillisecond(), stream_number, max_streams_number)); } + auto pipe = Pipe::unitePipes(std::move(pipes)); + + auto convert_actions_dag = ActionsDAG::makeConvertingActions( + pipe.getHeader().getColumnsWithTypeAndName(), + block_io.out->getHeader().getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto actions = std::make_shared( + convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); + + pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); }); QueryPipeline pipeline; - pipeline.init(Pipe::unitePipes(std::move(pipes))); + pipeline.init(std::move(pipe)); assertBlocksHaveEqualStructure(pipeline.getHeader(), block_io.out->getHeader(), "StorageFileLog streamToViews"); @@ -384,6 +408,8 @@ void registerStorageFileLog(StorageFactory & factory) bool StorageFileLog::updateFileStatuses() { + if (!directory_watch) + return false; /// Do not need to hold file_status lock, since it will be holded /// by caller when call this function auto error = directory_watch->getErrorAndReset(); @@ -446,4 +472,14 @@ bool StorageFileLog::updateFileStatuses() return events.empty() || file_names.empty(); } + +NamesAndTypesList StorageFileLog::getVirtuals() const +{ + return NamesAndTypesList{{"_file_name", std::make_shared()}, {"_offset", std::make_shared()}}; +} + +Names StorageFileLog::getVirtualColumnNames() +{ + return {"_file_name", "_offset"}; +} } diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index d149fb9c548..dc9485fccb2 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -55,6 +55,10 @@ public: auto & getFileNames() { return file_names; } auto & getFileStatuses() { return file_statuses; } + NamesAndTypesList getVirtuals() const override; + + static Names getVirtualColumnNames(); + protected: StorageFileLog( const StorageID & table_id_, From 7af96b12b16ddc8f42b479db379a99a17075a241 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 26 Sep 2021 07:22:45 +0000 Subject: [PATCH 053/919] add meta file fix and update tests --- src/Common/ErrorCodes.cpp | 6 + .../FileLog/ReadBufferFromFileLog.cpp | 80 +--- src/Storages/FileLog/ReadBufferFromFileLog.h | 7 +- src/Storages/FileLog/StorageFileLog.cpp | 445 ++++++++++++++---- src/Storages/FileLog/StorageFileLog.h | 92 +++- .../02022_storage_filelog_one_file.reference | 41 ++ .../02022_storage_filelog_one_file.sh | 42 ++ .../02023_storage_filelog.reference | 123 ++++- .../0_stateless/02023_storage_filelog.sh | 62 +-- .../02024_storage_filelog_mv.reference | 121 +++++ .../0_stateless/02024_storage_filelog_mv.sh | 58 +++ ...2025_storage_filelog_virtual_col.reference | 144 ++++++ .../02025_storage_filelog_virtual_col.sh | 62 +++ 13 files changed, 1056 insertions(+), 227 deletions(-) create mode 100644 tests/queries/0_stateless/02022_storage_filelog_one_file.reference create mode 100755 tests/queries/0_stateless/02022_storage_filelog_one_file.sh create mode 100644 tests/queries/0_stateless/02024_storage_filelog_mv.reference create mode 100755 tests/queries/0_stateless/02024_storage_filelog_mv.sh create mode 100644 tests/queries/0_stateless/02025_storage_filelog_virtual_col.reference create mode 100755 tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 53276f5b196..9312fde8baa 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -588,6 +588,12 @@ M(618, LZ4_DECODER_FAILED) \ M(619, POSTGRESQL_REPLICATION_INTERNAL_ERROR) \ M(620, QUERY_NOT_ALLOWED) \ + M(621, PATH_DOESNT_EXIST) \ + M(622, FILE_META_NOT_FOUND) \ + M(623, CANNOT_GET_FILE_STAT) \ + M(624, NOT_REGULAR_FILE) \ + M(625, READ_META_FILE_FAILED) \ + M(626, FILE_STREAM_ERROR) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index ad23edac2d4..bfcfb3a2385 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -69,8 +69,6 @@ bool ReadBufferFromFileLog::poll() allowed = true; return true; } - - } ReadBufferFromFileLog::Records ReadBufferFromFileLog::pollBatch(size_t batch_size_) @@ -100,85 +98,52 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ size_t need_records_size = batch_size_ - new_records.size(); size_t read_records_size = 0; - const auto & file_names = storage.getFileNames(); - auto & file_statuses = storage.getFileStatuses(); + auto & file_infos = storage.getFileInfos(); - size_t files_per_stream = file_names.size() / max_streams_number; + size_t files_per_stream = file_infos.file_names.size() / max_streams_number; size_t start = stream_number * files_per_stream; - size_t end = stream_number == max_streams_number - 1 ? file_names.size() : (stream_number + 1) * files_per_stream; + size_t end = stream_number == max_streams_number - 1 ? file_infos.file_names.size() : (stream_number + 1) * files_per_stream; for (size_t i = start; i < end; ++i) { - auto & file = file_statuses[file_names[i]]; - if (file.status == StorageFileLog::FileStatus::NO_CHANGE) + auto file_name = file_infos.file_names[i]; + auto & file_ctx = file_infos.context_by_name.at(file_name); + if (file_ctx.status == StorageFileLog::FileStatus::NO_CHANGE) continue; - auto reader = std::ifstream(file_names[i]); - - /// check if ifstream is good. For example, if the file deleted during streamToViews, - /// this will return false because file does not exist anymore. - if (!reader.good()) - { - throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); - } - - reader.seekg(0, reader.end); - /// Exception may happen in seekg and tellg, then badbit will be set - if (!reader.good()) - { - throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); - } - - auto stream_end = reader.tellg(); - if (!reader.good()) - { - throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); - } - - /// file may broken(for example truncate), mark this file to BROKEN, - /// should be removed in next updateFileStatuses call - if (file.last_read_position > static_cast(stream_end)) - { - throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); - } - - reader.seekg(file.last_read_position); - if (!reader.good()) - { - throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); - } + auto & file_meta = file_infos.meta_by_inode.at(file_infos.inode_by_name.at(file_name)); Record record; - while (read_records_size < need_records_size && reader.tellg() < stream_end) + while (read_records_size < need_records_size && static_cast(file_ctx.reader.tellg()) < file_meta.last_open_end) { - if (!reader.good()) + if (!file_ctx.reader.good()) { - throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); + throw Exception("Can not read from file " + file_name + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); } - UInt64 start_offset = reader.tellg(); - std::getline(reader, record.data); - record.file_name = file_names[i]; + UInt64 start_offset = file_ctx.reader.tellg(); + std::getline(file_ctx.reader, record.data); + record.file_name = file_name; record.offset = start_offset; new_records.emplace_back(record); ++read_records_size; } - auto current_position = reader.tellg(); - if (!reader.good()) + UInt64 current_position = file_ctx.reader.tellg(); + if (!file_ctx.reader.good()) { - throw Exception("Can not read from file " + file_names[i] + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); + throw Exception("Can not read from file " + file_name + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); } - file.last_read_position = current_position; + file_meta.last_writen_position = current_position; /// stream reach to end - if (current_position == stream_end && file.status != StorageFileLog::FileStatus::BROKEN) + if (current_position == file_meta.last_open_end) { - file.status = StorageFileLog::FileStatus::NO_CHANGE; + file_ctx.status = StorageFileLog::FileStatus::NO_CHANGE; } - /// All ifstream reach end or broken - if (i == end - 1 && (file.status == StorageFileLog::FileStatus::NO_CHANGE || file.status == StorageFileLog::FileStatus::BEGIN)) + /// All ifstream reach end + if (i == end - 1 && (file_ctx.status == StorageFileLog::FileStatus::NO_CHANGE)) { stream_out = true; } @@ -199,6 +164,9 @@ bool ReadBufferFromFileLog::nextImpl() BufferBase::set(new_position, current->data.size(), 0); allowed = false; + current_file = current->file_name; + current_offset = current->offset; + ++current; return true; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index 747a5f316fe..47a2770550d 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -37,8 +37,8 @@ public: bool noRecords() { return buffer_status == BufferStatus::NO_RECORD_RETURNED; } - auto getFileName() const { return current != records.begin() ? (current - 1)->file_name : ""; } - auto getOffset() const { return current != records.begin() ? (current - 1)->offset : 0; } + auto getFileName() const { return current_file; } + auto getOffset() const { return current_offset; } private: enum class BufferStatus @@ -78,6 +78,9 @@ private: Records records; Records::const_iterator current; + String current_file; + UInt64 current_offset; + using TaskThread = BackgroundSchedulePool::TaskHolder; TaskThread wait_task; diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 04b76404f56..9b64be4c99b 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -1,5 +1,9 @@ #include #include +#include +#include +#include +#include #include #include #include @@ -9,13 +13,13 @@ #include #include #include -#include #include #include #include #include #include #include +#include #include #include #include @@ -23,6 +27,8 @@ #include #include +#include + namespace DB { @@ -31,6 +37,10 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; + extern const int CANNOT_GET_FILE_STAT; + extern const int NOT_REGULAR_FILE; + extern const int READ_META_FILE_FAILED; + extern const int FILE_STREAM_ERROR; } namespace @@ -46,7 +56,8 @@ StorageFileLog::StorageFileLog( const ColumnsDescription & columns_, const String & relative_path_, const String & format_name_, - std::unique_ptr settings) + std::unique_ptr settings, + bool attach) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) , filelog_settings(std::move(settings)) @@ -61,7 +72,18 @@ StorageFileLog::StorageFileLog( try { - init(); + loadMetaFiles(attach); + loadFiles(); + + assert( + file_infos.file_names.size() == file_infos.meta_by_inode.size() == file_infos.inode_by_name.size() + == file_infos.context_by_name.size()); + + if (path_is_directory) + directory_watch = std::make_unique(path); + + auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); + task = std::make_shared(std::move(thread)); } catch (...) { @@ -69,36 +91,184 @@ StorageFileLog::StorageFileLog( } } -void StorageFileLog::init() +void StorageFileLog::loadMetaFiles(bool attach) { + const auto database = DatabaseCatalog::instance().getDatabase(getStorageID().getDatabaseName()); + const auto table_name = getStorageID().getTableName(); + + root_meta_path = database->getMetadataPath() + "/." + table_name; + + /// Create table, just create meta data directory + if (!attach) + { + if (std::filesystem::exists(root_meta_path)) + { + std::filesystem::remove_all(root_meta_path); + } + std::filesystem::create_directories(root_meta_path); + } + /// Attach table + else + { + /// Meta file may lost, log and create directory + if (!std::filesystem::exists(root_meta_path)) + { + LOG_INFO(log, "Meta files of table {} may have lost.", getStorageID().getTableName()); + std::filesystem::create_directories(root_meta_path); + } + /// Load all meta info to file_infos; + deserialize(); + } +} + +void StorageFileLog::loadFiles() +{ + if (std::filesystem::is_regular_file(path)) { - auto normal_path = std::filesystem::path(path).lexically_normal().native(); - file_statuses[normal_path] = FileContext{}; - file_names.push_back(normal_path); + path_is_directory = false; + root_data_path = getContext()->getUserFilesPath(); + + file_infos.file_names.push_back(std::filesystem::path(path).filename()); } else if (std::filesystem::is_directory(path)) { + root_data_path = path; /// Just consider file with depth 1 for (const auto & dir_entry : std::filesystem::directory_iterator{path}) { if (dir_entry.is_regular_file()) { - auto normal_path = std::filesystem::path(dir_entry.path()).lexically_normal().native(); - file_statuses[normal_path] = FileContext{}; - file_names.push_back(normal_path); + file_infos.file_names.push_back(dir_entry.path().filename()); } } } else { - throw Exception("The path neither a regular file, nor a directory", ErrorCodes::BAD_ARGUMENTS); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The path {} neither a regular file, nor a directory", path); } - directory_watch = std::make_unique(path); + /// Get files inode + for (const auto & file : file_infos.file_names) + { + auto inode = getInode(getFullDataPath(file)); + file_infos.inode_by_name.emplace(file, inode); + file_infos.context_by_name.emplace(file, FileContext{}); + } - auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); - task = std::make_shared(std::move(thread)); + /// Update file meta or create file meta + for (const auto & file_inode : file_infos.inode_by_name) + { + if (auto it = file_infos.meta_by_inode.find(file_inode.second); it != file_infos.meta_by_inode.end()) + { + /// data file have been renamed, need update meta file's name + if (it->second.file_name != file_inode.first) + { + it->second.file_name = file_inode.first; + if (std::filesystem::exists(getFullMetaPath(it->second.file_name))) + { + std::filesystem::rename(getFullMetaPath(it->second.file_name), getFullMetaPath(file_inode.first)); + } + } + } + /// New file + else + { + FileMeta meta{file_inode.first, 0, 0}; + file_infos.meta_by_inode.emplace(file_inode.second, meta); + } + } + + /// Clear unneeded meta file, because data files may be deleted + if (file_infos.meta_by_inode.size() > file_infos.inode_by_name.size()) + { + InodeToFileMeta valid_metas; + valid_metas.reserve(file_infos.inode_by_name.size()); + for (const auto & it : file_infos.meta_by_inode) + { + if (file_infos.inode_by_name.contains(it.second.file_name)) + valid_metas.emplace(it); + } + file_infos.meta_by_inode.swap(valid_metas); + } +} + +void StorageFileLog::serialize(bool with_end_pos) const +{ + for (const auto & it : file_infos.meta_by_inode) + { + auto full_name = getFullMetaPath(it.second.file_name); + if (!std::filesystem::exists(full_name)) + { + Poco::File{full_name}.createFile(); + } + WriteBufferFromFile buf(full_name); + writeIntText(it.first, buf); + writeChar('\n', buf); + writeIntText(it.second.last_writen_position, buf); + + if (with_end_pos) + { + writeChar('\n', buf); + writeIntText(it.second.last_open_end, buf); + } + } +} + +void StorageFileLog::deserialize() +{ + for (const auto & dir_entry : std::filesystem::directory_iterator{root_meta_path}) + { + if (!dir_entry.is_regular_file()) + { + throw Exception( + ErrorCodes::NOT_REGULAR_FILE, + "The file {} under {} is not a regular file when deserializing meta files.", + dir_entry.path().c_str(), + root_meta_path); + } + + ReadBufferFromFile buf(dir_entry.path().c_str()); + FileMeta meta; + UInt64 inode, last_written_pos; + + if (!tryReadIntText(inode, buf)) + { + throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); + } + if (!checkChar('\n', buf)) + { + throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); + } + if (!tryReadIntText(last_written_pos, buf)) + { + throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); + } + + meta.file_name = dir_entry.path().filename(); + meta.last_writen_position = last_written_pos; + + /// May have last open end in meta file + if (checkChar('\n', buf)) + { + if (!tryReadIntText(meta.last_open_end, buf)) + { + throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); + } + } + + file_infos.meta_by_inode.emplace(inode, meta); + } +} + +UInt64 StorageFileLog::getInode(const String & file_name) +{ + struct stat file_stat; + if (stat(file_name.c_str(), &file_stat)) + { + throw Exception(ErrorCodes::CANNOT_GET_FILE_STAT, "Can not get stat info of file {}", file_name); + } + return file_stat.st_ino; } Pipe StorageFileLog::read( @@ -112,45 +282,47 @@ Pipe StorageFileLog::read( { std::lock_guard lock(status_mutex); - updateFileStatuses(); + updateFileInfos(); /// No files to parse - if (file_names.empty()) + if (file_infos.file_names.empty()) { return Pipe{}; } auto modified_context = Context::createCopy(local_context); - auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_names.size()); + auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_infos.file_names.size()); + + openFilesAndSetPos(); + serialize(true); Pipes pipes; pipes.reserve(max_streams_number); for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) { - pipes.emplace_back(std::make_shared( - *this, - metadata_snapshot, - modified_context, - getMaxBlockSize(), - getPollTimeoutMillisecond(), - stream_number, - max_streams_number)); + Pipe pipe(std::make_shared( + *this, metadata_snapshot, modified_context, getMaxBlockSize(), getPollTimeoutMillisecond(), stream_number, max_streams_number)); + + auto convert_actions_dag = ActionsDAG::makeConvertingActions( + pipe.getHeader().getColumnsWithTypeAndName(), + metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()).getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto actions = std::make_shared( + convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); + + pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); }); + pipes.emplace_back(std::move(pipe)); } - auto pipe = Pipe::unitePipes(std::move(pipes)); + return Pipe::unitePipes(std::move(pipes)); +} - auto convert_actions_dag = ActionsDAG::makeConvertingActions( - pipe.getHeader().getColumnsWithTypeAndName(), - metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()).getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Name); - - auto actions = std::make_shared( - convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); - - pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); }); - - return pipe; +void StorageFileLog::drop() +{ + if (std::filesystem::exists(root_meta_path)) + std::filesystem::remove_all(root_meta_path); } void StorageFileLog::startup() @@ -161,7 +333,6 @@ void StorageFileLog::startup() } } - void StorageFileLog::shutdown() { if (task) @@ -171,6 +342,59 @@ void StorageFileLog::shutdown() LOG_TRACE(log, "Waiting for cleanup"); task->holder->deactivate(); } + closeFilesAndStoreMeta(); +} + +void StorageFileLog::openFilesAndSetPos() +{ + for (const auto & file : file_infos.file_names) + { + auto & context = file_infos.context_by_name.at(file); + if (context.status != FileStatus::NO_CHANGE) + { + context.reader = std::ifstream(getFullDataPath(file)); + if (!context.reader.good()) + { + throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Open file {} failed.", file); + } + + context.reader.seekg(0, context.reader.end); + if (!context.reader.good()) + { + throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Seekg file {} failed.", file); + } + + auto file_end = context.reader.tellg(); + if (!context.reader.good()) + { + throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Tellg file {} failed.", file); + } + + auto & meta = file_infos.meta_by_inode.at(file_infos.inode_by_name.at(file)); + if (meta.last_writen_position > static_cast(file_end)) + { + throw Exception(ErrorCodes::FILE_STREAM_ERROR, "File {} has been broken.", file); + } + /// updte file end at the monment, used in ReadBuffer and serialize + meta.last_open_end = file_end; + + context.reader.seekg(meta.last_writen_position); + if (!context.reader.good()) + { + throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Seekg file {} failed.", file); + } + } + } +} + +void StorageFileLog::closeFilesAndStoreMeta() +{ + for (auto & it : file_infos.context_by_name) + { + if (it.second.reader.is_open()) + it.second.reader.close(); + } + serialize(); } size_t StorageFileLog::getMaxBlockSize() const @@ -192,7 +416,6 @@ size_t StorageFileLog::getPollTimeoutMillisecond() const : getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); } - bool StorageFileLog::checkDependencies(const StorageID & table_id) { // Check if all dependencies are attached @@ -257,7 +480,7 @@ void StorageFileLog::threadFunc() LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); break; } - updateFileStatuses(); + updateFileInfos(); } } } @@ -271,7 +494,6 @@ void StorageFileLog::threadFunc() task->holder->scheduleAfter(milliseconds_to_wait); } - bool StorageFileLog::streamToViews() { std::lock_guard lock(status_mutex); @@ -283,7 +505,7 @@ bool StorageFileLog::streamToViews() throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); auto metadata_snapshot = getInMemoryMetadataPtr(); - auto max_streams_number = std::min(filelog_settings->filelog_max_threads.value, file_names.size()); + auto max_streams_number = std::min(filelog_settings->filelog_max_threads.value, file_infos.file_names.size()); /// No files to parse if (max_streams_number == 0) { @@ -299,33 +521,30 @@ bool StorageFileLog::streamToViews() InterpreterInsertQuery interpreter(insert, new_context, false, true, true); auto block_io = interpreter.execute(); + openFilesAndSetPos(); + serialize(); + Pipes pipes; pipes.reserve(max_streams_number); for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) { - pipes.emplace_back(std::make_shared( - *this, - metadata_snapshot, - new_context, - getPollMaxBatchSize(), - getPollTimeoutMillisecond(), - stream_number, - max_streams_number)); + Pipe pipe(std::make_shared( + *this, metadata_snapshot, new_context, getPollMaxBatchSize(), getPollTimeoutMillisecond(), stream_number, max_streams_number)); + + auto convert_actions_dag = ActionsDAG::makeConvertingActions( + pipe.getHeader().getColumnsWithTypeAndName(), + block_io.out->getHeader().getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto actions = std::make_shared( + convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); + + pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); }); + pipes.emplace_back(std::move(pipe)); } - auto pipe = Pipe::unitePipes(std::move(pipes)); - - auto convert_actions_dag = ActionsDAG::makeConvertingActions( - pipe.getHeader().getColumnsWithTypeAndName(), - block_io.out->getHeader().getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Name); - - auto actions = std::make_shared( - convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); - - pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); }); QueryPipeline pipeline; - pipeline.init(std::move(pipe)); + pipeline.init(Pipe::unitePipes(std::move(pipes))); assertBlocksHaveEqualStructure(pipeline.getHeader(), block_io.out->getHeader(), "StorageFileLog streamToViews"); @@ -338,14 +557,18 @@ bool StorageFileLog::streamToViews() { block_io.out->write(block); rows += block.rows(); + /// During files open, also save file end at the openning moment + serialize(true); } block_io.out->writeSuffix(); + serialize(); + UInt64 milliseconds = watch.elapsedMilliseconds(); LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); - return updateFileStatuses(); + return updateFileInfos(); } void registerStorageFileLog(StorageFactory & factory) @@ -395,7 +618,8 @@ void registerStorageFileLog(StorageFactory & factory) auto path = path_ast->as().value.safeGet(); auto format = format_ast->as().value.safeGet(); - return StorageFileLog::create(args.table_id, args.getContext(), args.columns, path, format, std::move(filelog_settings)); + return StorageFileLog::create( + args.table_id, args.getContext(), args.columns, path, format, std::move(filelog_settings), args.attach); }; factory.registerStorage( @@ -406,10 +630,24 @@ void registerStorageFileLog(StorageFactory & factory) }); } -bool StorageFileLog::updateFileStatuses() +bool StorageFileLog::updateFileInfos() { if (!directory_watch) + { + /// For table just watch one file, we can not use directory monitor to watch it + if (!path_is_directory) + { + assert( + file_infos.file_names.size() == file_infos.meta_by_inode.size() == file_infos.inode_by_name.size() + == file_infos.context_by_name.size() == 1); + if (auto it = file_infos.context_by_name.find(file_infos.file_names[0]); it != file_infos.context_by_name.end()) + { + it->second.status = FileStatus::UPDATED; + return true; + } + } return false; + } /// Do not need to hold file_status lock, since it will be holded /// by caller when call this function auto error = directory_watch->getErrorAndReset(); @@ -423,54 +661,95 @@ bool StorageFileLog::updateFileStatuses() switch (event.type) { case Poco::DirectoryWatcher::DW_ITEM_ADDED: { - auto normal_path = std::filesystem::path(event.path).lexically_normal().native(); LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - if (std::filesystem::is_regular_file(normal_path)) + if (std::filesystem::is_regular_file(event.path)) { - file_statuses[event.path] = FileContext{}; - file_names.push_back(normal_path); + auto file_name = std::filesystem::path(event.path).filename(); + auto inode = getInode(event.path); + + file_infos.file_names.push_back(file_name); + file_infos.inode_by_name.emplace(file_name, inode); + + FileMeta meta{file_name, 0, 0}; + file_infos.meta_by_inode.emplace(inode, meta); + file_infos.context_by_name.emplace(file_name, FileContext{}); } break; } case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: { - auto normal_path = std::filesystem::path(event.path).lexically_normal().native(); - LOG_TRACE(log, "New event {} watched, path: {}", event.callback, normal_path); - if (std::filesystem::is_regular_file(normal_path) && file_statuses.contains(normal_path)) + auto file_name = std::filesystem::path(event.path).filename(); + LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); + if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) { - file_statuses[normal_path].status = FileStatus::UPDATED; + it->second.status = FileStatus::UPDATED; } break; } case Poco::DirectoryWatcher::DW_ITEM_REMOVED: - case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: { - auto normal_path = std::filesystem::path(event.path).lexically_normal().native(); - LOG_TRACE(log, "New event {} watched, path: {}", event.callback, normal_path); - if (std::filesystem::is_regular_file(normal_path) && file_statuses.contains(normal_path)) + auto file_name = std::filesystem::path(event.path).filename(); + LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); + if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) { - file_statuses[normal_path].status = FileStatus::REMOVED; + it->second.status = FileStatus::REMOVED; + } + break; + } + /// file rename + case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: { + auto file_name = std::filesystem::path(event.path).filename(); + LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); + + file_infos.file_names.push_back(file_name); + file_infos.context_by_name.emplace(file_name, FileContext{}); + + auto inode = getInode(event.path); + file_infos.inode_by_name.emplace(file_name, inode); + + if (auto it = file_infos.meta_by_inode.find(inode); it != file_infos.meta_by_inode.end()) + { + // rename meta file + auto old_name = it->second.file_name; + it->second.file_name = file_name; + if (std::filesystem::exists(getFullMetaPath(old_name))) + { + std::filesystem::rename(getFullMetaPath(old_name), getFullMetaPath(file_name)); + } } } } } std::vector valid_files; - for (const auto & file_name : file_names) + + for (const auto & file_name : file_infos.file_names) { - if (file_statuses.at(file_name).status == FileStatus::REMOVED || file_statuses.at(file_name).status == FileStatus::BROKEN) + if (auto it = file_infos.context_by_name.find(file_name); + it != file_infos.context_by_name.end() && it->second.status == FileStatus::REMOVED) { - file_statuses.erase(file_name); + file_infos.context_by_name.erase(it); + if (auto inode = file_infos.inode_by_name.find(file_name); inode != file_infos.inode_by_name.end()) + { + file_infos.inode_by_name.erase(inode); + file_infos.meta_by_inode.erase(inode->second); + if (std::filesystem::exists(getFullMetaPath(file_name))) + std::filesystem::remove(getFullMetaPath(file_name)); + } } else { valid_files.push_back(file_name); } } + file_infos.file_names.swap(valid_files); - file_names.swap(valid_files); + /// These file infos should always have same size(one for one) + assert( + file_infos.file_names.size() == file_infos.meta_by_inode.size() == file_infos.inode_by_name.size() + == file_infos.context_by_name.size()); - return events.empty() || file_names.empty(); + return events.empty() || file_infos.file_names.empty(); } NamesAndTypesList StorageFileLog::getVirtuals() const diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index dc9485fccb2..d715fbbfefc 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -23,14 +23,6 @@ class StorageFileLog final : public shared_ptr_helper, public IS friend struct shared_ptr_helper; public: - enum class FileStatus - { - BEGIN, - NO_CHANGE, - UPDATED, - REMOVED, - BROKEN - }; using Files = std::vector; @@ -50,15 +42,55 @@ public: size_t max_block_size, unsigned num_streams) override; + void drop() override; + const auto & getFormatName() const { return format_name; } - auto & getFileNames() { return file_names; } - auto & getFileStatuses() { return file_statuses; } + enum class FileStatus + { + OPEN, /// first time open file after table start up + NO_CHANGE, + UPDATED, + REMOVED, + }; + + struct FileContext + { + FileStatus status = FileStatus::OPEN; + std::ifstream reader; + }; + + struct FileMeta + { + String file_name; + UInt64 last_writen_position = 0; + UInt64 last_open_end; + }; + + using FileNameToInode = std::unordered_map; + using InodeToFileMeta = std::unordered_map; + using FileNameToContext = std::unordered_map; + + struct FileInfos + { + FileNameToInode inode_by_name; + InodeToFileMeta meta_by_inode; + FileNameToContext context_by_name; + /// file names without path + Names file_names; + }; + + auto & getFileInfos() { return file_infos; } + + auto getFullMetaPath(const String & file_name) const { return root_meta_path + "/" + file_name; } + auto getFullDataPath(const String & file_name) const { return root_data_path + "/" + file_name; } NamesAndTypesList getVirtuals() const override; static Names getVirtualColumnNames(); + static UInt64 getInode(const String & file_name); + protected: StorageFileLog( const StorageID & table_id_, @@ -66,33 +98,33 @@ protected: const ColumnsDescription & columns_, const String & relative_path_, const String & format_name_, - std::unique_ptr settings); + std::unique_ptr settings, + bool attach); private: std::unique_ptr filelog_settings; + + /// user_files_path/ + path_argument/ const String path; + bool path_is_directory = true; + + /// If path argument of the table is a regular file, it equals to user_files_path + /// otherwise, it equals to user_files_path/ + path_argument/, e.g. path + String root_data_path; + /// Database meta_path/ + .table_name/ + String root_meta_path; + + FileInfos file_infos; const String format_name; Poco::Logger * log; - struct FileContext - { - FileStatus status = FileStatus::BEGIN; - size_t last_read_position = 0; - }; - - using NameToFile = std::unordered_map; - NameToFile file_statuses; - - std::vector file_names; - std::mutex status_mutex; std::unique_ptr directory_watch = nullptr; uint64_t milliseconds_to_wait; - // Stream thread struct TaskContext { BackgroundSchedulePool::TaskHolder holder; @@ -105,7 +137,9 @@ private: using TaskThread = BackgroundSchedulePool::TaskHolder; - void init(); + void loadFiles(); + + void loadMetaFiles(bool attach); void threadFunc(); @@ -116,7 +150,15 @@ private: bool streamToViews(); bool checkDependencies(const StorageID & table_id); - bool updateFileStatuses(); + bool updateFileInfos(); + + void openFilesAndSetPos(); + void closeFilesAndStoreMeta(); + + /// Serialize all file meta + void serialize(bool with_end_pos = false) const; + + void deserialize(); }; } diff --git a/tests/queries/0_stateless/02022_storage_filelog_one_file.reference b/tests/queries/0_stateless/02022_storage_filelog_one_file.reference new file mode 100644 index 00000000000..88332de037f --- /dev/null +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.reference @@ -0,0 +1,41 @@ +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 +100 100 +101 101 +102 102 +103 103 +104 104 +105 105 +106 106 +107 107 +108 108 +109 109 +110 110 +111 111 +112 112 +113 113 +114 114 +115 115 +116 116 +117 117 +118 118 +119 119 +120 120 diff --git a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh new file mode 100755 index 00000000000..63450a83311 --- /dev/null +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh @@ -0,0 +1,42 @@ +#!/usr/bin/env bash +# Tags: long, no-parallel + +set -eu + +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')" +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}') + +for i in {1..20} +do + echo $i, $i >> ${user_files_path}/a.txt +done + +${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('a.txt', 'CSV');" + +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" + +for i in {100..120} +do + echo $i, $i >> ${user_files_path}/a.txt +done + +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" + +# touch does not change file content, no event +touch ${user_files_path}/a.txt +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" + +${CLICKHOUSE_CLIENT} --query "detach table file_log;" +${CLICKHOUSE_CLIENT} --query "attach table file_log;" + +# should no records return +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" + +rm -rf ${user_files_path}/a.txt diff --git a/tests/queries/0_stateless/02023_storage_filelog.reference b/tests/queries/0_stateless/02023_storage_filelog.reference index f1aa3abf257..0ab71c65c6b 100644 --- a/tests/queries/0_stateless/02023_storage_filelog.reference +++ b/tests/queries/0_stateless/02023_storage_filelog.reference @@ -8,7 +8,16 @@ 8 8 9 9 10 10 -100 100 +11 11 +12 12 +13 13 +14 14 +15 15 +16 16 +17 17 +18 18 +19 19 +20 20 1 1 2 2 3 3 @@ -19,7 +28,16 @@ 8 8 9 9 10 10 -100 100 +11 11 +12 12 +13 13 +14 14 +15 15 +16 16 +17 17 +18 18 +19 19 +20 20 1 1 1 1 2 2 @@ -40,28 +58,87 @@ 9 9 10 10 10 10 +11 11 +11 11 +12 12 +12 12 +13 13 +13 13 +14 14 +14 14 +15 15 +15 15 +16 16 +16 16 +17 17 +17 17 +18 18 +18 18 +19 19 +19 19 +20 20 +20 20 100 100 100 100 -1 1 -1 1 -2 2 -2 2 -3 3 -3 3 -4 4 -4 4 -5 5 -5 5 -6 6 -6 6 -7 7 -7 7 -8 8 -8 8 -9 9 -9 9 -10 10 -10 10 -100 100 100 100 +101 101 +101 101 +101 101 +102 102 +102 102 +102 102 +103 103 +103 103 +103 103 +104 104 +104 104 +104 104 +105 105 +105 105 +105 105 +106 106 +106 106 +106 106 +107 107 +107 107 +107 107 +108 108 +108 108 +108 108 +109 109 +109 109 +109 109 +110 110 +110 110 +110 110 111 111 +111 111 +111 111 +112 112 +112 112 +112 112 +113 113 +113 113 +113 113 +114 114 +114 114 +114 114 +115 115 +115 115 +115 115 +116 116 +116 116 +116 116 +117 117 +117 117 +117 117 +118 118 +118 118 +118 118 +119 119 +119 119 +119 119 +120 120 +120 120 +120 120 +OK diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 97a147479ac..eae9821d950 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -10,67 +10,53 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # 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')" -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}') +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 ${user_files_path}/logs/ -echo 1, 1 >> ${user_files_path}/logs/a.txt -echo 2, 2 >> ${user_files_path}/logs/a.txt -echo 3, 3 >> ${user_files_path}/logs/a.txt -echo 4, 4 >> ${user_files_path}/logs/a.txt -echo 5, 5 >> ${user_files_path}/logs/a.txt -echo 6, 6 >> ${user_files_path}/logs/a.txt -echo 7, 7 >> ${user_files_path}/logs/a.txt -echo 8, 8 >> ${user_files_path}/logs/a.txt -echo 9, 9 >> ${user_files_path}/logs/a.txt -echo 10, 10 >> ${user_files_path}/logs/a.txt + +rm -rf ${user_files_path}/logs/* + +for i in {1..20} +do + echo $i, $i >> ${user_files_path}/logs/a.txt +done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" ${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('logs', 'CSV');" ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" -echo 100, 100 >> ${user_files_path}/logs/a.txt +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/b.txt ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" -cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/b.txt +for i in {100..120} +do + echo $i, $i >> ${user_files_path}/logs/a.txt +done # touch does not change file content, no event touch ${user_files_path}/logs/a.txt -${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" - -${CLICKHOUSE_CLIENT} --query "drop table if exists mv;" -${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order by k as select * from file_log;" - cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/c.txt cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt -# MOVE_FILE_FROM and MOVE_FILE_TO monitored -mv ${user_files_path}/logs/a.txt ${user_files_path}/logs/e.txt -rm ${user_files_path}/logs/e.txt +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/e.txt -# the max value of reschedule time is 32s, so 40s should -# always be enough to finish streamToViews -for _ in {1..150} -do - sleep 0.1 -done +rm ${user_files_path}/logs/d.txt -${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" -echo 111, 111 >> ${user_files_path}/logs/c.txt -truncate ${user_files_path}/logs/d.txt --size 0 +${CLICKHOUSE_CLIENT} --query "detach table file_log;" +${CLICKHOUSE_CLIENT} --query "attach table file_log;" -# the max value of reschedule time is 32s, so 40s should -# always be enough to finish streamToViews -for _ in {1..150} -do - sleep 0.1 -done +# should no records return +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" -${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" +truncate ${user_files_path}/logs/a.txt --size 0 + +# exception happend +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" 2>&1 | grep -q "Code: 626" && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} --query "drop table file_log;" -${CLICKHOUSE_CLIENT} --query "drop table mv;" rm -rf ${user_files_path}/logs diff --git a/tests/queries/0_stateless/02024_storage_filelog_mv.reference b/tests/queries/0_stateless/02024_storage_filelog_mv.reference new file mode 100644 index 00000000000..2ebb200a43a --- /dev/null +++ b/tests/queries/0_stateless/02024_storage_filelog_mv.reference @@ -0,0 +1,121 @@ +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 +1 1 +1 1 +1 1 +1 1 +2 2 +2 2 +2 2 +2 2 +3 3 +3 3 +3 3 +3 3 +4 4 +4 4 +4 4 +4 4 +5 5 +5 5 +5 5 +5 5 +6 6 +6 6 +6 6 +6 6 +7 7 +7 7 +7 7 +7 7 +8 8 +8 8 +8 8 +8 8 +9 9 +9 9 +9 9 +9 9 +10 10 +10 10 +10 10 +10 10 +11 11 +11 11 +11 11 +11 11 +12 12 +12 12 +12 12 +12 12 +13 13 +13 13 +13 13 +13 13 +14 14 +14 14 +14 14 +14 14 +15 15 +15 15 +15 15 +15 15 +16 16 +16 16 +16 16 +16 16 +17 17 +17 17 +17 17 +17 17 +18 18 +18 18 +18 18 +18 18 +19 19 +19 19 +19 19 +19 19 +20 20 +20 20 +20 20 +20 20 +100 100 +101 101 +102 102 +103 103 +104 104 +105 105 +106 106 +107 107 +108 108 +109 109 +110 110 +111 111 +112 112 +113 113 +114 114 +115 115 +116 116 +117 117 +118 118 +119 119 +120 120 diff --git a/tests/queries/0_stateless/02024_storage_filelog_mv.sh b/tests/queries/0_stateless/02024_storage_filelog_mv.sh new file mode 100755 index 00000000000..d6d912624a8 --- /dev/null +++ b/tests/queries/0_stateless/02024_storage_filelog_mv.sh @@ -0,0 +1,58 @@ +#!/usr/bin/env bash +# Tags: long, no-parallel + +set -eu + +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')" +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 ${user_files_path}/logs/ +rm -rf ${user_files_path}/logs/* +for i in {1..20} +do + echo $i, $i >> ${user_files_path}/logs/a.txt +done + +${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('logs', 'CSV');" + +${CLICKHOUSE_CLIENT} --query "drop table if exists mv;" +${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order by k as select * from file_log;" + +for i in {1..200} +do + sleep 0.1 +done + +${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" + +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/b.txt + +# touch does not change file content, no event +touch ${user_files_path}/logs/a.txt + +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/c.txt +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt + +for i in {100..120} +do + echo $i, $i >> ${user_files_path}/logs/d.txt +done + +for _ in {1..300} +do + sleep 0.1 +done + +${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" + +${CLICKHOUSE_CLIENT} --query "drop table mv;" +${CLICKHOUSE_CLIENT} --query "drop table file_log;" + +rm -rf ${user_files_path}/logs diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.reference b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.reference new file mode 100644 index 00000000000..aaa25ebb451 --- /dev/null +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.reference @@ -0,0 +1,144 @@ +1 1 a.txt 0 +2 2 a.txt 5 +3 3 a.txt 10 +4 4 a.txt 15 +5 5 a.txt 20 +6 6 a.txt 25 +7 7 a.txt 30 +8 8 a.txt 35 +9 9 a.txt 40 +10 10 a.txt 45 +11 11 a.txt 52 +12 12 a.txt 59 +13 13 a.txt 66 +14 14 a.txt 73 +15 15 a.txt 80 +16 16 a.txt 87 +17 17 a.txt 94 +18 18 a.txt 101 +19 19 a.txt 108 +20 20 a.txt 115 +1 1 b.txt 0 +2 2 b.txt 5 +3 3 b.txt 10 +4 4 b.txt 15 +5 5 b.txt 20 +6 6 b.txt 25 +7 7 b.txt 30 +8 8 b.txt 35 +9 9 b.txt 40 +10 10 b.txt 45 +11 11 b.txt 52 +12 12 b.txt 59 +13 13 b.txt 66 +14 14 b.txt 73 +15 15 b.txt 80 +16 16 b.txt 87 +17 17 b.txt 94 +18 18 b.txt 101 +19 19 b.txt 108 +20 20 b.txt 115 +100 100 a.txt 122 +101 101 a.txt 131 +102 102 a.txt 140 +103 103 a.txt 149 +104 104 a.txt 158 +105 105 a.txt 167 +106 106 a.txt 176 +107 107 a.txt 185 +108 108 a.txt 194 +109 109 a.txt 203 +110 110 a.txt 212 +111 111 a.txt 221 +112 112 a.txt 230 +113 113 a.txt 239 +114 114 a.txt 248 +115 115 a.txt 257 +116 116 a.txt 266 +117 117 a.txt 275 +118 118 a.txt 284 +119 119 a.txt 293 +120 120 a.txt 302 +1 1 c.txt 0 +2 2 c.txt 5 +3 3 c.txt 10 +4 4 c.txt 15 +5 5 c.txt 20 +6 6 c.txt 25 +7 7 c.txt 30 +8 8 c.txt 35 +9 9 c.txt 40 +10 10 c.txt 45 +11 11 c.txt 52 +12 12 c.txt 59 +13 13 c.txt 66 +14 14 c.txt 73 +15 15 c.txt 80 +16 16 c.txt 87 +17 17 c.txt 94 +18 18 c.txt 101 +19 19 c.txt 108 +20 20 c.txt 115 +100 100 c.txt 122 +101 101 c.txt 131 +102 102 c.txt 140 +103 103 c.txt 149 +104 104 c.txt 158 +105 105 c.txt 167 +106 106 c.txt 176 +107 107 c.txt 185 +108 108 c.txt 194 +109 109 c.txt 203 +110 110 c.txt 212 +111 111 c.txt 221 +112 112 c.txt 230 +113 113 c.txt 239 +114 114 c.txt 248 +115 115 c.txt 257 +116 116 c.txt 266 +117 117 c.txt 275 +118 118 c.txt 284 +119 119 c.txt 293 +120 120 c.txt 302 +1 1 e.txt 0 +2 2 e.txt 5 +3 3 e.txt 10 +4 4 e.txt 15 +5 5 e.txt 20 +6 6 e.txt 25 +7 7 e.txt 30 +8 8 e.txt 35 +9 9 e.txt 40 +10 10 e.txt 45 +11 11 e.txt 52 +12 12 e.txt 59 +13 13 e.txt 66 +14 14 e.txt 73 +15 15 e.txt 80 +16 16 e.txt 87 +17 17 e.txt 94 +18 18 e.txt 101 +19 19 e.txt 108 +20 20 e.txt 115 +100 100 e.txt 122 +101 101 e.txt 131 +102 102 e.txt 140 +103 103 e.txt 149 +104 104 e.txt 158 +105 105 e.txt 167 +106 106 e.txt 176 +107 107 e.txt 185 +108 108 e.txt 194 +109 109 e.txt 203 +110 110 e.txt 212 +111 111 e.txt 221 +112 112 e.txt 230 +113 113 e.txt 239 +114 114 e.txt 248 +115 115 e.txt 257 +116 116 e.txt 266 +117 117 e.txt 275 +118 118 e.txt 284 +119 119 e.txt 293 +120 120 e.txt 302 +OK diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh new file mode 100755 index 00000000000..478b320bfd9 --- /dev/null +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -0,0 +1,62 @@ +#!/usr/bin/env bash +# Tags: long, no-parallel + +set -eu + +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')" +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 ${user_files_path}/logs/ + +rm -rf ${user_files_path}/logs/* + +for i in {1..20} +do + echo $i, $i >> ${user_files_path}/logs/a.txt +done + +${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('logs', 'CSV');" + +${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" + +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/b.txt + +${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" + +for i in {100..120} +do + echo $i, $i >> ${user_files_path}/logs/a.txt +done + +# touch does not change file content, no event +touch ${user_files_path}/logs/a.txt + +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/c.txt +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt +cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/e.txt + +rm ${user_files_path}/logs/d.txt + +${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" + +${CLICKHOUSE_CLIENT} --query "detach table t;" +${CLICKHOUSE_CLIENT} --query "attach table t;" + +# should no records return +${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" + +truncate ${user_files_path}/logs/a.txt --size 0 + +# exception happend +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" 2>&1 | grep -q "Code: 626" && echo 'OK' || echo 'FAIL' + +${CLICKHOUSE_CLIENT} --query "drop table file_log;" + +rm -rf ${user_files_path}/logs From 30527879b9626e1c7591b990bbab4f7226d647c9 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 26 Sep 2021 12:19:19 +0000 Subject: [PATCH 054/919] fix style --- src/Storages/FileLog/StorageFileLog.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 9b64be4c99b..ebc8bb9ba8a 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -34,7 +34,6 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; extern const int CANNOT_GET_FILE_STAT; @@ -375,7 +374,7 @@ void StorageFileLog::openFilesAndSetPos() { throw Exception(ErrorCodes::FILE_STREAM_ERROR, "File {} has been broken.", file); } - /// updte file end at the monment, used in ReadBuffer and serialize + /// update file end at the monment, used in ReadBuffer and serialize meta.last_open_end = file_end; context.reader.seekg(meta.last_writen_position); @@ -557,7 +556,7 @@ bool StorageFileLog::streamToViews() { block_io.out->write(block); rows += block.rows(); - /// During files open, also save file end at the openning moment + /// During files open, also save file end at the opening moment serialize(true); } block_io.out->writeSuffix(); From a1e61ebedf51a2ec30e50a9342007c0ef6be1a41 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 26 Sep 2021 12:21:20 +0000 Subject: [PATCH 055/919] udpate tests --- tests/queries/0_stateless/02022_storage_filelog_one_file.sh | 2 +- tests/queries/0_stateless/02023_storage_filelog.sh | 2 +- tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh index 63450a83311..3500aabec65 100755 --- a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: no-parallel set -eu diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index eae9821d950..f9ecf1e7101 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: no-parallel set -eu diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index 478b320bfd9..56c5dffe365 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: no-parallel set -eu From f8f22f9a513f0a78b1f3798dc82dbf1eaf6377e1 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 26 Sep 2021 13:07:31 +0000 Subject: [PATCH 056/919] fix test --- tests/queries/0_stateless/02022_storage_filelog_one_file.sh | 2 +- tests/queries/0_stateless/02023_storage_filelog.sh | 2 +- tests/queries/0_stateless/02024_storage_filelog_mv.sh | 2 +- tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh index 3500aabec65..fa36775a731 100755 --- a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # 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')" -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}') +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}') for i in {1..20} do diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index f9ecf1e7101..7e843075181 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # 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')" -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}') +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 ${user_files_path}/logs/ diff --git a/tests/queries/0_stateless/02024_storage_filelog_mv.sh b/tests/queries/0_stateless/02024_storage_filelog_mv.sh index d6d912624a8..f73dc988c92 100755 --- a/tests/queries/0_stateless/02024_storage_filelog_mv.sh +++ b/tests/queries/0_stateless/02024_storage_filelog_mv.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # 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')" -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}') +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 ${user_files_path}/logs/ rm -rf ${user_files_path}/logs/* diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index 56c5dffe365..8e9769d060d 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # 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')" -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}') +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 ${user_files_path}/logs/ From 6c4b0f2d37d0ced4e2302b93edd858fed6a37e80 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 26 Sep 2021 13:37:57 +0000 Subject: [PATCH 057/919] drop table immediately fix style update --- src/Databases/DatabaseAtomic.cpp | 5 +++++ src/Interpreters/DatabaseCatalog.cpp | 2 +- src/Storages/FileLog/FileLogSource.cpp | 7 +------ src/Storages/FileLog/StorageFileLog.cpp | 3 ++- src/Storages/FileLog/StorageFileLog.h | 2 ++ src/Storages/IStorage.h | 2 ++ .../0_stateless/02025_storage_filelog_virtual_col.sh | 4 ++-- 7 files changed, 15 insertions(+), 10 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 5c75f6f1036..26f694cb5dd 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -140,6 +140,11 @@ void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_na if (table->storesDataOnDisk()) tryRemoveSymlink(table_name); + if (table->dropTableImmediately()) + { + table->drop(); + } + /// Notify DatabaseCatalog that table was dropped. It will remove table data in background. /// Cleanup is performed outside of database to allow easily DROP DATABASE without waiting for cleanup to complete. DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop, no_delay); diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index f273f8a165d..c87a9b91640 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -898,7 +898,7 @@ void DatabaseCatalog::dropTableDataTask() void DatabaseCatalog::dropTableFinally(const TableMarkedAsDropped & table) { - if (table.table) + if (table.table && !table.table->dropTableImmediately()) { table.table->drop(); } diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index c272df1c2b2..a03ad1a4c51 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -11,12 +11,7 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -const auto MAX_FAILED_POLL_ATTEMPTS = 10; +static constexpr auto MAX_FAILED_POLL_ATTEMPTS = 10; FileLogSource::FileLogSource( StorageFileLog & storage_, diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index ebc8bb9ba8a..ca7df3c5e4b 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -39,7 +39,8 @@ namespace ErrorCodes extern const int CANNOT_GET_FILE_STAT; extern const int NOT_REGULAR_FILE; extern const int READ_META_FILE_FAILED; - extern const int FILE_STREAM_ERROR; + extern const int FILE_STREAM_ERROR; + extern const int LOGICAL_ERROR; } namespace diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index d715fbbfefc..4daf2bd175a 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -44,6 +44,8 @@ public: void drop() override; + bool dropTableImmediately() const override { return true; } + const auto & getFormatName() const { return format_name; } enum class FileStatus diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 90cb963e064..baf57c40185 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -578,6 +578,8 @@ public: /// Does not takes underlying Storage (if any) into account. virtual std::optional lifetimeBytes() const { return {}; } + virtual bool dropTableImmediately() const { return false; } + private: /// Lock required for alter queries (lockForAlter). Always taken for write /// (actually can be replaced with std::mutex, but for consistency we use diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index 8e9769d060d..c5e1553c41a 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -46,8 +46,8 @@ rm ${user_files_path}/logs/d.txt ${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" -${CLICKHOUSE_CLIENT} --query "detach table t;" -${CLICKHOUSE_CLIENT} --query "attach table t;" +${CLICKHOUSE_CLIENT} --query "detach table file_log;" +${CLICKHOUSE_CLIENT} --query "attach table file_log;" # should no records return ${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" From efddc03246e02c2dbcf35cbe0643ea436c0780f0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 26 Sep 2021 22:49:28 +0300 Subject: [PATCH 058/919] Async read for disk web --- src/Disks/DiskWebServer.cpp | 17 +++++++++++++---- src/Disks/ReadIndirectBufferFromWebServer.cpp | 15 ++++++++++++--- src/Disks/ReadIndirectBufferFromWebServer.h | 10 ++++++---- 3 files changed, 31 insertions(+), 11 deletions(-) diff --git a/src/Disks/DiskWebServer.cpp b/src/Disks/DiskWebServer.cpp index 439d59f21a9..9c4747aefdb 100644 --- a/src/Disks/DiskWebServer.cpp +++ b/src/Disks/DiskWebServer.cpp @@ -116,19 +116,22 @@ public: ContextPtr context_, size_t buf_size_, size_t backoff_threshold_, - size_t max_tries_) + size_t max_tries_, + size_t threadpool_read_) : ReadBufferFromRemoteFS(metadata_) , uri(uri_) , context(context_) , buf_size(buf_size_) , backoff_threshold(backoff_threshold_) , max_tries(max_tries_) + , threadpool_read(threadpool_read_) { } SeekableReadBufferPtr createReadBuffer(const String & path) const override { - return std::make_unique(fs::path(uri) / path, context, buf_size, backoff_threshold, max_tries); + return std::make_unique( + fs::path(uri) / path, context, buf_size, backoff_threshold, max_tries, threadpool_read); } private: @@ -137,6 +140,7 @@ private: size_t buf_size; size_t backoff_threshold; size_t max_tries; + bool threadpool_read; }; @@ -198,9 +202,14 @@ std::unique_ptr DiskWebServer::readFile(const String & p RemoteMetadata meta(path, remote_path); meta.remote_fs_objects.emplace_back(std::make_pair(remote_path, iter->second.size)); + bool threadpool_read = read_settings.remote_fs_method == RemoteFSReadMethod::read_threadpool; + auto web_impl = std::make_unique(url, meta, getContext(), - read_settings.remote_fs_buffer_size, read_settings.remote_fs_backoff_threshold, read_settings.remote_fs_backoff_max_tries); - if (read_settings.remote_fs_method == RemoteFSReadMethod::read_threadpool) + read_settings.remote_fs_buffer_size, + read_settings.remote_fs_backoff_threshold, read_settings.remote_fs_backoff_max_tries, + threadpool_read); + + if (threadpool_read) { auto reader = IDiskRemote::getThreadPoolReader(); auto buf = std::make_unique(reader, read_settings.priority, std::move(web_impl)); diff --git a/src/Disks/ReadIndirectBufferFromWebServer.cpp b/src/Disks/ReadIndirectBufferFromWebServer.cpp index 809e6d67107..c0d41bb006a 100644 --- a/src/Disks/ReadIndirectBufferFromWebServer.cpp +++ b/src/Disks/ReadIndirectBufferFromWebServer.cpp @@ -24,7 +24,8 @@ static const auto WAIT_MS = 10; ReadIndirectBufferFromWebServer::ReadIndirectBufferFromWebServer( - const String & url_, ContextPtr context_, size_t buf_size_, size_t backoff_threshold_, size_t max_tries_) + const String & url_, ContextPtr context_, size_t buf_size_, + size_t backoff_threshold_, size_t max_tries_, bool use_external_buffer_) : BufferWithOwnMemory(buf_size_) , log(&Poco::Logger::get("ReadIndirectBufferFromWebServer")) , context(context_) @@ -32,6 +33,7 @@ ReadIndirectBufferFromWebServer::ReadIndirectBufferFromWebServer( , buf_size(buf_size_) , backoff_threshold_ms(backoff_threshold_) , max_tries(max_tries_) + , use_external_buffer(use_external_buffer_) { } @@ -70,8 +72,15 @@ bool ReadIndirectBufferFromWebServer::nextImpl() if (impl) { - /// Restore correct position at the needed offset. - impl->position() = position(); + if (use_external_buffer) + { + impl->set(working_buffer.begin(), working_buffer.size()); + } + else + { + impl->position() = position(); + } + assert(!impl->hasPendingData()); } diff --git a/src/Disks/ReadIndirectBufferFromWebServer.h b/src/Disks/ReadIndirectBufferFromWebServer.h index 04bb155f83b..55885427887 100644 --- a/src/Disks/ReadIndirectBufferFromWebServer.h +++ b/src/Disks/ReadIndirectBufferFromWebServer.h @@ -16,10 +16,10 @@ namespace DB class ReadIndirectBufferFromWebServer : public BufferWithOwnMemory { public: - explicit ReadIndirectBufferFromWebServer(const String & url_, - ContextPtr context_, - size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE, - size_t backoff_threshold_ = 10000, size_t max_tries_ = 4); + explicit ReadIndirectBufferFromWebServer( + const String & url_, ContextPtr context_, + size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE, + size_t backoff_threshold_ = 10000, size_t max_tries_ = 4, bool use_external_buffer_ = false); bool nextImpl() override; @@ -42,6 +42,8 @@ private: size_t backoff_threshold_ms; size_t max_tries; + + bool use_external_buffer; }; } From 69985ac4ae928ebe762f47dcbebd31c63ea7b7c5 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 27 Sep 2021 02:27:24 +0000 Subject: [PATCH 059/919] fix --- src/Storages/FileLog/StorageFileLog.cpp | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index ca7df3c5e4b..b9e9566cddc 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -349,23 +349,23 @@ void StorageFileLog::openFilesAndSetPos() { for (const auto & file : file_infos.file_names) { - auto & context = file_infos.context_by_name.at(file); - if (context.status != FileStatus::NO_CHANGE) + auto & file_ctx = file_infos.context_by_name.at(file); + if (file_ctx.status != FileStatus::NO_CHANGE) { - context.reader = std::ifstream(getFullDataPath(file)); - if (!context.reader.good()) + file_ctx.reader = std::ifstream(getFullDataPath(file)); + if (!file_ctx.reader.good()) { throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Open file {} failed.", file); } - context.reader.seekg(0, context.reader.end); - if (!context.reader.good()) + file_ctx.reader.seekg(0, file_ctx.reader.end); + if (!file_ctx.reader.good()) { throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Seekg file {} failed.", file); } - auto file_end = context.reader.tellg(); - if (!context.reader.good()) + auto file_end = file_ctx.reader.tellg(); + if (!file_ctx.reader.good()) { throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Tellg file {} failed.", file); } @@ -378,8 +378,8 @@ void StorageFileLog::openFilesAndSetPos() /// update file end at the monment, used in ReadBuffer and serialize meta.last_open_end = file_end; - context.reader.seekg(meta.last_writen_position); - if (!context.reader.good()) + file_ctx.reader.seekg(meta.last_writen_position); + if (!file_ctx.reader.good()) { throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Seekg file {} failed.", file); } From c8062ad3971de46866bac10485a0de5194199aaa Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 27 Sep 2021 04:25:59 +0000 Subject: [PATCH 060/919] fix --- src/Storages/FileLog/StorageFileLog.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index b9e9566cddc..5507180807b 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -294,8 +294,8 @@ Pipe StorageFileLog::read( auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_infos.file_names.size()); + /// When call closeFilesAndStoreMeta() ? openFilesAndSetPos(); - serialize(true); Pipes pipes; pipes.reserve(max_streams_number); @@ -352,7 +352,7 @@ void StorageFileLog::openFilesAndSetPos() auto & file_ctx = file_infos.context_by_name.at(file); if (file_ctx.status != FileStatus::NO_CHANGE) { - file_ctx.reader = std::ifstream(getFullDataPath(file)); + file_ctx.reader.open(getFullDataPath(file)); if (!file_ctx.reader.good()) { throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Open file {} failed.", file); @@ -385,6 +385,7 @@ void StorageFileLog::openFilesAndSetPos() } } } + serialize(true); } void StorageFileLog::closeFilesAndStoreMeta() @@ -522,7 +523,6 @@ bool StorageFileLog::streamToViews() auto block_io = interpreter.execute(); openFilesAndSetPos(); - serialize(); Pipes pipes; pipes.reserve(max_streams_number); @@ -562,7 +562,7 @@ bool StorageFileLog::streamToViews() } block_io.out->writeSuffix(); - serialize(); + closeFilesAndStoreMeta(); UInt64 milliseconds = watch.elapsedMilliseconds(); LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", From 1eb96c2fceef8b1c9f904c55bfb55993e79f9bfd Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 27 Sep 2021 04:39:50 +0000 Subject: [PATCH 061/919] fix --- src/Storages/FileLog/FileLogSource.cpp | 16 ++++++++++++++++ src/Storages/FileLog/FileLogSource.h | 5 +++++ src/Storages/FileLog/StorageFileLog.cpp | 7 ------- src/Storages/FileLog/StorageFileLog.h | 6 +++--- 4 files changed, 24 insertions(+), 10 deletions(-) diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index a03ad1a4c51..590630658ca 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -27,11 +27,27 @@ FileLogSource::FileLogSource( , context(context_) , max_block_size(max_block_size_) , poll_time_out(poll_time_out_) + , stream_number(stream_number_) + , max_streams_number(max_streams_number_) , non_virtual_header(metadata_snapshot_->getSampleBlockNonMaterialized()) , virtual_header( metadata_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames(), storage.getVirtuals(), storage.getStorageID())) { buffer = std::make_unique(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_); + /// The last FileLogSource responsible for open files + if (stream_number == max_streams_number - 1) + { + storage.openFilesAndSetPos(); + } +} + +FileLogSource::~FileLogSource() +{ + /// The last FileLogSource responsible for close files + if (stream_number == max_streams_number - 1) + { + storage.closeFilesAndStoreMeta(); + } } Chunk FileLogSource::generate() diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index 7f7d75bfaad..115c68f2e25 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -28,6 +28,8 @@ public: bool noRecords() { return !buffer || buffer->noRecords(); } + virtual ~FileLogSource() override; + protected: Chunk generate() override; @@ -39,6 +41,9 @@ private: size_t poll_time_out; + size_t stream_number; + size_t max_streams_number; + std::unique_ptr buffer; Block non_virtual_header; diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 5507180807b..5eec284b92b 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -294,9 +294,6 @@ Pipe StorageFileLog::read( auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_infos.file_names.size()); - /// When call closeFilesAndStoreMeta() ? - openFilesAndSetPos(); - Pipes pipes; pipes.reserve(max_streams_number); for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) @@ -522,8 +519,6 @@ bool StorageFileLog::streamToViews() InterpreterInsertQuery interpreter(insert, new_context, false, true, true); auto block_io = interpreter.execute(); - openFilesAndSetPos(); - Pipes pipes; pipes.reserve(max_streams_number); for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) @@ -562,8 +557,6 @@ bool StorageFileLog::streamToViews() } block_io.out->writeSuffix(); - closeFilesAndStoreMeta(); - UInt64 milliseconds = watch.elapsedMilliseconds(); LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 4daf2bd175a..58e4dc9d6eb 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -93,6 +93,9 @@ public: static UInt64 getInode(const String & file_name); + void openFilesAndSetPos(); + void closeFilesAndStoreMeta(); + protected: StorageFileLog( const StorageID & table_id_, @@ -154,9 +157,6 @@ private: bool updateFileInfos(); - void openFilesAndSetPos(); - void closeFilesAndStoreMeta(); - /// Serialize all file meta void serialize(bool with_end_pos = false) const; From 56be3fab0035eb46cab41f376d896c45e43b41e7 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 27 Sep 2021 07:17:16 +0000 Subject: [PATCH 062/919] fix --- src/Storages/FileLog/StorageFileLog.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 5eec284b92b..2acff24059c 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -75,9 +75,9 @@ StorageFileLog::StorageFileLog( loadMetaFiles(attach); loadFiles(); - assert( - file_infos.file_names.size() == file_infos.meta_by_inode.size() == file_infos.inode_by_name.size() - == file_infos.context_by_name.size()); + assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); + assert(file_infos.file_names.size() == file_infos.inode_by_name.size()); + assert(file_infos.file_names.size() == file_infos.context_by_name.size()); if (path_is_directory) directory_watch = std::make_unique(path); @@ -738,9 +738,9 @@ bool StorageFileLog::updateFileInfos() file_infos.file_names.swap(valid_files); /// These file infos should always have same size(one for one) - assert( - file_infos.file_names.size() == file_infos.meta_by_inode.size() == file_infos.inode_by_name.size() - == file_infos.context_by_name.size()); + assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); + assert(file_infos.file_names.size() == file_infos.inode_by_name.size()); + assert(file_infos.file_names.size() == file_infos.context_by_name.size()); return events.empty() || file_infos.file_names.empty(); } From 14b1c438ccf2dff7fedd4c9b8b1f64d5ba65da04 Mon Sep 17 00:00:00 2001 From: Kevin Michel Date: Tue, 21 Sep 2021 17:47:48 +0200 Subject: [PATCH 063/919] Enable all tests using ATTACH/FETCH on Replicated database --- .../queries/0_stateless/00626_replace_partition_from_table.sql | 3 +-- .../00626_replace_partition_from_table_zookeeper.sh | 3 +-- tests/queries/0_stateless/00753_alter_attach.sql | 3 +-- tests/queries/0_stateless/00955_test_final_mark.sql | 3 +-- tests/queries/0_stateless/00955_test_final_mark_use.sh | 3 +-- tests/queries/0_stateless/01015_attach_part.sql | 3 +-- tests/queries/0_stateless/01021_only_tuple_columns.sql | 3 +-- .../queries/0_stateless/01060_shutdown_table_after_detach.sql | 3 +-- tests/queries/0_stateless/01130_in_memory_parts_partitons.sql | 3 +-- .../0_stateless/01417_freeze_partition_verbose_zookeeper.sh | 2 +- tests/queries/0_stateless/01451_detach_drop_part.sql | 3 --- .../0_stateless/01451_replicated_detach_drop_part_long.sql | 2 +- .../01650_fetch_patition_with_macro_in_zk_path_long.sql | 3 +-- tests/queries/0_stateless/02009_array_join_partition.sql | 3 --- .../0_stateless/02012_changed_enum_type_non_replicated.sql | 3 +-- .../queries/0_stateless/02012_zookeeper_changed_enum_type.sql | 2 +- .../02012_zookeeper_changed_enum_type_incompatible.sql | 2 +- tests/queries/1_stateful/00054_merge_tree_partitions.sql | 3 --- .../queries/1_stateful/00152_insert_different_granularity.sql | 2 +- 19 files changed, 16 insertions(+), 36 deletions(-) diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table.sql b/tests/queries/0_stateless/00626_replace_partition_from_table.sql index 1e3e07f8274..958d2184c46 100644 --- a/tests/queries/0_stateless/00626_replace_partition_from_table.sql +++ b/tests/queries/0_stateless/00626_replace_partition_from_table.sql @@ -1,5 +1,4 @@ --- Tags: no-replicated-database, no-parallel --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tags: no-parallel DROP TABLE IF EXISTS src; DROP TABLE IF EXISTS dst; diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh index 7b3f787eeab..d78f93d6bb3 100755 --- a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh @@ -1,6 +1,5 @@ #!/usr/bin/env bash -# Tags: zookeeper, no-replicated-database, no-parallel -# Tag no-replicated-database: Unsupported type of ALTER query +# Tags: zookeeper, no-parallel # Because REPLACE PARTITION does not forces immediate removal of replaced data parts from local filesystem # (it tries to do it as quick as possible, but it still performed in separate thread asynchronously) diff --git a/tests/queries/0_stateless/00753_alter_attach.sql b/tests/queries/0_stateless/00753_alter_attach.sql index a9661d3a304..ca43fb3aeae 100644 --- a/tests/queries/0_stateless/00753_alter_attach.sql +++ b/tests/queries/0_stateless/00753_alter_attach.sql @@ -1,5 +1,4 @@ --- Tags: no-replicated-database, no-parallel --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tags: no-parallel DROP TABLE IF EXISTS alter_attach; CREATE TABLE alter_attach (x UInt64, p UInt8) ENGINE = MergeTree ORDER BY tuple() PARTITION BY p; diff --git a/tests/queries/0_stateless/00955_test_final_mark.sql b/tests/queries/0_stateless/00955_test_final_mark.sql index 3c92d96e9a9..6615c945ef8 100644 --- a/tests/queries/0_stateless/00955_test_final_mark.sql +++ b/tests/queries/0_stateless/00955_test_final_mark.sql @@ -1,5 +1,4 @@ --- Tags: no-replicated-database, no-parallel --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tags: no-parallel SET send_logs_level = 'fatal'; diff --git a/tests/queries/0_stateless/00955_test_final_mark_use.sh b/tests/queries/0_stateless/00955_test_final_mark_use.sh index 8642011b670..2c3219fbffd 100755 --- a/tests/queries/0_stateless/00955_test_final_mark_use.sh +++ b/tests/queries/0_stateless/00955_test_final_mark_use.sh @@ -1,6 +1,5 @@ #!/usr/bin/env bash -# Tags: no-replicated-database, no-parallel -# Tag no-replicated-database: Unsupported type of ALTER query +# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01015_attach_part.sql b/tests/queries/0_stateless/01015_attach_part.sql index 3d2058f757e..6b786bfbab9 100644 --- a/tests/queries/0_stateless/01015_attach_part.sql +++ b/tests/queries/0_stateless/01015_attach_part.sql @@ -1,5 +1,4 @@ --- Tags: no-replicated-database, no-parallel --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tags: no-parallel DROP TABLE IF EXISTS table_01; diff --git a/tests/queries/0_stateless/01021_only_tuple_columns.sql b/tests/queries/0_stateless/01021_only_tuple_columns.sql index a0be59c3120..02db21bc0b2 100644 --- a/tests/queries/0_stateless/01021_only_tuple_columns.sql +++ b/tests/queries/0_stateless/01021_only_tuple_columns.sql @@ -1,5 +1,4 @@ --- Tags: no-replicated-database, no-parallel --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tags: no-parallel CREATE TABLE test ( diff --git a/tests/queries/0_stateless/01060_shutdown_table_after_detach.sql b/tests/queries/0_stateless/01060_shutdown_table_after_detach.sql index 5e0ed8b298d..bfe928d7003 100644 --- a/tests/queries/0_stateless/01060_shutdown_table_after_detach.sql +++ b/tests/queries/0_stateless/01060_shutdown_table_after_detach.sql @@ -1,5 +1,4 @@ --- Tags: no-replicated-database, no-parallel --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tags: no-parallel DROP TABLE IF EXISTS test; CREATE TABLE test Engine = MergeTree ORDER BY number AS SELECT number, toString(rand()) x from numbers(10000000); diff --git a/tests/queries/0_stateless/01130_in_memory_parts_partitons.sql b/tests/queries/0_stateless/01130_in_memory_parts_partitons.sql index a0e8d24eff6..aa6f281e0eb 100644 --- a/tests/queries/0_stateless/01130_in_memory_parts_partitons.sql +++ b/tests/queries/0_stateless/01130_in_memory_parts_partitons.sql @@ -1,5 +1,4 @@ --- Tags: no-replicated-database, no-parallel --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tags: no-parallel DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh b/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh index 5b936b3f2ba..01678d1b500 100755 --- a/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh +++ b/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh @@ -1,6 +1,6 @@ #!/usr/bin/env bash # Tags: zookeeper, no-replicated-database, no-parallel -# Tag no-replicated-database: Unsupported type of ALTER query +# Tag no-replicated-database: Fails due to additional replicas or shards CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01451_detach_drop_part.sql b/tests/queries/0_stateless/01451_detach_drop_part.sql index 0bc7e13d243..a285730e45f 100644 --- a/tests/queries/0_stateless/01451_detach_drop_part.sql +++ b/tests/queries/0_stateless/01451_detach_drop_part.sql @@ -1,6 +1,3 @@ --- Tags: no-replicated-database --- Tag no-replicated-database: Unsupported type of ALTER query - DROP TABLE IF EXISTS mt_01451; CREATE TABLE mt_01451 (v UInt8) ENGINE = MergeTree() order by tuple(); diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql index eb4d5e43115..9f9d1db78f1 100644 --- a/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql @@ -1,5 +1,5 @@ -- Tags: long, replica, no-replicated-database --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tag no-replicated-database: Fails due to additional replicas or shards SET replication_alter_partitions_sync = 2; diff --git a/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql b/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql index f9b9181b5b7..ce48ad6a02a 100644 --- a/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql +++ b/tests/queries/0_stateless/01650_fetch_patition_with_macro_in_zk_path_long.sql @@ -1,5 +1,4 @@ --- Tags: long, no-replicated-database --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tags: long DROP TABLE IF EXISTS test_01640; DROP TABLE IF EXISTS restore_01640; diff --git a/tests/queries/0_stateless/02009_array_join_partition.sql b/tests/queries/0_stateless/02009_array_join_partition.sql index 1b99deec0f9..a78efe96f66 100644 --- a/tests/queries/0_stateless/02009_array_join_partition.sql +++ b/tests/queries/0_stateless/02009_array_join_partition.sql @@ -1,6 +1,3 @@ --- Tags: no-replicated-database --- Tag no-replicated-database: Unsupported type of ALTER query - CREATE TABLE table_2009_part (`i` Int64, `d` Date, `s` String) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY i; ALTER TABLE table_2009_part ATTACH PARTITION tuple(arrayJoin([0, 1])); -- {serverError 248} diff --git a/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.sql b/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.sql index 5f53984a8e4..50d5bfe0d39 100644 --- a/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.sql +++ b/tests/queries/0_stateless/02012_changed_enum_type_non_replicated.sql @@ -1,5 +1,4 @@ --- Tags: replica, no-replicated-database --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tags: replica create table enum_alter_issue (a Enum8('one' = 1, 'two' = 2)) engine = MergeTree() ORDER BY a; insert into enum_alter_issue values ('one'), ('two'); diff --git a/tests/queries/0_stateless/02012_zookeeper_changed_enum_type.sql b/tests/queries/0_stateless/02012_zookeeper_changed_enum_type.sql index 790db4c5c3d..5dc9ef8ca17 100644 --- a/tests/queries/0_stateless/02012_zookeeper_changed_enum_type.sql +++ b/tests/queries/0_stateless/02012_zookeeper_changed_enum_type.sql @@ -1,5 +1,5 @@ -- Tags: zookeeper, no-replicated-database --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tag no-replicated-database: Fails due to additional replicas or shards create table enum_alter_issue (a Enum8('one' = 1, 'two' = 2), b Int) engine = ReplicatedMergeTree('/clickhouse/tables/{database}/test_02012/enum_alter_issue', 'r1') diff --git a/tests/queries/0_stateless/02012_zookeeper_changed_enum_type_incompatible.sql b/tests/queries/0_stateless/02012_zookeeper_changed_enum_type_incompatible.sql index 697758be0c8..e236e6d2767 100644 --- a/tests/queries/0_stateless/02012_zookeeper_changed_enum_type_incompatible.sql +++ b/tests/queries/0_stateless/02012_zookeeper_changed_enum_type_incompatible.sql @@ -1,5 +1,5 @@ -- Tags: zookeeper, no-replicated-database --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tag no-replicated-database: Fails due to additional replicas or shards drop table if exists enum_alter_issue; create table enum_alter_issue (a Enum16('one' = 1, 'two' = 2), b Int) diff --git a/tests/queries/1_stateful/00054_merge_tree_partitions.sql b/tests/queries/1_stateful/00054_merge_tree_partitions.sql index 91e67ac857a..73020952977 100644 --- a/tests/queries/1_stateful/00054_merge_tree_partitions.sql +++ b/tests/queries/1_stateful/00054_merge_tree_partitions.sql @@ -1,6 +1,3 @@ --- Tags: no-replicated-database --- Tag no-replicated-database: Unsupported type of ALTER query - DROP TABLE IF EXISTS test.partitions; CREATE TABLE test.partitions (EventDate Date, CounterID UInt32) ENGINE = MergeTree(EventDate, CounterID, 8192); INSERT INTO test.partitions SELECT EventDate + UserID % 365 AS EventDate, CounterID FROM test.hits WHERE CounterID = 1704509; diff --git a/tests/queries/1_stateful/00152_insert_different_granularity.sql b/tests/queries/1_stateful/00152_insert_different_granularity.sql index 5c802642a5c..da99173ed96 100644 --- a/tests/queries/1_stateful/00152_insert_different_granularity.sql +++ b/tests/queries/1_stateful/00152_insert_different_granularity.sql @@ -1,5 +1,5 @@ -- Tags: no-tsan, no-replicated-database --- Tag no-replicated-database: Unsupported type of ALTER query +-- Tag no-replicated-database: Fails due to additional replicas or shards DROP TABLE IF EXISTS fixed_granularity_table; From 045d4f2d3278653b296397e78a49c238978024a9 Mon Sep 17 00:00:00 2001 From: Kevin Michel Date: Sun, 26 Sep 2021 18:47:24 +0200 Subject: [PATCH 064/919] Don't replicate DROP/DETACH/DROP DETACHED on Replicated database --- .../en/engines/database-engines/replicated.md | 2 +- src/Interpreters/InterpreterAlterQuery.cpp | 3 +- src/Parsers/ASTAlterQuery.cpp | 5 ++ src/Parsers/ASTAlterQuery.h | 2 + .../configs/settings.xml | 1 + .../test_replicated_database/test.py | 51 +++++++++++++++++++ 6 files changed, 62 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/database-engines/replicated.md b/docs/en/engines/database-engines/replicated.md index 67671e11cab..bdc17d32393 100644 --- a/docs/en/engines/database-engines/replicated.md +++ b/docs/en/engines/database-engines/replicated.md @@ -35,7 +35,7 @@ The [system.clusters](../../operations/system-tables/clusters.md) system table c When creating a new replica of the database, this replica creates tables by itself. If the replica has been unavailable for a long time and has lagged behind the replication log — it checks its local metadata with the current metadata in ZooKeeper, moves the extra tables with data to a separate non-replicated database (so as not to accidentally delete anything superfluous), creates the missing tables, updates the table names if they have been renamed. The data is replicated at the `ReplicatedMergeTree` level, i.e. if the table is not replicated, the data will not be replicated (the database is responsible only for metadata). -[`ALTER TABLE FETCH`](../../sql-reference/statements/alter/partition.md) and [`ALTER TABLE ATTACH`](../../sql-reference/statements/alter/partition.md) queries are allowed but not replicated. The database engine will only add the partition/part to the current replica. However, if the table itself uses a Replicated table engine, then the data will be replicated after using `ATTACH`. +[`ALTER TABLE ATTACH|FETCH|DROP|DROP DETACHED|DETACH PARTITION|PART`](../../sql-reference/statements/alter/partition.md) queries are allowed but not replicated. The database engine will only add/fetch/remove the partition/part to the current replica. However, if the table itself uses a Replicated table engine, then the data will be replicated after using `ATTACH`. ## Usage Example {#usage-example} diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index e2b2d487be5..a1d083e298d 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -57,7 +57,8 @@ BlockIO InterpreterAlterQuery::execute() if (typeid_cast(database.get()) && !getContext()->getClientInfo().is_replicated_database_internal && !alter.isAttachAlter() - && !alter.isFetchAlter()) + && !alter.isFetchAlter() + && !alter.isDropPartitionAlter()) { auto guard = DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name); guard->releaseTableLock(); diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 140ef972eeb..f3f83258ac8 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -460,6 +460,11 @@ bool ASTAlterQuery::isFetchAlter() const return isOneCommandTypeOnly(ASTAlterCommand::FETCH_PARTITION); } +bool ASTAlterQuery::isDropPartitionAlter() const +{ + return isOneCommandTypeOnly(ASTAlterCommand::DROP_PARTITION) || isOneCommandTypeOnly(ASTAlterCommand::DROP_DETACHED_PARTITION); +} + /** Get the text that identifies this element. */ String ASTAlterQuery::getID(char delim) const diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 432f0c887fd..2f53026c8d8 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -222,6 +222,8 @@ public: bool isFetchAlter() const; + bool isDropPartitionAlter() const; + String getID(char) const override; ASTPtr clone() const override; diff --git a/tests/integration/test_replicated_database/configs/settings.xml b/tests/integration/test_replicated_database/configs/settings.xml index 7f45502e20d..83e83a47c0b 100644 --- a/tests/integration/test_replicated_database/configs/settings.xml +++ b/tests/integration/test_replicated_database/configs/settings.xml @@ -1,6 +1,7 @@ + 1 1 1 diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index ff400f4466b..fa1e9cf49fa 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -136,6 +136,57 @@ def test_alter_attach(started_cluster, attachable_part, engine): assert dummy_node.query(f"SELECT CounterID FROM testdb.{name}") == "" +@pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) +def test_alter_drop_part(started_cluster, engine): + table = f"alter_drop_{engine}" + part_name = "all_0_0_0" if engine == "ReplicatedMergeTree" else "all_1_1_0" + main_node.query(f"CREATE TABLE testdb.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)") + main_node.query(f"INSERT INTO testdb.{table} VALUES (123)") + if engine == "MergeTree": + dummy_node.query(f"INSERT INTO testdb.{table} VALUES (456)") + main_node.query(f"ALTER TABLE testdb.{table} DROP PART '{part_name}'") + assert main_node.query(f"SELECT CounterID FROM testdb.{table}") == "" + if engine == "ReplicatedMergeTree": + # The DROP operation is still replicated at the table engine level + assert dummy_node.query(f"SELECT CounterID FROM testdb.{table}") == "" + else: + assert dummy_node.query(f"SELECT CounterID FROM testdb.{table}") == "456\n" + + +@pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) +def test_alter_detach_part(started_cluster, engine): + table = f"alter_detach_{engine}" + part_name = "all_0_0_0" if engine == "ReplicatedMergeTree" else "all_1_1_0" + main_node.query(f"CREATE TABLE testdb.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)") + main_node.query(f"INSERT INTO testdb.{table} VALUES (123)") + if engine == "MergeTree": + dummy_node.query(f"INSERT INTO testdb.{table} VALUES (456)") + main_node.query(f"ALTER TABLE testdb.{table} DETACH PART '{part_name}'") + detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='testdb' AND table='{table}'" + assert main_node.query(detached_parts_query) == f"{part_name}\n" + if engine == "ReplicatedMergeTree": + # The detach operation is still replicated at the table engine level + assert dummy_node.query(detached_parts_query) == f"{part_name}\n" + else: + assert dummy_node.query(detached_parts_query) == "" + + +@pytest.mark.parametrize("engine", ["MergeTree", "ReplicatedMergeTree"]) +def test_alter_drop_detached_part(started_cluster, engine): + table = f"alter_drop_detached_{engine}" + part_name = "all_0_0_0" if engine == "ReplicatedMergeTree" else "all_1_1_0" + main_node.query(f"CREATE TABLE testdb.{table} (CounterID UInt32) ENGINE = {engine} ORDER BY (CounterID)") + main_node.query(f"INSERT INTO testdb.{table} VALUES (123)") + main_node.query(f"ALTER TABLE testdb.{table} DETACH PART '{part_name}'") + if engine == "MergeTree": + dummy_node.query(f"INSERT INTO testdb.{table} VALUES (456)") + dummy_node.query(f"ALTER TABLE testdb.{table} DETACH PART '{part_name}'") + main_node.query(f"ALTER TABLE testdb.{table} DROP DETACHED PART '{part_name}'") + detached_parts_query = f"SELECT name FROM system.detached_parts WHERE database='testdb' AND table='{table}'" + assert main_node.query(detached_parts_query) == "" + assert dummy_node.query(detached_parts_query) == f"{part_name}\n" + + def test_alter_fetch(started_cluster): main_node.query("CREATE TABLE testdb.fetch_source (CounterID UInt32) ENGINE = ReplicatedMergeTree ORDER BY (CounterID)") main_node.query("CREATE TABLE testdb.fetch_target (CounterID UInt32) ENGINE = ReplicatedMergeTree ORDER BY (CounterID)") From 30428ad1b068ed170c2cf6d14f3a981bf582977f Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 27 Sep 2021 08:44:48 +0000 Subject: [PATCH 065/919] fix --- src/Storages/FileLog/ReadBufferFromFileLog.h | 4 ++-- src/Storages/FileLog/StorageFileLog.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index 47a2770550d..8a3fdf235d7 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -78,8 +78,8 @@ private: Records records; Records::const_iterator current; - String current_file; - UInt64 current_offset; + String current_file = {}; + UInt64 current_offset = 0; using TaskThread = BackgroundSchedulePool::TaskHolder; diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 58e4dc9d6eb..ec81558438c 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -66,7 +66,7 @@ public: { String file_name; UInt64 last_writen_position = 0; - UInt64 last_open_end; + UInt64 last_open_end = 0; }; using FileNameToInode = std::unordered_map; From 814f6d5068af5504366e128686900c7a8864e244 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 27 Sep 2021 12:09:01 +0000 Subject: [PATCH 066/919] fix --- src/Storages/FileLog/ReadBufferFromFileLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index bfcfb3a2385..e39fd84acfe 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -23,7 +23,7 @@ ReadBufferFromFileLog::ReadBufferFromFileLog( size_t stream_number_, size_t max_streams_number_) : ReadBuffer(nullptr, 0) - , log(&Poco::Logger::get("ReadBufferFromFileLog " + toString(stream_number))) + , log(&Poco::Logger::get("ReadBufferFromFileLog " + toString(stream_number_))) , storage(storage_) , batch_size(max_batch_size) , poll_timeout(poll_timeout_) From ebca30694dee1244ccb133e05b76e3fc7dd5abd1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 27 Sep 2021 12:41:18 +0300 Subject: [PATCH 067/919] Fix --- ...chronousReadIndirectBufferFromRemoteFS.cpp | 36 +++++++------------ ...ynchronousReadIndirectBufferFromRemoteFS.h | 8 +++-- src/Disks/IDiskRemote.cpp | 4 ++- src/Disks/ReadIndirectBufferFromRemoteFS.cpp | 21 +++++------ src/Disks/ReadIndirectBufferFromRemoteFS.h | 8 +++-- src/IO/ReadBufferFromRemoteFS.cpp | 32 ++++++++++++----- src/IO/ReadBufferFromRemoteFS.h | 17 ++++----- src/IO/ReadBufferFromS3.cpp | 1 - src/IO/ThreadPoolRemoteFSReader.cpp | 11 +++--- src/IO/ThreadPoolRemoteFSReader.h | 18 +++++++--- 10 files changed, 87 insertions(+), 69 deletions(-) diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp index c754bfc004c..e651608d749 100644 --- a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -26,7 +26,7 @@ namespace ErrorCodes AsynchronousReadIndirectBufferFromRemoteFS::AsynchronousReadIndirectBufferFromRemoteFS( - AsynchronousReaderPtr reader_, Int32 priority_, ReadBufferFromRemoteFSImpl impl_) + AsynchronousReaderPtr reader_, Int32 priority_, std::shared_ptr impl_) : reader(reader_), priority(priority_), impl(impl_) { } @@ -35,16 +35,9 @@ AsynchronousReadIndirectBufferFromRemoteFS::AsynchronousReadIndirectBufferFromRe std::future AsynchronousReadIndirectBufferFromRemoteFS::readNext() { IAsynchronousReader::Request request; - - auto remote_fd = std::make_shared(); - - /// Resize buffer to 0 and move pos to start. - impl->set(impl->buffer().begin(), impl->buffer().size()); - remote_fd->impl = impl; - - request.descriptor = std::move(remote_fd); + request.descriptor = std::make_shared(impl); + request.offset = absolute_position; request.priority = priority; - return reader->submit(request); } @@ -77,11 +70,9 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() if (size) { - size_t offset = pos - working_buffer.begin(); - assert(offset >= 0); swap(*impl); - impl->absolute_position += working_buffer.size(); - position() = working_buffer.begin() + offset; + absolute_position += size; + impl->reset(); } prefetch_future = {}; @@ -94,24 +85,24 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence if (whence == SEEK_CUR) { /// If position within current working buffer - shift pos. - if (!working_buffer.empty() && static_cast(getPosition() + offset_) < impl->absolute_position) + if (!working_buffer.empty() && static_cast(getPosition() + offset_) < absolute_position) { pos += offset_; return getPosition(); } else { - impl->absolute_position += offset_; + absolute_position += offset_; } } else if (whence == SEEK_SET) { /// If position is within current working buffer - shift pos. if (!working_buffer.empty() - && static_cast(offset_) >= impl->absolute_position - working_buffer.size() - && size_t(offset_) < impl->absolute_position) + && static_cast(offset_) >= absolute_position - working_buffer.size() + && size_t(offset_) < absolute_position) { - pos = working_buffer.end() - (impl->absolute_position - offset_); + pos = working_buffer.end() - (absolute_position - offset_); assert(pos >= working_buffer.begin()); assert(pos <= working_buffer.end()); @@ -120,7 +111,7 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence } else { - impl->absolute_position = offset_; + absolute_position = offset_; } } else @@ -130,14 +121,13 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence { std::cerr << "Ignoring prefetched data" << "\n"; prefetch_future.wait(); - impl->reset(); /// Clean the buffer, we do no need it. prefetch_future = {}; } pos = working_buffer.end(); - impl->reset(); + impl->reset(true); - return impl->absolute_position; + return absolute_position; } diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h index 8ce9292e487..c11c8d12735 100644 --- a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -19,13 +19,13 @@ class AsynchronousReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase { public: explicit AsynchronousReadIndirectBufferFromRemoteFS( - AsynchronousReaderPtr reader_, Int32 priority_, ReadBufferFromRemoteFSImpl impl_); + AsynchronousReaderPtr reader_, Int32 priority_, std::shared_ptr impl_); ~AsynchronousReadIndirectBufferFromRemoteFS() override; off_t seek(off_t offset_, int whence) override; - off_t getPosition() override { return impl->absolute_position - available(); } + off_t getPosition() override { return absolute_position - available(); } String getFileName() const override { return impl->getFileName(); } @@ -40,8 +40,10 @@ private: AsynchronousReaderPtr reader; Int32 priority; - ReadBufferFromRemoteFSImpl impl; + std::shared_ptr impl; std::future prefetch_future; + + size_t absolute_position = 0; }; } diff --git a/src/Disks/IDiskRemote.cpp b/src/Disks/IDiskRemote.cpp index d09fe1e8880..23b49c1f32d 100644 --- a/src/Disks/IDiskRemote.cpp +++ b/src/Disks/IDiskRemote.cpp @@ -500,7 +500,9 @@ String IDiskRemote::getUniqueId(const String & path) const AsynchronousReaderPtr IDiskRemote::getThreadPoolReader() { - static AsynchronousReaderPtr reader = std::make_shared(16, 1000000); + constexpr size_t pool_size = 16; + constexpr size_t queue_size = 1000000; + static AsynchronousReaderPtr reader = std::make_shared(pool_size, queue_size); return reader; } diff --git a/src/Disks/ReadIndirectBufferFromRemoteFS.cpp b/src/Disks/ReadIndirectBufferFromRemoteFS.cpp index f03c84952db..5ffa744c93d 100644 --- a/src/Disks/ReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/ReadIndirectBufferFromRemoteFS.cpp @@ -10,7 +10,8 @@ namespace ErrorCodes } -ReadIndirectBufferFromRemoteFS::ReadIndirectBufferFromRemoteFS(ImplPtr impl_) : impl(std::move(impl_)) +ReadIndirectBufferFromRemoteFS::ReadIndirectBufferFromRemoteFS( + std::shared_ptr impl_) : impl(std::move(impl_)) { } @@ -20,38 +21,38 @@ off_t ReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence) if (whence == SEEK_CUR) { /// If position within current working buffer - shift pos. - if (!working_buffer.empty() && size_t(getPosition() + offset_) < impl->absolute_position) + if (!working_buffer.empty() && size_t(getPosition() + offset_) < absolute_position) { pos += offset_; return getPosition(); } else { - impl->absolute_position += offset_; + absolute_position += offset_; } } else if (whence == SEEK_SET) { /// If position within current working buffer - shift pos. if (!working_buffer.empty() - && size_t(offset_) >= impl->absolute_position - working_buffer.size() - && size_t(offset_) < impl->absolute_position) + && size_t(offset_) >= absolute_position - working_buffer.size() + && size_t(offset_) < absolute_position) { - pos = working_buffer.end() - (impl->absolute_position - offset_); + pos = working_buffer.end() - (absolute_position - offset_); return getPosition(); } else { - impl->absolute_position = offset_; + absolute_position = offset_; } } else throw Exception("Only SEEK_SET or SEEK_CUR modes are allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); - impl->seek(impl->absolute_position, SEEK_SET); + impl->seek(absolute_position, SEEK_SET); pos = working_buffer.end(); - return impl->absolute_position; + return absolute_position; } @@ -65,7 +66,7 @@ bool ReadIndirectBufferFromRemoteFS::nextImpl() swap(*impl); if (result) - impl->absolute_position += working_buffer.size(); + absolute_position += working_buffer.size(); return result; } diff --git a/src/Disks/ReadIndirectBufferFromRemoteFS.h b/src/Disks/ReadIndirectBufferFromRemoteFS.h index 48af28f67f9..4e8027e7319 100644 --- a/src/Disks/ReadIndirectBufferFromRemoteFS.h +++ b/src/Disks/ReadIndirectBufferFromRemoteFS.h @@ -19,18 +19,20 @@ class ReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase using ImplPtr = std::unique_ptr; public: - explicit ReadIndirectBufferFromRemoteFS(ImplPtr impl_); + explicit ReadIndirectBufferFromRemoteFS(std::shared_ptr impl_); off_t seek(off_t offset_, int whence) override; - off_t getPosition() override { return impl->absolute_position - available(); } + off_t getPosition() override { return absolute_position - available(); } String getFileName() const override { return impl->getFileName(); } private: bool nextImpl() override; - ImplPtr impl; + std::shared_ptr impl; + + size_t absolute_position = 0; }; } diff --git a/src/IO/ReadBufferFromRemoteFS.cpp b/src/IO/ReadBufferFromRemoteFS.cpp index b52104790af..a8d2bba00b2 100644 --- a/src/IO/ReadBufferFromRemoteFS.cpp +++ b/src/IO/ReadBufferFromRemoteFS.cpp @@ -50,7 +50,7 @@ bool ReadBufferFromRemoteFS::nextImpl() /// If current buffer has remaining data - use it. if (current_buf) { - if (readImpl()) + if (read()) return true; } @@ -63,11 +63,11 @@ bool ReadBufferFromRemoteFS::nextImpl() const auto & path = metadata.remote_fs_objects[current_buf_idx].first; current_buf = createReadBuffer(path); - return readImpl(); + return read(); } -bool ReadBufferFromRemoteFS::readImpl() +bool ReadBufferFromRemoteFS::read() { /// Transfer current position and working_buffer to actual ReadBuffer swap(*current_buf); @@ -75,26 +75,40 @@ bool ReadBufferFromRemoteFS::readImpl() auto result = current_buf->next(); /// Assign result to current buffer. swap(*current_buf); - /// Absolute position is updated by *IndirectBufferFromRemoteFS only. + + if (result) + absolute_position += working_buffer.size(); return result; } -off_t ReadBufferFromRemoteFS::seek([[maybe_unused]] off_t offset_, int whence) +size_t ReadBufferFromRemoteFS::fetch(size_t offset) +{ + absolute_position = offset; + auto result = nextImpl(); + if (result) + return working_buffer.size(); + return 0; +} + + +off_t ReadBufferFromRemoteFS::seek(off_t offset, int whence) { if (whence != SEEK_SET) throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only SEEK_SET is allowed"); - // current_buf = initialize(); + absolute_position = offset; + current_buf = initialize(); return absolute_position; } -void ReadBufferFromRemoteFS::reset() +void ReadBufferFromRemoteFS::reset(bool reset_inner_buf) { - current_buf.reset(); - // set(nullptr, 0); + if (reset_inner_buf) + current_buf.reset(); + set(working_buffer.begin(), working_buffer.size()); } } diff --git a/src/IO/ReadBufferFromRemoteFS.h b/src/IO/ReadBufferFromRemoteFS.h index 842755945a8..559a6550cfc 100644 --- a/src/IO/ReadBufferFromRemoteFS.h +++ b/src/IO/ReadBufferFromRemoteFS.h @@ -9,8 +9,7 @@ namespace DB class ReadBufferFromRemoteFS : public ReadBufferFromFileBase { -friend class ReadIndirectBufferFromRemoteFS; -friend class AsynchronousReadIndirectBufferFromRemoteFS; +friend class ThreadPoolRemoteFSReader; public: explicit ReadBufferFromRemoteFS(const RemoteMetadata & metadata_); @@ -21,14 +20,13 @@ public: String getFileName() const override { return metadata.metadata_file_path; } - bool readNext() { return nextImpl(); } - - void reset(); - - bool initialized() const { return current_buf != nullptr; } + void reset(bool reset_inner_buf = false); protected: + size_t fetch(size_t offset); + virtual SeekableReadBufferPtr createReadBuffer(const String & path) const = 0; + RemoteMetadata metadata; private: @@ -36,14 +34,13 @@ private: SeekableReadBufferPtr initialize(); - bool readImpl(); + bool read(); SeekableReadBufferPtr current_buf; size_t current_buf_idx = 0; + size_t absolute_position = 0; }; -using ReadBufferFromRemoteFSImpl = std::shared_ptr; - } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index cc00635b5bd..a482b124fee 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -103,7 +103,6 @@ bool ReadBufferFromS3::nextImpl() } } - std::cerr << "s3 buffer size: " << impl->buffer().size() << std::endl; if (!next_result) return false; diff --git a/src/IO/ThreadPoolRemoteFSReader.cpp b/src/IO/ThreadPoolRemoteFSReader.cpp index 6e4a66f2205..726e7b1dade 100644 --- a/src/IO/ThreadPoolRemoteFSReader.cpp +++ b/src/IO/ThreadPoolRemoteFSReader.cpp @@ -28,6 +28,11 @@ namespace CurrentMetrics namespace DB { +size_t ThreadPoolRemoteFSReader::RemoteFSFileDescriptor::fetch(size_t offset) +{ + return reader->fetch(offset); +} + ThreadPoolRemoteFSReader::ThreadPoolRemoteFSReader(size_t pool_size, size_t queue_size_) : pool(pool_size, pool_size, queue_size_) @@ -42,13 +47,9 @@ std::future ThreadPoolRemoteFSReader::submit(Reques setThreadName("ThreadPoolRead"); CurrentMetrics::Increment metric_increment{CurrentMetrics::Read}; auto * remote_fs_fd = assert_cast(request.descriptor.get()); - auto * remote_fs_buf = dynamic_cast(remote_fs_fd->impl.get()); Stopwatch watch(CLOCK_MONOTONIC); - size_t bytes_read = 0; - auto result = remote_fs_buf->readNext(); - if (result) - bytes_read = remote_fs_buf->buffer().size(); + auto bytes_read = remote_fs_fd->fetch(request.offset); watch.stop(); ProfileEvents::increment(ProfileEvents::RemoteFSReadMicroseconds, watch.elapsedMicroseconds()); diff --git a/src/IO/ThreadPoolRemoteFSReader.h b/src/IO/ThreadPoolRemoteFSReader.h index cb2c4240a20..b82c17be47c 100644 --- a/src/IO/ThreadPoolRemoteFSReader.h +++ b/src/IO/ThreadPoolRemoteFSReader.h @@ -9,6 +9,8 @@ namespace DB { +class ReadBufferFromRemoteFS; + class ThreadPoolRemoteFSReader : public IAsynchronousReader { @@ -20,11 +22,19 @@ public: std::future submit(Request request) override; - struct RemoteFSFileDescriptor : public IFileDescriptor - { - ReadBufferPtr impl; - }; + struct RemoteFSFileDescriptor; +}; + +struct ThreadPoolRemoteFSReader::RemoteFSFileDescriptor : public IFileDescriptor +{ +public: + RemoteFSFileDescriptor(std::shared_ptr reader_) : reader(reader_) {} + + size_t fetch(size_t offset); + +private: + std::shared_ptr reader; }; } From df96b3e65695285760adfadaf4bf60fd8d812324 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 28 Sep 2021 02:06:16 +0000 Subject: [PATCH 068/919] fix test --- tests/queries/0_stateless/02022_storage_filelog_one_file.sh | 2 +- tests/queries/0_stateless/02023_storage_filelog.sh | 2 +- tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh index fa36775a731..2cf23957c9e 100755 --- a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: long, no-parallel set -eu diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 7e843075181..095c0f05c40 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: long, no-parallel set -eu diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index c5e1553c41a..076e2af44c2 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: long, no-parallel set -eu From 1f90c8dc220dd1cfe869f8f6175458a31bbe35a5 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 28 Sep 2021 05:20:02 +0000 Subject: [PATCH 069/919] refactor fix style --- .../FileLog/FileLogDirectoryWatcher.cpp | 10 +++---- src/Storages/FileLog/StorageFileLog.cpp | 27 ++++++------------- 2 files changed, 13 insertions(+), 24 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index b27b4719933..5bec4fff650 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -43,7 +43,7 @@ void FileLogDirectoryWatcher::onItemAdded(const Poco::DirectoryWatcher::Director de.callback = "onItemAdded"; de.path = ev.item.path(); de.type = ev.event; - events.emplace_back(de); + events.push_back(de); } @@ -54,7 +54,7 @@ void FileLogDirectoryWatcher::onItemRemoved(const Poco::DirectoryWatcher::Direct de.callback = "onItemRemoved"; de.path = ev.item.path(); de.type = ev.event; - events.emplace_back(de); + events.push_back(de); } @@ -65,7 +65,7 @@ void FileLogDirectoryWatcher::onItemModified(const Poco::DirectoryWatcher::Direc de.callback = "onItemModified"; de.path = ev.item.path(); de.type = ev.event; - events.emplace_back(de); + events.push_back(de); } void FileLogDirectoryWatcher::onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent& ev) @@ -75,7 +75,7 @@ void FileLogDirectoryWatcher::onItemMovedFrom(const Poco::DirectoryWatcher::Dire de.callback = "onItemMovedFrom"; de.path = ev.item.path(); de.type = ev.event; - events.emplace_back(de); + events.push_back(de); } void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent& ev) @@ -85,7 +85,7 @@ void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::Direct de.callback = "onItemMovedTo"; de.path = ev.item.path(); de.type = ev.event; - events.emplace_back(de); + events.push_back(de); } void FileLogDirectoryWatcher::onError(const Poco::Exception & e) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 2acff24059c..d5d86eec0b8 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include @@ -528,7 +528,7 @@ bool StorageFileLog::streamToViews() auto convert_actions_dag = ActionsDAG::makeConvertingActions( pipe.getHeader().getColumnsWithTypeAndName(), - block_io.out->getHeader().getColumnsWithTypeAndName(), + block_io.pipeline.getHeader().getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); auto actions = std::make_shared( @@ -538,28 +538,17 @@ bool StorageFileLog::streamToViews() pipes.emplace_back(std::move(pipe)); } - QueryPipeline pipeline; - pipeline.init(Pipe::unitePipes(std::move(pipes))); + auto input= Pipe::unitePipes(std::move(pipes)); - assertBlocksHaveEqualStructure(pipeline.getHeader(), block_io.out->getHeader(), "StorageFileLog streamToViews"); + assertBlocksHaveEqualStructure(input.getHeader(), block_io.pipeline.getHeader(), "StorageFileLog streamToViews"); - size_t rows = 0; + block_io.pipeline.complete(std::move(input)); - PullingPipelineExecutor executor(pipeline); - Block block; - block_io.out->writePrefix(); - while (executor.pull(block)) - { - block_io.out->write(block); - rows += block.rows(); - /// During files open, also save file end at the opening moment - serialize(true); - } - block_io.out->writeSuffix(); + CompletedPipelineExecutor executor(block_io.pipeline); + executor.execute(); UInt64 milliseconds = watch.elapsedMilliseconds(); - LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", - formatReadableQuantity(rows), table_id.getNameForLogs(), milliseconds); + LOG_DEBUG(log, "Pushing data to {} took {} ms.", table_id.getNameForLogs(), milliseconds); return updateFileInfos(); } From 20192352adeb572142362a62ffb02b0c7ac51252 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 29 Sep 2021 09:35:03 +0300 Subject: [PATCH 070/919] Better --- ...chronousReadIndirectBufferFromRemoteFS.cpp | 27 ++++++++++++------- ...ynchronousReadIndirectBufferFromRemoteFS.h | 1 + src/IO/ReadBufferFromRemoteFS.cpp | 2 +- 3 files changed, 20 insertions(+), 10 deletions(-) diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp index e651608d749..371c1327433 100644 --- a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -47,6 +47,7 @@ void AsynchronousReadIndirectBufferFromRemoteFS::prefetch() if (prefetch_future.valid()) return; + std::lock_guard lock(mutex); prefetch_future = readNext(); } @@ -59,20 +60,29 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() { CurrentMetrics::Increment metric_increment{CurrentMetrics::AsynchronousReadWait}; Stopwatch watch; - size = prefetch_future.get(); + { + std::lock_guard lock(mutex); + size = prefetch_future.get(); + if (size) + { + swap(*impl); + impl->reset(); + absolute_position += size; + } + } watch.stop(); ProfileEvents::increment(ProfileEvents::AsynchronousReadWaitMicroseconds, watch.elapsedMicroseconds()); } else { + std::lock_guard lock(mutex); size = readNext().get(); - } - - if (size) - { - swap(*impl); - absolute_position += size; - impl->reset(); + if (size) + { + swap(*impl); + impl->reset(); + absolute_position += size; + } } prefetch_future = {}; @@ -119,7 +129,6 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence if (prefetch_future.valid()) { - std::cerr << "Ignoring prefetched data" << "\n"; prefetch_future.wait(); prefetch_future = {}; } diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h index c11c8d12735..d108aa387b4 100644 --- a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -44,6 +44,7 @@ private: std::future prefetch_future; size_t absolute_position = 0; + std::mutex mutex; }; } diff --git a/src/IO/ReadBufferFromRemoteFS.cpp b/src/IO/ReadBufferFromRemoteFS.cpp index a8d2bba00b2..dfe1502ce77 100644 --- a/src/IO/ReadBufferFromRemoteFS.cpp +++ b/src/IO/ReadBufferFromRemoteFS.cpp @@ -108,7 +108,7 @@ void ReadBufferFromRemoteFS::reset(bool reset_inner_buf) { if (reset_inner_buf) current_buf.reset(); - set(working_buffer.begin(), working_buffer.size()); + BufferBase::set(nullptr, 0, 0); } } From 72473e26e2bda8ef019bfdfe95b68e80ba6c9e86 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Wed, 29 Sep 2021 21:08:20 +0300 Subject: [PATCH 071/919] update example --- docs/en/operations/system-tables/replicas.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/replicas.md b/docs/en/operations/system-tables/replicas.md index 251c7e65537..7bca3e78ff9 100644 --- a/docs/en/operations/system-tables/replicas.md +++ b/docs/en/operations/system-tables/replicas.md @@ -45,6 +45,9 @@ last_queue_update: 2020-02-20 08:34:32 absolute_delay: 0 total_replicas: 2 active_replicas: 2 +last_queue_update_exception: +zookeeper_exception: +replica_is_active: {'example01-06-1.yandex.ru':1} ``` Columns: @@ -82,8 +85,9 @@ The next 4 columns have a non-zero value only where there is an active session w - `absolute_delay` (`UInt64`) - How big lag in seconds the current replica has. - `total_replicas` (`UInt8`) - The total number of known replicas of this table. - `active_replicas` (`UInt8`) - The number of replicas of this table that have a session in ZooKeeper (i.e., the number of functioning replicas). -- `replica_is_active` ([Map(String, UInt8)](../../sql-reference/data-types/map.md)) — Map between replica name and is replica active. - `last_queue_update_exception` (`String`) - When the queue contains broken entries. Especially important when ClickHouse breaks backward compatibility between versions and log entries written by newer versions aren't parseable by old versions. +- `zookeeper_exception` (`String`) - The last exception message, got if the error happened when fetching the info from ZooKeeper. +- `replica_is_active` ([Map(String, UInt8)](../../sql-reference/data-types/map.md)) — Map between replica name and is replica active. If you request all the columns, the table may work a bit slowly, since several reads from ZooKeeper are made for each row. If you do not request the last 4 columns (log_max_index, log_pointer, total_replicas, active_replicas), the table works quickly. From 85aae03a26423e6f8ff108c1f7ab208f3647e8df Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 29 Sep 2021 22:54:49 +0300 Subject: [PATCH 072/919] Fix --- src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp | 6 ++++-- src/Disks/ReadIndirectBufferFromWebServer.cpp | 7 ++++--- src/Disks/ReadIndirectBufferFromWebServer.h | 3 ++- src/IO/ReadBufferFromRemoteFS.cpp | 5 ++++- 4 files changed, 14 insertions(+), 7 deletions(-) diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp index 371c1327433..62a8d816782 100644 --- a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -65,7 +65,8 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() size = prefetch_future.get(); if (size) { - swap(*impl); + set(impl->buffer().begin(), impl->buffer().size()); + working_buffer.resize(size); impl->reset(); absolute_position += size; } @@ -79,7 +80,8 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() size = readNext().get(); if (size) { - swap(*impl); + set(impl->buffer().begin(), impl->buffer().size()); + working_buffer.resize(size); impl->reset(); absolute_position += size; } diff --git a/src/Disks/ReadIndirectBufferFromWebServer.cpp b/src/Disks/ReadIndirectBufferFromWebServer.cpp index c0d41bb006a..17fa4d7d35c 100644 --- a/src/Disks/ReadIndirectBufferFromWebServer.cpp +++ b/src/Disks/ReadIndirectBufferFromWebServer.cpp @@ -60,8 +60,7 @@ std::unique_ptr ReadIndirectBufferFromWebServer::initialize() http_keep_alive_timeout), 0, Poco::Net::HTTPBasicCredentials{}, - buf_size, - headers); + buf_size, headers, context->getRemoteHostFilter(), use_external_buffer); } @@ -74,7 +73,9 @@ bool ReadIndirectBufferFromWebServer::nextImpl() { if (use_external_buffer) { - impl->set(working_buffer.begin(), working_buffer.size()); + impl.reset(); + // impl->set(working_buffer.begin(), working_buffer.size()); + // impl->BufferBase::set(nullptr, 0, 0); } else { diff --git a/src/Disks/ReadIndirectBufferFromWebServer.h b/src/Disks/ReadIndirectBufferFromWebServer.h index 55885427887..95b86f7db5e 100644 --- a/src/Disks/ReadIndirectBufferFromWebServer.h +++ b/src/Disks/ReadIndirectBufferFromWebServer.h @@ -19,7 +19,8 @@ public: explicit ReadIndirectBufferFromWebServer( const String & url_, ContextPtr context_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE, - size_t backoff_threshold_ = 10000, size_t max_tries_ = 4, bool use_external_buffer_ = false); + size_t backoff_threshold_ = 10000, size_t max_tries_ = 4, + bool use_external_buffer_ = false); bool nextImpl() override; diff --git a/src/IO/ReadBufferFromRemoteFS.cpp b/src/IO/ReadBufferFromRemoteFS.cpp index dfe1502ce77..5af34447bc5 100644 --- a/src/IO/ReadBufferFromRemoteFS.cpp +++ b/src/IO/ReadBufferFromRemoteFS.cpp @@ -15,13 +15,14 @@ namespace ErrorCodes ReadBufferFromRemoteFS::ReadBufferFromRemoteFS(const RemoteMetadata & metadata_) - : metadata(std::move(metadata_)) + : metadata(metadata_) { } SeekableReadBufferPtr ReadBufferFromRemoteFS::initialize() { + /// One clickhouse file can be split into multiple files in remote fs. auto current_buf_offset = absolute_position; for (size_t i = 0; i < metadata.remote_fs_objects.size(); ++i) { @@ -53,6 +54,8 @@ bool ReadBufferFromRemoteFS::nextImpl() if (read()) return true; } + else + return false; /// If there is no available buffers - nothing to read. if (current_buf_idx + 1 >= metadata.remote_fs_objects.size()) From cc9ce13bac2ba8c48b6a37478fa29556a8264105 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 30 Sep 2021 15:35:59 +0300 Subject: [PATCH 073/919] Finally all fixed --- ...chronousReadIndirectBufferFromRemoteFS.cpp | 17 +++++++++---- ...ynchronousReadIndirectBufferFromRemoteFS.h | 15 +++++++++++- src/IO/ReadBufferFromRemoteFS.cpp | 9 ++++++- src/IO/ReadBufferFromRemoteFS.h | 1 + src/IO/ReadBufferFromS3.cpp | 24 ++++++++++++++----- 5 files changed, 53 insertions(+), 13 deletions(-) diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp index 62a8d816782..fe433c76d9b 100644 --- a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -27,7 +27,8 @@ namespace ErrorCodes AsynchronousReadIndirectBufferFromRemoteFS::AsynchronousReadIndirectBufferFromRemoteFS( AsynchronousReaderPtr reader_, Int32 priority_, std::shared_ptr impl_) - : reader(reader_), priority(priority_), impl(impl_) + : ReadBufferFromFileBase(DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0) + , reader(reader_), priority(priority_), impl(impl_), prefetch_buffer(DBMS_DEFAULT_BUFFER_SIZE) { } @@ -48,6 +49,9 @@ void AsynchronousReadIndirectBufferFromRemoteFS::prefetch() return; std::lock_guard lock(mutex); + assert(prefetch_buffer.data() != nullptr); + prefetch_buffer.resize(DBMS_DEFAULT_BUFFER_SIZE); + impl->set(prefetch_buffer.data(), prefetch_buffer.size()); prefetch_future = readNext(); } @@ -65,9 +69,9 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() size = prefetch_future.get(); if (size) { - set(impl->buffer().begin(), impl->buffer().size()); + memory.swap(prefetch_buffer); + set(memory.data(), memory.size()); working_buffer.resize(size); - impl->reset(); absolute_position += size; } } @@ -77,12 +81,15 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() else { std::lock_guard lock(mutex); + impl->check = true; + impl->set(memory.data(), memory.size()); + assert(memory.data() != nullptr); + assert(impl->buffer().begin() != nullptr); size = readNext().get(); if (size) { - set(impl->buffer().begin(), impl->buffer().size()); + set(memory.data(), memory.size()); working_buffer.resize(size); - impl->reset(); absolute_position += size; } } diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h index d108aa387b4..0dd59d98864 100644 --- a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -14,7 +14,17 @@ namespace DB { -/// Reads data from S3/HDFS/Web using stored paths in metadata. +/** +* Reads data from S3/HDFS/Web using stored paths in metadata. +* +* Buffers chain for diskS3: +* AsynchronousIndirectReadBufferFromRemoteFS -> ReadBufferFromRemoteFS -> +* -> ReadBufferFromS3 -> ReadBufferFromIStream. +* +* Buffers chain for diskWeb: +* AsynchronousIndirectReadBufferFromRemoteFS -> ReadBufferFromRemoteFS -> +* -> ReadIndirectBufferFromWebServer -> ReadBufferFromHttp -> ReadBufferFromIStream. +*/ class AsynchronousReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase { public: @@ -31,6 +41,8 @@ public: void prefetch() override; + bool check = false; + private: bool nextImpl() override; @@ -45,6 +57,7 @@ private: size_t absolute_position = 0; std::mutex mutex; + Memory<> prefetch_buffer; }; } diff --git a/src/IO/ReadBufferFromRemoteFS.cpp b/src/IO/ReadBufferFromRemoteFS.cpp index 5af34447bc5..63403ba0599 100644 --- a/src/IO/ReadBufferFromRemoteFS.cpp +++ b/src/IO/ReadBufferFromRemoteFS.cpp @@ -72,9 +72,16 @@ bool ReadBufferFromRemoteFS::nextImpl() bool ReadBufferFromRemoteFS::read() { + if (check) + { + assert(!internal_buffer.empty()); + assert(working_buffer.begin() != nullptr); + } /// Transfer current position and working_buffer to actual ReadBuffer swap(*current_buf); /// Position and working_buffer will be updated in next() call + if (check) + assert(current_buf->buffer().begin() != nullptr); auto result = current_buf->next(); /// Assign result to current buffer. swap(*current_buf); @@ -111,7 +118,7 @@ void ReadBufferFromRemoteFS::reset(bool reset_inner_buf) { if (reset_inner_buf) current_buf.reset(); - BufferBase::set(nullptr, 0, 0); + // BufferBase::set(nullptr, 0, 0); } } diff --git a/src/IO/ReadBufferFromRemoteFS.h b/src/IO/ReadBufferFromRemoteFS.h index 559a6550cfc..da522e36020 100644 --- a/src/IO/ReadBufferFromRemoteFS.h +++ b/src/IO/ReadBufferFromRemoteFS.h @@ -22,6 +22,7 @@ public: void reset(bool reset_inner_buf = false); + bool check = false; protected: size_t fetch(size_t offset); diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index a482b124fee..d99f565419d 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -50,16 +50,12 @@ bool ReadBufferFromS3::nextImpl() /// `impl` has been initialized earlier and now we're at the end of the current portion of data. if (impl) { - if (use_external_buffer) - { - impl->set(working_buffer.begin(), working_buffer.size()); - } - else + if (!use_external_buffer) { impl->position() = position(); + assert(!impl->hasPendingData()); } - assert(!impl->hasPendingData()); } else { @@ -67,6 +63,14 @@ bool ReadBufferFromS3::nextImpl() impl = initialize(); next_result = impl->hasPendingData(); } + if (use_external_buffer) + { + // assert(!impl->buffer().begin() || impl->buffer().begin() != working_buffer.begin()); + impl->set(internal_buffer.begin(), internal_buffer.size()); + assert(working_buffer.begin() != nullptr); + assert(!internal_buffer.empty()); + // impl->BufferBase::set(nullptr, 0, 0); + } auto sleep_time_with_backoff_milliseconds = std::chrono::milliseconds(100); for (size_t attempt = 0; (attempt < max_single_read_retries) && !next_result; ++attempt) @@ -99,6 +103,14 @@ bool ReadBufferFromS3::nextImpl() /// Try to reinitialize `impl`. impl.reset(); impl = initialize(); + if (use_external_buffer) + { + // assert(!impl->buffer().begin() || impl->buffer().begin() != working_buffer.begin()); + impl->set(internal_buffer.begin(), internal_buffer.size()); + assert(working_buffer.begin() != nullptr); + assert(!internal_buffer.empty()); + // impl->BufferBase::set(nullptr, 0, 0); + } next_result = impl->hasPendingData(); } } From 1758ff2d321d3cd7b0cc1b3b4d785a8e389e53d4 Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 30 Sep 2021 16:02:17 +0000 Subject: [PATCH 074/919] fix --- src/Storages/FileLog/FileLogSettings.h | 8 +- src/Storages/FileLog/FileLogSource.cpp | 40 +- src/Storages/FileLog/FileLogSource.h | 3 +- .../FileLog/ReadBufferFromFileLog.cpp | 33 +- src/Storages/FileLog/ReadBufferFromFileLog.h | 4 - src/Storages/FileLog/StorageFileLog.cpp | 356 ++++++++++-------- src/Storages/FileLog/StorageFileLog.h | 39 +- 7 files changed, 275 insertions(+), 208 deletions(-) diff --git a/src/Storages/FileLog/FileLogSettings.h b/src/Storages/FileLog/FileLogSettings.h index ba76b4d02da..5ed55d94426 100644 --- a/src/Storages/FileLog/FileLogSettings.h +++ b/src/Storages/FileLog/FileLogSettings.h @@ -11,10 +11,10 @@ class ASTStorage; #define FILELOG_RELATED_SETTINGS(M) \ /* default is stream_poll_timeout_ms */ \ - M(Milliseconds, filelog_poll_timeout_ms, 0, "Timeout for single poll from FileLog.", 0) \ - M(UInt64, filelog_poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single filelog poll.", 0) \ - M(UInt64, filelog_max_block_size, 0, "Number of row collected by poll(s) for flushing data from filelog.", 0) \ - M(UInt64, filelog_max_threads, 8, "Number of max threads to parse files, default is 8", 0) + M(Milliseconds, poll_timeout_ms, 0, "Timeout for single poll from StorageFileLog.", 0) \ + M(UInt64, poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single StorageFileLog poll.", 0) \ + M(UInt64, max_block_size, 0, "Number of row collected by poll(s) for flushing data from StorageFileLog.", 0) \ + M(UInt64, max_threads, 8, "Number of max threads to parse files, default is 8", 0) #define LIST_OF_FILELOG_SETTINGS(M) \ FILELOG_RELATED_SETTINGS(M) \ diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index 590630658ca..e2223b3d3c2 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -17,14 +17,16 @@ FileLogSource::FileLogSource( StorageFileLog & storage_, const StorageMetadataPtr & metadata_snapshot_, const ContextPtr & context_, + const Names & columns, size_t max_block_size_, size_t poll_time_out_, size_t stream_number_, size_t max_streams_number_) - : SourceWithProgress(metadata_snapshot_->getSampleBlockWithVirtuals(storage_.getVirtuals())) + : SourceWithProgress(metadata_snapshot_->getSampleBlockForColumns(columns, storage_.getVirtuals(), storage_.getStorageID())) , storage(storage_) , metadata_snapshot(metadata_snapshot_) , context(context_) + , column_names(columns) , max_block_size(max_block_size_) , poll_time_out(poll_time_out_) , stream_number(stream_number_) @@ -34,19 +36,20 @@ FileLogSource::FileLogSource( metadata_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames(), storage.getVirtuals(), storage.getStorageID())) { buffer = std::make_unique(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_); - /// The last FileLogSource responsible for open files - if (stream_number == max_streams_number - 1) - { - storage.openFilesAndSetPos(); - } } FileLogSource::~FileLogSource() { - /// The last FileLogSource responsible for close files - if (stream_number == max_streams_number - 1) + auto & file_infos = storage.getFileInfos(); + + size_t files_per_stream = file_infos.file_names.size() / max_streams_number; + size_t start = stream_number * files_per_stream; + size_t end = stream_number == max_streams_number - 1 ? file_infos.file_names.size() : (stream_number + 1) * files_per_stream; + + /// Each stream responsible for close it's files and store meta + for (size_t i = start; i < end; ++i) { - storage.closeFilesAndStoreMeta(); + storage.closeFileAndStoreMeta(file_infos.file_names[i]); } } @@ -99,14 +102,21 @@ Chunk FileLogSource::generate() if (total_rows == 0) return {}; - auto result_columns = executor.getResultColumns(); + auto result_block = non_virtual_header.cloneWithColumns(executor.getResultColumns()); + auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); - for (auto & column : virtual_columns) - { - result_columns.emplace_back(std::move(column)); - } + for (const auto & column : virtual_block.getColumnsWithTypeAndName()) + result_block.insert(column); - return Chunk(std::move(result_columns), total_rows); + auto converting_dag = ActionsDAG::makeConvertingActions( + result_block.cloneEmpty().getColumnsWithTypeAndName(), + getPort().getHeader().getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto converting_actions = std::make_shared(std::move(converting_dag)); + converting_actions->execute(result_block); + + return Chunk(result_block.getColumns(), result_block.rows()); } } diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index 115c68f2e25..d257bb54f06 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -5,7 +5,6 @@ #include #include - namespace Poco { class Logger; @@ -19,6 +18,7 @@ public: StorageFileLog & storage_, const StorageMetadataPtr & metadata_snapshot_, const ContextPtr & context_, + const Names & columns, size_t max_block_size_, size_t poll_time_out_, size_t stream_number_, @@ -37,6 +37,7 @@ private: StorageFileLog & storage; StorageMetadataPtr metadata_snapshot; ContextPtr context; + Names column_names; UInt64 max_block_size; size_t poll_time_out; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index e39fd84acfe..e161c69a960 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -31,15 +31,8 @@ ReadBufferFromFileLog::ReadBufferFromFileLog( , stream_number(stream_number_) , max_streams_number(max_streams_number_) { - cleanUnprocessed(); - allowed = false; -} - -void ReadBufferFromFileLog::cleanUnprocessed() -{ - records.clear(); current = records.begin(); - BufferBase::set(nullptr, 0, 0); + allowed = false; } bool ReadBufferFromFileLog::poll() @@ -50,11 +43,10 @@ bool ReadBufferFromFileLog::poll() return true; } - buffer_status = BufferStatus::NO_RECORD_RETURNED; - auto new_records = pollBatch(batch_size); if (new_records.empty()) { + buffer_status = BufferStatus::NO_RECORD_RETURNED; LOG_TRACE(log, "No records returned"); return false; } @@ -106,33 +98,28 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ for (size_t i = start; i < end; ++i) { - auto file_name = file_infos.file_names[i]; - auto & file_ctx = file_infos.context_by_name.at(file_name); + const auto & file_name = file_infos.file_names[i]; + + auto & file_ctx = StorageFileLog::findInMap(file_infos.context_by_name, file_name); if (file_ctx.status == StorageFileLog::FileStatus::NO_CHANGE) continue; - auto & file_meta = file_infos.meta_by_inode.at(file_infos.inode_by_name.at(file_name)); + auto & file_meta = StorageFileLog::findInMap(file_infos.meta_by_inode, file_ctx.inode); Record record; while (read_records_size < need_records_size && static_cast(file_ctx.reader.tellg()) < file_meta.last_open_end) { - if (!file_ctx.reader.good()) - { - throw Exception("Can not read from file " + file_name + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); - } - UInt64 start_offset = file_ctx.reader.tellg(); + StorageFileLog::assertStreamGood(file_ctx.reader); + std::getline(file_ctx.reader, record.data); record.file_name = file_name; - record.offset = start_offset; + record.offset = file_ctx.reader.tellg(); new_records.emplace_back(record); ++read_records_size; } UInt64 current_position = file_ctx.reader.tellg(); - if (!file_ctx.reader.good()) - { - throw Exception("Can not read from file " + file_name + ", stream broken.", ErrorCodes::CANNOT_READ_FROM_ISTREAM); - } + StorageFileLog::assertStreamGood(file_ctx.reader); file_meta.last_writen_position = current_position; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index 8a3fdf235d7..c792978a68a 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -83,14 +83,10 @@ private: using TaskThread = BackgroundSchedulePool::TaskHolder; - TaskThread wait_task; - Records pollBatch(size_t batch_size_); void readNewRecords(Records & new_records, size_t batch_size_); - void cleanUnprocessed(); - bool nextImpl() override; }; } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index d5d86eec0b8..18a76b72af8 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -19,9 +20,9 @@ #include #include #include -#include #include #include +#include #include #include #include @@ -41,6 +42,7 @@ namespace ErrorCodes extern const int READ_META_FILE_FAILED; extern const int FILE_STREAM_ERROR; extern const int LOGICAL_ERROR; + extern const int TABLE_METADATA_ALREADY_EXISTS; } namespace @@ -54,20 +56,24 @@ StorageFileLog::StorageFileLog( const StorageID & table_id_, ContextPtr context_, const ColumnsDescription & columns_, - const String & relative_path_, + const String & path_, + const String & relative_data_path_, const String & format_name_, std::unique_ptr settings, + const String & comment, bool attach) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) , filelog_settings(std::move(settings)) - , path(getContext()->getUserFilesPath() + "/" + relative_path_) + , path(path_) + , relative_data_path(relative_data_path_) , format_name(format_name_) , log(&Poco::Logger::get("StorageFileLog (" + table_id_.table_name + ")")) , milliseconds_to_wait(RESCHEDULE_MS) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); + storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); try @@ -75,9 +81,11 @@ StorageFileLog::StorageFileLog( loadMetaFiles(attach); loadFiles(); +#ifndef NDEBUG assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); assert(file_infos.file_names.size() == file_infos.inode_by_name.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); +#endif if (path_is_directory) directory_watch = std::make_unique(path); @@ -93,17 +101,20 @@ StorageFileLog::StorageFileLog( void StorageFileLog::loadMetaFiles(bool attach) { - const auto database = DatabaseCatalog::instance().getDatabase(getStorageID().getDatabaseName()); - const auto table_name = getStorageID().getTableName(); - - root_meta_path = database->getMetadataPath() + "/." + table_name; + /// We just use default storage policy + auto storage_policy = getContext()->getStoragePolicy("default"); + auto data_volume = storage_policy->getVolume(0); + root_meta_path = std::filesystem::path(data_volume->getDisk()->getPath()) / getStorageID().getTableName(); /// Create table, just create meta data directory if (!attach) { if (std::filesystem::exists(root_meta_path)) { - std::filesystem::remove_all(root_meta_path); + throw Exception( + ErrorCodes::TABLE_METADATA_ALREADY_EXISTS, + "Metadata files already exist by path: {}, remove them manually if it is intended", + root_meta_path); } std::filesystem::create_directories(root_meta_path); } @@ -113,7 +124,7 @@ void StorageFileLog::loadMetaFiles(bool attach) /// Meta file may lost, log and create directory if (!std::filesystem::exists(root_meta_path)) { - LOG_INFO(log, "Meta files of table {} may have lost.", getStorageID().getTableName()); + LOG_ERROR(log, "Metadata files of table {} are lost.", getStorageID().getTableName()); std::filesystem::create_directories(root_meta_path); } /// Load all meta info to file_infos; @@ -123,19 +134,27 @@ void StorageFileLog::loadMetaFiles(bool attach) void StorageFileLog::loadFiles() { + if (!symlinkStartsWith(path, getContext()->getUserFilesPath())) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "The absolute data path should start with user_files_path {}", getContext()->getUserFilesPath()); + } - if (std::filesystem::is_regular_file(path)) + auto absolute_path = std::filesystem::absolute(path); + absolute_path = absolute_path.lexically_normal(); /// Normalize path. + + if (std::filesystem::is_regular_file(absolute_path)) { path_is_directory = false; - root_data_path = getContext()->getUserFilesPath(); + root_data_path = absolute_path.parent_path(); - file_infos.file_names.push_back(std::filesystem::path(path).filename()); + file_infos.file_names.push_back(absolute_path.filename()); } - else if (std::filesystem::is_directory(path)) + else if (std::filesystem::is_directory(absolute_path)) { - root_data_path = path; + root_data_path = absolute_path; /// Just consider file with depth 1 - for (const auto & dir_entry : std::filesystem::directory_iterator{path}) + for (const auto & dir_entry : std::filesystem::directory_iterator{absolute_path}) { if (dir_entry.is_regular_file()) { @@ -145,76 +164,86 @@ void StorageFileLog::loadFiles() } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "The path {} neither a regular file, nor a directory", path); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The path {} neither a regular file, nor a directory", absolute_path.c_str()); } /// Get files inode for (const auto & file : file_infos.file_names) { auto inode = getInode(getFullDataPath(file)); - file_infos.inode_by_name.emplace(file, inode); - file_infos.context_by_name.emplace(file, FileContext{}); + file_infos.context_by_name.emplace(file, FileContext{.inode = inode}); } /// Update file meta or create file meta - for (const auto & file_inode : file_infos.inode_by_name) + for (const auto & [file, ctx] : file_infos.context_by_name) { - if (auto it = file_infos.meta_by_inode.find(file_inode.second); it != file_infos.meta_by_inode.end()) + if (auto it = file_infos.meta_by_inode.find(ctx.inode); it != file_infos.meta_by_inode.end()) { /// data file have been renamed, need update meta file's name - if (it->second.file_name != file_inode.first) + if (it->second.file_name != file) { - it->second.file_name = file_inode.first; + it->second.file_name = file; if (std::filesystem::exists(getFullMetaPath(it->second.file_name))) { - std::filesystem::rename(getFullMetaPath(it->second.file_name), getFullMetaPath(file_inode.first)); + std::filesystem::rename(getFullMetaPath(it->second.file_name), getFullMetaPath(file)); } } } /// New file else { - FileMeta meta{file_inode.first, 0, 0}; - file_infos.meta_by_inode.emplace(file_inode.second, meta); + FileMeta meta{file, 0, 0}; + file_infos.meta_by_inode.emplace(ctx.inode, meta); } } /// Clear unneeded meta file, because data files may be deleted - if (file_infos.meta_by_inode.size() > file_infos.inode_by_name.size()) + if (file_infos.meta_by_inode.size() > file_infos.context_by_name.size()) { InodeToFileMeta valid_metas; - valid_metas.reserve(file_infos.inode_by_name.size()); + valid_metas.reserve(file_infos.context_by_name.size()); for (const auto & it : file_infos.meta_by_inode) { - if (file_infos.inode_by_name.contains(it.second.file_name)) + auto file_name = it.second.file_name; + if (file_infos.context_by_name.contains(file_name)) valid_metas.emplace(it); + /// Delete meta file from filesystem + else + std::filesystem::remove(getFullMetaPath(file_name)); } file_infos.meta_by_inode.swap(valid_metas); } } -void StorageFileLog::serialize(bool with_end_pos) const +void StorageFileLog::serialize() const { for (const auto & it : file_infos.meta_by_inode) { auto full_name = getFullMetaPath(it.second.file_name); if (!std::filesystem::exists(full_name)) { - Poco::File{full_name}.createFile(); - } - WriteBufferFromFile buf(full_name); - writeIntText(it.first, buf); - writeChar('\n', buf); - writeIntText(it.second.last_writen_position, buf); - - if (with_end_pos) - { - writeChar('\n', buf); - writeIntText(it.second.last_open_end, buf); + FS::createFile(full_name); } + WriteBufferFromFile out(full_name); + writeIntText(it.first, out); + writeChar('\n', out); + writeIntText(it.second.last_writen_position, out); } } +void StorageFileLog::serialize(UInt64 inode, const FileMeta & file_meta) const +{ + auto full_name = getFullMetaPath(file_meta.file_name); + if (!std::filesystem::exists(full_name)) + { + FS::createFile(full_name); + } + WriteBufferFromFile out(full_name); + writeIntText(inode, out); + writeChar('\n', out); + writeIntText(file_meta.last_writen_position, out); +} + void StorageFileLog::deserialize() { for (const auto & dir_entry : std::filesystem::directory_iterator{root_meta_path}) @@ -228,19 +257,16 @@ void StorageFileLog::deserialize() root_meta_path); } - ReadBufferFromFile buf(dir_entry.path().c_str()); + ReadBufferFromFile in(dir_entry.path().c_str()); FileMeta meta; UInt64 inode, last_written_pos; - if (!tryReadIntText(inode, buf)) + if (!tryReadIntText(inode, in)) { throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); } - if (!checkChar('\n', buf)) - { - throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); - } - if (!tryReadIntText(last_written_pos, buf)) + assertChar('\n', in); + if (!tryReadIntText(last_written_pos, in)) { throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); } @@ -248,15 +274,6 @@ void StorageFileLog::deserialize() meta.file_name = dir_entry.path().filename(); meta.last_writen_position = last_written_pos; - /// May have last open end in meta file - if (checkChar('\n', buf)) - { - if (!tryReadIntText(meta.last_open_end, buf)) - { - throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); - } - } - file_infos.meta_by_inode.emplace(inode, meta); } } @@ -280,6 +297,7 @@ Pipe StorageFileLog::read( size_t /* max_block_size */, unsigned /* num_streams */) { + /// We need this lock, in case read and streamToViews execute at the same time std::lock_guard lock(status_mutex); updateFileInfos(); @@ -287,30 +305,30 @@ Pipe StorageFileLog::read( /// No files to parse if (file_infos.file_names.empty()) { + LOG_INFO(log, "There is a idle table named {}, no files need to parse.", getName()); return Pipe{}; } auto modified_context = Context::createCopy(local_context); - auto max_streams_number = std::min(filelog_settings->filelog_max_threads, file_infos.file_names.size()); + auto max_streams_number = std::min(filelog_settings->max_threads, file_infos.file_names.size()); + + /// Each stream responsible for closing it's files and store meta + openFilesAndSetPos(); Pipes pipes; pipes.reserve(max_streams_number); for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) { - Pipe pipe(std::make_shared( - *this, metadata_snapshot, modified_context, getMaxBlockSize(), getPollTimeoutMillisecond(), stream_number, max_streams_number)); - - auto convert_actions_dag = ActionsDAG::makeConvertingActions( - pipe.getHeader().getColumnsWithTypeAndName(), - metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()).getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Name); - - auto actions = std::make_shared( - convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); - - pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); }); - pipes.emplace_back(std::move(pipe)); + pipes.emplace_back(std::make_shared( + *this, + metadata_snapshot, + modified_context, + column_names, + getMaxBlockSize(), + getPollTimeoutMillisecond(), + stream_number, + max_streams_number)); } return Pipe::unitePipes(std::move(pipes)); @@ -318,15 +336,29 @@ Pipe StorageFileLog::read( void StorageFileLog::drop() { - if (std::filesystem::exists(root_meta_path)) - std::filesystem::remove_all(root_meta_path); + try + { + if (std::filesystem::exists(root_meta_path)) + std::filesystem::remove_all(root_meta_path); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } void StorageFileLog::startup() { - if (task) + try { - task->holder->activateAndSchedule(); + if (task) + { + task->holder->activateAndSchedule(); + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); } } @@ -342,32 +374,31 @@ void StorageFileLog::shutdown() closeFilesAndStoreMeta(); } +void StorageFileLog::assertStreamGood(const std::ifstream & reader) +{ + if (!reader.good()) + { + throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Stream is in bad state."); + } +} + void StorageFileLog::openFilesAndSetPos() { for (const auto & file : file_infos.file_names) { - auto & file_ctx = file_infos.context_by_name.at(file); + auto & file_ctx = findInMap(file_infos.context_by_name, file); if (file_ctx.status != FileStatus::NO_CHANGE) { file_ctx.reader.open(getFullDataPath(file)); - if (!file_ctx.reader.good()) - { - throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Open file {} failed.", file); - } + assertStreamGood(file_ctx.reader); file_ctx.reader.seekg(0, file_ctx.reader.end); - if (!file_ctx.reader.good()) - { - throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Seekg file {} failed.", file); - } + assertStreamGood(file_ctx.reader); auto file_end = file_ctx.reader.tellg(); - if (!file_ctx.reader.good()) - { - throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Tellg file {} failed.", file); - } + assertStreamGood(file_ctx.reader); - auto & meta = file_infos.meta_by_inode.at(file_infos.inode_by_name.at(file)); + auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); if (meta.last_writen_position > static_cast(file_end)) { throw Exception(ErrorCodes::FILE_STREAM_ERROR, "File {} has been broken.", file); @@ -376,13 +407,10 @@ void StorageFileLog::openFilesAndSetPos() meta.last_open_end = file_end; file_ctx.reader.seekg(meta.last_writen_position); - if (!file_ctx.reader.good()) - { - throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Seekg file {} failed.", file); - } + assertStreamGood(file_ctx.reader); } } - serialize(true); + serialize(); } void StorageFileLog::closeFilesAndStoreMeta() @@ -395,23 +423,33 @@ void StorageFileLog::closeFilesAndStoreMeta() serialize(); } +void StorageFileLog::closeFileAndStoreMeta(const String & file_name) +{ + auto & file_ctx = findInMap(file_infos.context_by_name, file_name); + if (file_ctx.reader.is_open()) + file_ctx.reader.close(); + + auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); + serialize(file_ctx.inode, meta); +} + size_t StorageFileLog::getMaxBlockSize() const { - return filelog_settings->filelog_max_block_size.changed ? filelog_settings->filelog_max_block_size.value - : getContext()->getSettingsRef().max_insert_block_size.value; + return filelog_settings->max_block_size.changed ? filelog_settings->max_block_size.value + : getContext()->getSettingsRef().max_insert_block_size.value; } size_t StorageFileLog::getPollMaxBatchSize() const { - size_t batch_size = filelog_settings->filelog_poll_max_batch_size.changed ? filelog_settings->filelog_poll_max_batch_size.value - : getContext()->getSettingsRef().max_block_size.value; + size_t batch_size = filelog_settings->poll_max_batch_size.changed ? filelog_settings->poll_max_batch_size.value + : getContext()->getSettingsRef().max_block_size.value; return std::min(batch_size, getMaxBlockSize()); } size_t StorageFileLog::getPollTimeoutMillisecond() const { - return filelog_settings->filelog_poll_timeout_ms.changed ? filelog_settings->filelog_poll_timeout_ms.totalMilliseconds() - : getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); + return filelog_settings->poll_timeout_ms.changed ? filelog_settings->poll_timeout_ms.totalMilliseconds() + : getContext()->getSettingsRef().stream_poll_timeout_ms.totalMilliseconds(); } bool StorageFileLog::checkDependencies(const StorageID & table_id) @@ -503,10 +541,11 @@ bool StorageFileLog::streamToViews() throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); auto metadata_snapshot = getInMemoryMetadataPtr(); - auto max_streams_number = std::min(filelog_settings->filelog_max_threads.value, file_infos.file_names.size()); + auto max_streams_number = std::min(filelog_settings->max_threads.value, file_infos.file_names.size()); /// No files to parse if (max_streams_number == 0) { + LOG_INFO(log, "There is a idle table named {}, no files need to parse.", getName()); return false; } @@ -519,36 +558,38 @@ bool StorageFileLog::streamToViews() InterpreterInsertQuery interpreter(insert, new_context, false, true, true); auto block_io = interpreter.execute(); + /// Each stream responsible for closing it's files and store meta + openFilesAndSetPos(); + Pipes pipes; pipes.reserve(max_streams_number); for (size_t stream_number = 0; stream_number < max_streams_number; ++stream_number) { - Pipe pipe(std::make_shared( - *this, metadata_snapshot, new_context, getPollMaxBatchSize(), getPollTimeoutMillisecond(), stream_number, max_streams_number)); - - auto convert_actions_dag = ActionsDAG::makeConvertingActions( - pipe.getHeader().getColumnsWithTypeAndName(), - block_io.pipeline.getHeader().getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Name); - - auto actions = std::make_shared( - convert_actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); - - pipe.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); }); - pipes.emplace_back(std::move(pipe)); + pipes.emplace_back(std::make_shared( + *this, + metadata_snapshot, + new_context, + block_io.pipeline.getHeader().getNames(), + getPollMaxBatchSize(), + getPollTimeoutMillisecond(), + stream_number, + max_streams_number)); } auto input= Pipe::unitePipes(std::move(pipes)); assertBlocksHaveEqualStructure(input.getHeader(), block_io.pipeline.getHeader(), "StorageFileLog streamToViews"); - block_io.pipeline.complete(std::move(input)); - - CompletedPipelineExecutor executor(block_io.pipeline); - executor.execute(); + size_t rows = 0; + { + block_io.pipeline.complete(std::move(input)); + block_io.pipeline.setProgressCallback([&](const Progress & progress) { rows += progress.read_rows.load(); }); + CompletedPipelineExecutor executor(block_io.pipeline); + executor.execute(); + } UInt64 milliseconds = watch.elapsedMilliseconds(); - LOG_DEBUG(log, "Pushing data to {} took {} ms.", table_id.getNameForLogs(), milliseconds); + LOG_DEBUG(log, "Pushing {} rows to {} took {} ms.", rows, table_id.getNameForLogs(), milliseconds); return updateFileInfos(); } @@ -569,7 +610,7 @@ void registerStorageFileLog(StorageFactory & factory) } auto physical_cpu_cores = getNumberOfPhysicalCPUCores(); - auto num_threads = filelog_settings->filelog_max_threads.value; + auto num_threads = filelog_settings->max_threads.value; if (num_threads > physical_cpu_cores) { @@ -580,12 +621,12 @@ void registerStorageFileLog(StorageFactory & factory) throw Exception("Number of threads to parse files can not be lower than 1", ErrorCodes::BAD_ARGUMENTS); } - if (filelog_settings->filelog_max_block_size.changed && filelog_settings->filelog_max_block_size.value < 1) + if (filelog_settings->max_block_size.changed && filelog_settings->max_block_size.value < 1) { throw Exception("filelog_max_block_size can not be lower than 1", ErrorCodes::BAD_ARGUMENTS); } - if (filelog_settings->filelog_poll_max_batch_size.changed && filelog_settings->filelog_poll_max_batch_size.value < 1) + if (filelog_settings->poll_max_batch_size.changed && filelog_settings->poll_max_batch_size.value < 1) { throw Exception("filelog_poll_max_batch_size can not be lower than 1", ErrorCodes::BAD_ARGUMENTS); } @@ -601,7 +642,15 @@ void registerStorageFileLog(StorageFactory & factory) auto format = format_ast->as().value.safeGet(); return StorageFileLog::create( - args.table_id, args.getContext(), args.columns, path, format, std::move(filelog_settings), args.attach); + args.table_id, + args.getContext(), + args.columns, + path, + args.relative_data_path, + format, + std::move(filelog_settings), + args.comment, + args.attach); }; factory.registerStorage( @@ -619,9 +668,13 @@ bool StorageFileLog::updateFileInfos() /// For table just watch one file, we can not use directory monitor to watch it if (!path_is_directory) { - assert( - file_infos.file_names.size() == file_infos.meta_by_inode.size() == file_infos.inode_by_name.size() - == file_infos.context_by_name.size() == 1); +#ifndef NDEBUG + assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); + assert(file_infos.file_names.size() == file_infos.inode_by_name.size()); + assert(file_infos.file_names.size() == file_infos.context_by_name.size()); + assert(file_infos.file_names.size() == 1); +#endif + if (auto it = file_infos.context_by_name.find(file_infos.file_names[0]); it != file_infos.context_by_name.end()) { it->second.status = FileStatus::UPDATED; @@ -642,7 +695,8 @@ bool StorageFileLog::updateFileInfos() { switch (event.type) { - case Poco::DirectoryWatcher::DW_ITEM_ADDED: { + case Poco::DirectoryWatcher::DW_ITEM_ADDED: + { LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); if (std::filesystem::is_regular_file(event.path)) { @@ -650,45 +704,40 @@ bool StorageFileLog::updateFileInfos() auto inode = getInode(event.path); file_infos.file_names.push_back(file_name); - file_infos.inode_by_name.emplace(file_name, inode); - FileMeta meta{file_name, 0, 0}; - file_infos.meta_by_inode.emplace(inode, meta); - file_infos.context_by_name.emplace(file_name, FileContext{}); + file_infos.meta_by_inode.emplace(inode, FileMeta{.file_name = file_name}); + file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); } break; } - case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: { - auto file_name = std::filesystem::path(event.path).filename(); + case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: + { + String file_name = std::filesystem::path(event.path).filename(); LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) - { - it->second.status = FileStatus::UPDATED; - } + auto & file_ctx = findInMap(file_infos.context_by_name, file_name); + file_ctx.status = FileStatus::UPDATED; break; } case Poco::DirectoryWatcher::DW_ITEM_REMOVED: - case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: { - auto file_name = std::filesystem::path(event.path).filename(); + case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: + { + String file_name = std::filesystem::path(event.path).filename(); LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) - { - it->second.status = FileStatus::REMOVED; - } + auto & file_ctx = findInMap(file_infos.context_by_name, file_name); + file_ctx.status = FileStatus::REMOVED; break; } /// file rename - case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: { + case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: + { auto file_name = std::filesystem::path(event.path).filename(); LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); file_infos.file_names.push_back(file_name); - file_infos.context_by_name.emplace(file_name, FileContext{}); - auto inode = getInode(event.path); - file_infos.inode_by_name.emplace(file_name, inode); + file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); if (auto it = file_infos.meta_by_inode.find(inode); it != file_infos.meta_by_inode.end()) { @@ -705,19 +754,18 @@ bool StorageFileLog::updateFileInfos() } std::vector valid_files; + /// Remove file infos with REMOVE status for (const auto & file_name : file_infos.file_names) { if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end() && it->second.status == FileStatus::REMOVED) { + /// Erase meta_by_inode first, otherwise it become invalid + file_infos.meta_by_inode.erase(it->second.inode); file_infos.context_by_name.erase(it); - if (auto inode = file_infos.inode_by_name.find(file_name); inode != file_infos.inode_by_name.end()) - { - file_infos.inode_by_name.erase(inode); - file_infos.meta_by_inode.erase(inode->second); - if (std::filesystem::exists(getFullMetaPath(file_name))) - std::filesystem::remove(getFullMetaPath(file_name)); - } + + if (std::filesystem::exists(getFullMetaPath(file_name))) + std::filesystem::remove(getFullMetaPath(file_name)); } else { @@ -727,9 +775,11 @@ bool StorageFileLog::updateFileInfos() file_infos.file_names.swap(valid_files); /// These file infos should always have same size(one for one) +#ifndef NDEBUG assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); assert(file_infos.file_names.size() == file_infos.inode_by_name.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); +#endif return events.empty() || file_infos.file_names.empty(); } diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index ec81558438c..7384543fa36 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -18,6 +18,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + class StorageFileLog final : public shared_ptr_helper, public IStorage, WithContext { friend struct shared_ptr_helper; @@ -59,6 +64,7 @@ public: struct FileContext { FileStatus status = FileStatus::OPEN; + UInt64 inode; std::ifstream reader; }; @@ -69,13 +75,11 @@ public: UInt64 last_open_end = 0; }; - using FileNameToInode = std::unordered_map; using InodeToFileMeta = std::unordered_map; using FileNameToContext = std::unordered_map; struct FileInfos { - FileNameToInode inode_by_name; InodeToFileMeta meta_by_inode; FileNameToContext context_by_name; /// file names without path @@ -84,8 +88,8 @@ public: auto & getFileInfos() { return file_infos; } - auto getFullMetaPath(const String & file_name) const { return root_meta_path + "/" + file_name; } - auto getFullDataPath(const String & file_name) const { return root_data_path + "/" + file_name; } + String getFullMetaPath(const String & file_name) const { return std::filesystem::path(root_meta_path) / file_name; } + String getFullDataPath(const String & file_name) const { return std::filesystem::path(root_data_path) / file_name; } NamesAndTypesList getVirtuals() const override; @@ -94,23 +98,40 @@ public: static UInt64 getInode(const String & file_name); void openFilesAndSetPos(); + /// Used in shutdown() void closeFilesAndStoreMeta(); + /// Used in FileSource + void closeFileAndStoreMeta(const String & file_name); + + static void assertStreamGood(const std::ifstream & reader); + + template + static V & findInMap(std::unordered_map & map, const K & key) + { + if (auto it = map.find(key); it != map.end()) + return it->second; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "The key {} doesn't exist.", key); + } protected: StorageFileLog( const StorageID & table_id_, ContextPtr context_, const ColumnsDescription & columns_, - const String & relative_path_, + const String & path_, + const String & relative_data_path_, const String & format_name_, std::unique_ptr settings, + const String & comment, bool attach); private: std::unique_ptr filelog_settings; - /// user_files_path/ + path_argument/ const String path; + /// For meta file + const String relative_data_path; bool path_is_directory = true; /// If path argument of the table is a regular file, it equals to user_files_path @@ -157,8 +178,10 @@ private: bool updateFileInfos(); - /// Serialize all file meta - void serialize(bool with_end_pos = false) const; + /// Used in shutdown() + void serialize() const; + /// Used in FileSource closeFileAndStoreMeta(file_name); + void serialize(UInt64 inode, const FileMeta & file_meta) const; void deserialize(); }; From 1d0f0993163477ea1d2536c8a14d366411a17a51 Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 30 Sep 2021 16:22:26 +0000 Subject: [PATCH 075/919] update test --- tests/queries/0_stateless/02022_storage_filelog_one_file.sh | 2 +- tests/queries/0_stateless/02023_storage_filelog.sh | 2 +- tests/queries/0_stateless/02024_storage_filelog_mv.sh | 2 +- tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh index 2cf23957c9e..d1ed3e5a142 100755 --- a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh @@ -18,7 +18,7 @@ do done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" -${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('a.txt', 'CSV');" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/a.txt', 'CSV');" ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 095c0f05c40..6554f019a9f 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -22,7 +22,7 @@ do done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" -${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('logs', 'CSV');" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/logs/', 'CSV');" ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" diff --git a/tests/queries/0_stateless/02024_storage_filelog_mv.sh b/tests/queries/0_stateless/02024_storage_filelog_mv.sh index f73dc988c92..0503552b8dd 100755 --- a/tests/queries/0_stateless/02024_storage_filelog_mv.sh +++ b/tests/queries/0_stateless/02024_storage_filelog_mv.sh @@ -20,7 +20,7 @@ do done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" -${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('logs', 'CSV');" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/logs/', 'CSV');" ${CLICKHOUSE_CLIENT} --query "drop table if exists mv;" ${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order by k as select * from file_log;" diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index 076e2af44c2..a7293288522 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -22,7 +22,7 @@ do done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" -${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('logs', 'CSV');" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/logs/', 'CSV');" ${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" From 4390c8f0290069847f1f644df46c825e9253383a Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 30 Sep 2021 17:13:45 +0000 Subject: [PATCH 076/919] fix --- src/Storages/FileLog/StorageFileLog.cpp | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 18a76b72af8..dca7a678678 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -715,8 +715,14 @@ bool StorageFileLog::updateFileInfos() { String file_name = std::filesystem::path(event.path).filename(); LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - auto & file_ctx = findInMap(file_infos.context_by_name, file_name); - file_ctx.status = FileStatus::UPDATED; + /// When new file added and appended, it has two event: DW_ITEM_ADDED + /// and DW_ITEM_MODIFIED, since the order of these two events in the + /// sequence is uncentain, so we may can not find it in file_infos, just + /// skip it, the file info will be handled in DW_ITEM_ADDED case. + if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) + { + it->second.status = FileStatus::UPDATED; + } break; } @@ -725,11 +731,12 @@ bool StorageFileLog::updateFileInfos() { String file_name = std::filesystem::path(event.path).filename(); LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - auto & file_ctx = findInMap(file_infos.context_by_name, file_name); - file_ctx.status = FileStatus::REMOVED; + if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) + { + it->second.status = FileStatus::REMOVED; + } break; } - /// file rename case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: { auto file_name = std::filesystem::path(event.path).filename(); @@ -739,9 +746,9 @@ bool StorageFileLog::updateFileInfos() auto inode = getInode(event.path); file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); + /// File has been renamed, we should also rename meta file if (auto it = file_infos.meta_by_inode.find(inode); it != file_infos.meta_by_inode.end()) { - // rename meta file auto old_name = it->second.file_name; it->second.file_name = file_name; if (std::filesystem::exists(getFullMetaPath(old_name))) From e4bc81ac78e85f971a38559e812da3dd7589c043 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 30 Sep 2021 16:37:11 +0300 Subject: [PATCH 077/919] Better --- ...chronousReadIndirectBufferFromRemoteFS.cpp | 30 ++++++++----------- ...ynchronousReadIndirectBufferFromRemoteFS.h | 13 ++++---- src/Disks/ReadIndirectBufferFromWebServer.cpp | 24 +++++++++------ src/Disks/ReadIndirectBufferFromWebServer.h | 2 +- src/IO/ReadBufferFromRemoteFS.cpp | 16 +++------- src/IO/ReadBufferFromRemoteFS.h | 5 ++-- src/IO/ReadBufferFromS3.cpp | 6 +--- src/IO/ReadWriteBufferFromHTTP.h | 29 ++++++++++++++---- src/IO/ThreadPoolRemoteFSReader.cpp | 6 ++-- src/IO/ThreadPoolRemoteFSReader.h | 3 +- 10 files changed, 72 insertions(+), 62 deletions(-) diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp index fe433c76d9b..c6e73c369dd 100644 --- a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -26,17 +26,23 @@ namespace ErrorCodes AsynchronousReadIndirectBufferFromRemoteFS::AsynchronousReadIndirectBufferFromRemoteFS( - AsynchronousReaderPtr reader_, Int32 priority_, std::shared_ptr impl_) - : ReadBufferFromFileBase(DBMS_DEFAULT_BUFFER_SIZE, nullptr, 0) - , reader(reader_), priority(priority_), impl(impl_), prefetch_buffer(DBMS_DEFAULT_BUFFER_SIZE) + AsynchronousReaderPtr reader_, Int32 priority_, + std::shared_ptr impl_, size_t buf_size_) + : ReadBufferFromFileBase(buf_size_, nullptr, 0) + , reader(reader_) + , priority(priority_) + , impl(impl_) + , prefetch_buffer(buf_size_) { } -std::future AsynchronousReadIndirectBufferFromRemoteFS::readNext() +std::future AsynchronousReadIndirectBufferFromRemoteFS::readInto(char * data, size_t size) { IAsynchronousReader::Request request; request.descriptor = std::make_shared(impl); + request.buf = data; + request.size = size; request.offset = absolute_position; request.priority = priority; return reader->submit(request); @@ -48,11 +54,7 @@ void AsynchronousReadIndirectBufferFromRemoteFS::prefetch() if (prefetch_future.valid()) return; - std::lock_guard lock(mutex); - assert(prefetch_buffer.data() != nullptr); - prefetch_buffer.resize(DBMS_DEFAULT_BUFFER_SIZE); - impl->set(prefetch_buffer.data(), prefetch_buffer.size()); - prefetch_future = readNext(); + prefetch_future = readInto(prefetch_buffer.data(), prefetch_buffer.size()); } @@ -65,7 +67,6 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() CurrentMetrics::Increment metric_increment{CurrentMetrics::AsynchronousReadWait}; Stopwatch watch; { - std::lock_guard lock(mutex); size = prefetch_future.get(); if (size) { @@ -80,12 +81,7 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() } else { - std::lock_guard lock(mutex); - impl->check = true; - impl->set(memory.data(), memory.size()); - assert(memory.data() != nullptr); - assert(impl->buffer().begin() != nullptr); - size = readNext().get(); + size = readInto(memory.data(), memory.size()).get(); if (size) { set(memory.data(), memory.size()); @@ -143,7 +139,7 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence } pos = working_buffer.end(); - impl->reset(true); + impl->reset(); return absolute_position; } diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h index 0dd59d98864..1095a22091f 100644 --- a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -29,7 +29,9 @@ class AsynchronousReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase { public: explicit AsynchronousReadIndirectBufferFromRemoteFS( - AsynchronousReaderPtr reader_, Int32 priority_, std::shared_ptr impl_); + AsynchronousReaderPtr reader_, Int32 priority_, + std::shared_ptr impl_, + size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE); ~AsynchronousReadIndirectBufferFromRemoteFS() override; @@ -41,22 +43,23 @@ public: void prefetch() override; - bool check = false; - private: bool nextImpl() override; void finalize(); - std::future readNext(); + std::future readInto(char * data, size_t size); AsynchronousReaderPtr reader; + Int32 priority; + std::shared_ptr impl; + std::future prefetch_future; size_t absolute_position = 0; - std::mutex mutex; + Memory<> prefetch_buffer; }; diff --git a/src/Disks/ReadIndirectBufferFromWebServer.cpp b/src/Disks/ReadIndirectBufferFromWebServer.cpp index 17fa4d7d35c..f5ef1efae02 100644 --- a/src/Disks/ReadIndirectBufferFromWebServer.cpp +++ b/src/Disks/ReadIndirectBufferFromWebServer.cpp @@ -26,7 +26,7 @@ static const auto WAIT_MS = 10; ReadIndirectBufferFromWebServer::ReadIndirectBufferFromWebServer( const String & url_, ContextPtr context_, size_t buf_size_, size_t backoff_threshold_, size_t max_tries_, bool use_external_buffer_) - : BufferWithOwnMemory(buf_size_) + : SeekableReadBuffer(nullptr, 0) , log(&Poco::Logger::get("ReadIndirectBufferFromWebServer")) , context(context_) , url(url_) @@ -71,18 +71,18 @@ bool ReadIndirectBufferFromWebServer::nextImpl() if (impl) { - if (use_external_buffer) - { - impl.reset(); - // impl->set(working_buffer.begin(), working_buffer.size()); - // impl->BufferBase::set(nullptr, 0, 0); - } - else + if (!use_external_buffer) { impl->position() = position(); + assert(!impl->hasPendingData()); } + } - assert(!impl->hasPendingData()); + if (use_external_buffer) + { + impl->set(internal_buffer.begin(), internal_buffer.size()); + assert(working_buffer.begin() != nullptr); + assert(!internal_buffer.empty()); } WriteBufferFromOwnString error_msg; @@ -95,6 +95,12 @@ bool ReadIndirectBufferFromWebServer::nextImpl() if (!impl) { impl = initialize(); + if (use_external_buffer) + { + impl->set(internal_buffer.begin(), internal_buffer.size()); + assert(working_buffer.begin() != nullptr); + assert(!internal_buffer.empty()); + } next_result = impl->hasPendingData(); if (next_result) break; diff --git a/src/Disks/ReadIndirectBufferFromWebServer.h b/src/Disks/ReadIndirectBufferFromWebServer.h index 95b86f7db5e..c8ed124dfff 100644 --- a/src/Disks/ReadIndirectBufferFromWebServer.h +++ b/src/Disks/ReadIndirectBufferFromWebServer.h @@ -13,7 +13,7 @@ namespace DB * * Usage: ReadIndirectBufferFromRemoteFS -> SeekAvoidingReadBuffer -> ReadIndirectBufferFromWebServer -> ReadWriteBufferFromHTTP. */ -class ReadIndirectBufferFromWebServer : public BufferWithOwnMemory +class ReadIndirectBufferFromWebServer : public SeekableReadBuffer { public: explicit ReadIndirectBufferFromWebServer( diff --git a/src/IO/ReadBufferFromRemoteFS.cpp b/src/IO/ReadBufferFromRemoteFS.cpp index 63403ba0599..06b663dba1c 100644 --- a/src/IO/ReadBufferFromRemoteFS.cpp +++ b/src/IO/ReadBufferFromRemoteFS.cpp @@ -72,16 +72,9 @@ bool ReadBufferFromRemoteFS::nextImpl() bool ReadBufferFromRemoteFS::read() { - if (check) - { - assert(!internal_buffer.empty()); - assert(working_buffer.begin() != nullptr); - } /// Transfer current position and working_buffer to actual ReadBuffer swap(*current_buf); /// Position and working_buffer will be updated in next() call - if (check) - assert(current_buf->buffer().begin() != nullptr); auto result = current_buf->next(); /// Assign result to current buffer. swap(*current_buf); @@ -93,8 +86,9 @@ bool ReadBufferFromRemoteFS::read() } -size_t ReadBufferFromRemoteFS::fetch(size_t offset) +size_t ReadBufferFromRemoteFS::readInto(char * data, size_t size, size_t offset) { + set(data, size); absolute_position = offset; auto result = nextImpl(); if (result) @@ -114,11 +108,9 @@ off_t ReadBufferFromRemoteFS::seek(off_t offset, int whence) } -void ReadBufferFromRemoteFS::reset(bool reset_inner_buf) +void ReadBufferFromRemoteFS::reset() { - if (reset_inner_buf) - current_buf.reset(); - // BufferBase::set(nullptr, 0, 0); + current_buf.reset(); } } diff --git a/src/IO/ReadBufferFromRemoteFS.h b/src/IO/ReadBufferFromRemoteFS.h index da522e36020..f680093c516 100644 --- a/src/IO/ReadBufferFromRemoteFS.h +++ b/src/IO/ReadBufferFromRemoteFS.h @@ -20,11 +20,10 @@ public: String getFileName() const override { return metadata.metadata_file_path; } - void reset(bool reset_inner_buf = false); + void reset(); - bool check = false; protected: - size_t fetch(size_t offset); + size_t readInto(char * data, size_t size, size_t offset); virtual SeekableReadBufferPtr createReadBuffer(const String & path) const = 0; diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index d99f565419d..8b815021186 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -55,7 +55,6 @@ bool ReadBufferFromS3::nextImpl() impl->position() = position(); assert(!impl->hasPendingData()); } - } else { @@ -63,13 +62,12 @@ bool ReadBufferFromS3::nextImpl() impl = initialize(); next_result = impl->hasPendingData(); } + if (use_external_buffer) { - // assert(!impl->buffer().begin() || impl->buffer().begin() != working_buffer.begin()); impl->set(internal_buffer.begin(), internal_buffer.size()); assert(working_buffer.begin() != nullptr); assert(!internal_buffer.empty()); - // impl->BufferBase::set(nullptr, 0, 0); } auto sleep_time_with_backoff_milliseconds = std::chrono::milliseconds(100); @@ -105,11 +103,9 @@ bool ReadBufferFromS3::nextImpl() impl = initialize(); if (use_external_buffer) { - // assert(!impl->buffer().begin() || impl->buffer().begin() != working_buffer.begin()); impl->set(internal_buffer.begin(), internal_buffer.size()); assert(working_buffer.begin() != nullptr); assert(!internal_buffer.empty()); - // impl->BufferBase::set(nullptr, 0, 0); } next_result = impl->hasPendingData(); } diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 2f0d598cd6e..2cf2c587922 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -151,6 +151,9 @@ namespace detail } } + private: + bool use_external_buffer; + public: using NextCallback = std::function; using OutStreamCallback = std::function; @@ -163,7 +166,8 @@ namespace detail const Poco::Net::HTTPBasicCredentials & credentials_ = {}, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, HTTPHeaderEntries http_header_entries_ = {}, - const RemoteHostFilter & remote_host_filter_ = {}) + const RemoteHostFilter & remote_host_filter_ = {}, + bool use_external_buffer_ = false) : ReadBuffer(nullptr, 0) , uri {uri_} , method {!method_.empty() ? method_ : out_stream_callback_ ? Poco::Net::HTTPRequest::HTTP_POST : Poco::Net::HTTPRequest::HTTP_GET} @@ -172,6 +176,7 @@ namespace detail , credentials {credentials_} , http_header_entries {http_header_entries_} , remote_host_filter {remote_host_filter_} + , use_external_buffer {use_external_buffer_} { Poco::Net::HTTPResponse response; @@ -205,10 +210,22 @@ namespace detail { if (next_callback) next_callback(count()); - if (!working_buffer.empty()) - impl->position() = position(); + + if (use_external_buffer) + { + impl->set(internal_buffer.begin(), internal_buffer.size()); + assert(working_buffer.begin() != nullptr); + assert(!internal_buffer.empty()); + } + else + { + if (!working_buffer.empty()) + impl->position() = position(); + } + if (!impl->next()) return false; + internal_buffer = impl->buffer(); working_buffer = internal_buffer; return true; @@ -274,9 +291,11 @@ public: const Poco::Net::HTTPBasicCredentials & credentials_ = {}, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, const HTTPHeaderEntries & http_header_entries_ = {}, - const RemoteHostFilter & remote_host_filter_ = {}) + const RemoteHostFilter & remote_host_filter_ = {}, + bool use_external_buffer_ = false) : Parent(std::make_shared(uri_, timeouts, max_redirects), - uri_, method_, out_stream_callback_, credentials_, buffer_size_, http_header_entries_, remote_host_filter_) + uri_, method_, out_stream_callback_, credentials_, buffer_size_, + http_header_entries_, remote_host_filter_, use_external_buffer_) { } }; diff --git a/src/IO/ThreadPoolRemoteFSReader.cpp b/src/IO/ThreadPoolRemoteFSReader.cpp index 726e7b1dade..2fbf6a4ae8b 100644 --- a/src/IO/ThreadPoolRemoteFSReader.cpp +++ b/src/IO/ThreadPoolRemoteFSReader.cpp @@ -28,9 +28,9 @@ namespace CurrentMetrics namespace DB { -size_t ThreadPoolRemoteFSReader::RemoteFSFileDescriptor::fetch(size_t offset) +size_t ThreadPoolRemoteFSReader::RemoteFSFileDescriptor::readInto(char * data, size_t size, size_t offset) { - return reader->fetch(offset); + return reader->readInto(data, size, offset); } @@ -49,7 +49,7 @@ std::future ThreadPoolRemoteFSReader::submit(Reques auto * remote_fs_fd = assert_cast(request.descriptor.get()); Stopwatch watch(CLOCK_MONOTONIC); - auto bytes_read = remote_fs_fd->fetch(request.offset); + auto bytes_read = remote_fs_fd->readInto(request.buf, request.size, request.offset); watch.stop(); ProfileEvents::increment(ProfileEvents::RemoteFSReadMicroseconds, watch.elapsedMicroseconds()); diff --git a/src/IO/ThreadPoolRemoteFSReader.h b/src/IO/ThreadPoolRemoteFSReader.h index b82c17be47c..14ba8ad468d 100644 --- a/src/IO/ThreadPoolRemoteFSReader.h +++ b/src/IO/ThreadPoolRemoteFSReader.h @@ -8,7 +8,6 @@ namespace DB { - class ReadBufferFromRemoteFS; class ThreadPoolRemoteFSReader : public IAsynchronousReader @@ -31,7 +30,7 @@ struct ThreadPoolRemoteFSReader::RemoteFSFileDescriptor : public IFileDescriptor public: RemoteFSFileDescriptor(std::shared_ptr reader_) : reader(reader_) {} - size_t fetch(size_t offset); + size_t readInto(char * data, size_t size, size_t offset); private: std::shared_ptr reader; From 85d3379ebf4e6ae2162013b05c8d26f028463c0b Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 30 Sep 2021 23:12:21 +0300 Subject: [PATCH 078/919] Bump CI From 6f49ebe6e7dffdb4c998a5dac9d499d70aeb956e Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 30 Sep 2021 23:48:22 +0300 Subject: [PATCH 079/919] Bump CI From 8c20a184ca4501d4f2713b2b4ce162f23b71a5ac Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 1 Oct 2021 05:06:19 +0000 Subject: [PATCH 080/919] optimize event handle --- .../FileLog/FileLogDirectoryWatcher.cpp | 46 ++++++++----------- .../FileLog/FileLogDirectoryWatcher.h | 5 +- src/Storages/FileLog/StorageFileLog.cpp | 26 +++++------ 3 files changed, 35 insertions(+), 42 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 5bec4fff650..0055d5d3eb1 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -39,53 +39,47 @@ const std::string & FileLogDirectoryWatcher::getPath() const void FileLogDirectoryWatcher::onItemAdded(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); - DirEvent de; - de.callback = "onItemAdded"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + EventInfo info{ev.event, "onItemAdded"}; + events.emplace(ev.item.path(), info); } void FileLogDirectoryWatcher::onItemRemoved(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); - DirEvent de; - de.callback = "onItemRemoved"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + EventInfo info{ev.event, "onItemRemoved"}; + events.emplace(ev.item.path(), info); } - +/// Optimize for MODIFY event, during a streamToViews period, since the log files +/// are append only, there are may a lots of MODIFY events produced for one file. +/// For example, appending 10000 logs into one file will result in 10000 MODIFY event. +/// So, if we record all of these events, it will use a lot of memory, and then we +/// need to handle it one by one in StorageFileLog::updateFileInfos, this is unnecessary +/// because it is equal to just record and handle one MODIY event void FileLogDirectoryWatcher::onItemModified(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); - DirEvent de; - de.callback = "onItemModified"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + auto event_path = ev.item.path(); + /// Already have MODIFY event for this file + if (auto it = events.find(event_path); it != events.end() && it->second.type == ev.event) + return; + EventInfo info{ev.event, "onItemModified"}; + events.emplace(event_path, info); } void FileLogDirectoryWatcher::onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); - DirEvent de; - de.callback = "onItemMovedFrom"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + EventInfo info{ev.event, "onItemMovedFrom"}; + events.emplace(ev.item.path(), info); } void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); - DirEvent de; - de.callback = "onItemMovedTo"; - de.path = ev.item.path(); - de.type = ev.event; - events.push_back(de); + EventInfo info{ev.event, "onItemMovedTo"}; + events.emplace(ev.item.path(), info); } void FileLogDirectoryWatcher::onError(const Poco::Exception & e) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index d21d3d1e1ce..e8ce1a2bbd5 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -12,14 +12,13 @@ class FileLogDirectoryWatcher { public: - struct DirEvent + struct EventInfo { Poco::DirectoryWatcher::DirectoryEventType type; std::string callback; - std::string path; }; - using Events = std::vector; + using Events = std::unordered_map; struct Error { diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index dca7a678678..fe5562d72fb 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -691,17 +691,17 @@ bool StorageFileLog::updateFileInfos() auto events = directory_watch->getEventsAndReset(); - for (const auto & event : events) + for (const auto & [event_path, event_info] : events) { - switch (event.type) + switch (event_info.type) { case Poco::DirectoryWatcher::DW_ITEM_ADDED: { - LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); - if (std::filesystem::is_regular_file(event.path)) + LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); + if (std::filesystem::is_regular_file(event_path)) { - auto file_name = std::filesystem::path(event.path).filename(); - auto inode = getInode(event.path); + auto file_name = std::filesystem::path(event_path).filename(); + auto inode = getInode(event_path); file_infos.file_names.push_back(file_name); @@ -713,8 +713,8 @@ bool StorageFileLog::updateFileInfos() case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: { - String file_name = std::filesystem::path(event.path).filename(); - LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); + String file_name = std::filesystem::path(event_path).filename(); + LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); /// When new file added and appended, it has two event: DW_ITEM_ADDED /// and DW_ITEM_MODIFIED, since the order of these two events in the /// sequence is uncentain, so we may can not find it in file_infos, just @@ -729,8 +729,8 @@ bool StorageFileLog::updateFileInfos() case Poco::DirectoryWatcher::DW_ITEM_REMOVED: case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: { - String file_name = std::filesystem::path(event.path).filename(); - LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); + String file_name = std::filesystem::path(event_path).filename(); + LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) { it->second.status = FileStatus::REMOVED; @@ -739,11 +739,11 @@ bool StorageFileLog::updateFileInfos() } case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: { - auto file_name = std::filesystem::path(event.path).filename(); - LOG_TRACE(log, "New event {} watched, path: {}", event.callback, event.path); + auto file_name = std::filesystem::path(event_path).filename(); + LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); file_infos.file_names.push_back(file_name); - auto inode = getInode(event.path); + auto inode = getInode(event_path); file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); /// File has been renamed, we should also rename meta file From ac9af1ca387d19250f241a971ef96c039687df4c Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 1 Oct 2021 08:24:57 +0000 Subject: [PATCH 081/919] fix wrong offset --- src/Storages/FileLog/ReadBufferFromFileLog.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index e161c69a960..48947bb6304 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -109,11 +109,14 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ Record record; while (read_records_size < need_records_size && static_cast(file_ctx.reader.tellg()) < file_meta.last_open_end) { + /// Need to get offset before reading record from stream + record.offset = file_ctx.reader.tellg(); + record.file_name = file_name; + StorageFileLog::assertStreamGood(file_ctx.reader); std::getline(file_ctx.reader, record.data); - record.file_name = file_name; - record.offset = file_ctx.reader.tellg(); + new_records.emplace_back(record); ++read_records_size; } From 55972db8668b1be2a65ef857590b5c9376961b42 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 1 Oct 2021 11:38:58 +0300 Subject: [PATCH 082/919] Small fix and more comments --- ...ynchronousReadIndirectBufferFromRemoteFS.h | 2 ++ src/Disks/ReadIndirectBufferFromWebServer.cpp | 27 +++++++++++++------ src/IO/ReadBufferFromS3.cpp | 13 +++++++++ src/IO/ReadWriteBufferFromHTTP.h | 12 +++++++++ 4 files changed, 46 insertions(+), 8 deletions(-) diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h index 1095a22091f..afe58ae1c6a 100644 --- a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -24,6 +24,8 @@ namespace DB * Buffers chain for diskWeb: * AsynchronousIndirectReadBufferFromRemoteFS -> ReadBufferFromRemoteFS -> * -> ReadIndirectBufferFromWebServer -> ReadBufferFromHttp -> ReadBufferFromIStream. +* +* We pass either `memory` or `prefetch_buffer` through all this chain and return it back. */ class AsynchronousReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase { diff --git a/src/Disks/ReadIndirectBufferFromWebServer.cpp b/src/Disks/ReadIndirectBufferFromWebServer.cpp index f5ef1efae02..02a38c1d546 100644 --- a/src/Disks/ReadIndirectBufferFromWebServer.cpp +++ b/src/Disks/ReadIndirectBufferFromWebServer.cpp @@ -71,20 +71,31 @@ bool ReadIndirectBufferFromWebServer::nextImpl() if (impl) { - if (!use_external_buffer) + if (use_external_buffer) { + /** + * use_external_buffer -- means we read into the buffer which + * was passed to us from somewhere else. We do not check whether + * previously returned buffer was read or not, because this branch + * means we are prefetching data, each nextImpl() call we can fill + * a different buffer. + */ + impl->set(internal_buffer.begin(), internal_buffer.size()); + assert(working_buffer.begin() != nullptr); + assert(!internal_buffer.empty()); + } + else + { + /** + * impl was initialized before, pass position() to it to make + * sure there is no pending data which was not read, becuase + * this branch means we read sequentially. + */ impl->position() = position(); assert(!impl->hasPendingData()); } } - if (use_external_buffer) - { - impl->set(internal_buffer.begin(), internal_buffer.size()); - assert(working_buffer.begin() != nullptr); - assert(!internal_buffer.empty()); - } - WriteBufferFromOwnString error_msg; for (size_t i = 0; (i < max_tries) && !successful_read && !next_result; ++i) { diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 8b815021186..9dbe8cfd578 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -52,6 +52,13 @@ bool ReadBufferFromS3::nextImpl() { if (!use_external_buffer) { + /** + * use_external_buffer -- means we read into the buffer which + * was passed to us from somewhere else. We do not check whether + * previously returned buffer was read or not, because this branch + * means we are prefetching data, each nextImpl() call we can fill + * a different buffer. + */ impl->position() = position(); assert(!impl->hasPendingData()); } @@ -65,6 +72,12 @@ bool ReadBufferFromS3::nextImpl() if (use_external_buffer) { + /** + * use_external_buffer -- means we read into the buffer which + * was passed to us from somewhere else. We do not check whether + * previously returned buffer was read or not, because this branch + * means we are prefetching data. + */ impl->set(internal_buffer.begin(), internal_buffer.size()); assert(working_buffer.begin() != nullptr); assert(!internal_buffer.empty()); diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 2cf2c587922..33edcdb2e42 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -213,12 +213,24 @@ namespace detail if (use_external_buffer) { + /** + * use_external_buffer -- means we read into the buffer which + * was passed to us from somewhere else. We do not check whether + * previously returned buffer was read or not, because this branch + * means we are prefetching data, each nextImpl() call we can fill + * a different buffer. + */ impl->set(internal_buffer.begin(), internal_buffer.size()); assert(working_buffer.begin() != nullptr); assert(!internal_buffer.empty()); } else { + /** + * impl was initialized before, pass position() to it to make + * sure there is no pending data which was not read, becuase + * this branch means we read sequentially. + */ if (!working_buffer.empty()) impl->position() = position(); } From 6eeade23fa4025ad34c62b1bb8805a8255532a1d Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 1 Oct 2021 08:29:19 +0000 Subject: [PATCH 083/919] revert drop table immediately remove unused code fix pvs check fix --- src/Databases/DatabaseAtomic.cpp | 5 ----- src/Interpreters/DatabaseCatalog.cpp | 2 +- src/Storages/FileLog/ReadBufferFromFileLog.cpp | 4 ---- src/Storages/FileLog/StorageFileLog.h | 12 +++++------- src/Storages/IStorage.h | 2 -- 5 files changed, 6 insertions(+), 19 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 26f694cb5dd..5c75f6f1036 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -140,11 +140,6 @@ void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_na if (table->storesDataOnDisk()) tryRemoveSymlink(table_name); - if (table->dropTableImmediately()) - { - table->drop(); - } - /// Notify DatabaseCatalog that table was dropped. It will remove table data in background. /// Cleanup is performed outside of database to allow easily DROP DATABASE without waiting for cleanup to complete. DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop, no_delay); diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index c87a9b91640..f273f8a165d 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -898,7 +898,7 @@ void DatabaseCatalog::dropTableDataTask() void DatabaseCatalog::dropTableFinally(const TableMarkedAsDropped & table) { - if (table.table && !table.table->dropTableImmediately()) + if (table.table) { table.table->drop(); } diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 48947bb6304..04af2f829ba 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -10,10 +10,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_READ_FROM_ISTREAM; -} ReadBufferFromFileLog::ReadBufferFromFileLog( StorageFileLog & storage_, diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 7384543fa36..f181ad570c3 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -49,8 +49,6 @@ public: void drop() override; - bool dropTableImmediately() const override { return true; } - const auto & getFormatName() const { return format_name; } enum class FileStatus @@ -64,15 +62,15 @@ public: struct FileContext { FileStatus status = FileStatus::OPEN; - UInt64 inode; - std::ifstream reader; + UInt64 inode{}; + std::ifstream reader{}; }; struct FileMeta { String file_name; - UInt64 last_writen_position = 0; - UInt64 last_open_end = 0; + UInt64 last_writen_position{}; + UInt64 last_open_end{}; }; using InodeToFileMeta = std::unordered_map; @@ -137,7 +135,7 @@ private: /// If path argument of the table is a regular file, it equals to user_files_path /// otherwise, it equals to user_files_path/ + path_argument/, e.g. path String root_data_path; - /// Database meta_path/ + .table_name/ + /// relative_data_path/ + table_name/ String root_meta_path; FileInfos file_infos; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index f64437e1889..6ce17552ba1 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -578,8 +578,6 @@ public: /// Does not takes underlying Storage (if any) into account. virtual std::optional lifetimeBytes() const { return {}; } - virtual bool dropTableImmediately() const { return false; } - private: /// Lock required for alter queries (lockForAlter). Always taken for write /// (actually can be replaced with std::mutex, but for consistency we use From b336947f77feb0fabc82ddc1522d9639f8ee3947 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Fri, 1 Oct 2021 16:06:10 +0300 Subject: [PATCH 084/919] new example --- docs/en/operations/system-tables/replicas.md | 68 ++++++++++---------- 1 file changed, 35 insertions(+), 33 deletions(-) diff --git a/docs/en/operations/system-tables/replicas.md b/docs/en/operations/system-tables/replicas.md index 7bca3e78ff9..c959100b6e4 100644 --- a/docs/en/operations/system-tables/replicas.md +++ b/docs/en/operations/system-tables/replicas.md @@ -13,41 +13,43 @@ FORMAT Vertical ``` ``` text +Query id: dc6dcbcb-dc28-4df9-ae27-4354f5b3b13e + Row 1: ────── -database: merge -table: visits -engine: ReplicatedCollapsingMergeTree -is_leader: 1 -can_become_leader: 1 -is_readonly: 0 -is_session_expired: 0 -future_parts: 1 -parts_to_check: 0 -zookeeper_path: /clickhouse/tables/01-06/visits -replica_name: example01-06-1.yandex.ru -replica_path: /clickhouse/tables/01-06/visits/replicas/example01-06-1.yandex.ru -columns_version: 9 -queue_size: 1 -inserts_in_queue: 0 -merges_in_queue: 1 -part_mutations_in_queue: 0 -queue_oldest_time: 2020-02-20 08:34:30 -inserts_oldest_time: 1970-01-01 00:00:00 -merges_oldest_time: 2020-02-20 08:34:30 -part_mutations_oldest_time: 1970-01-01 00:00:00 -oldest_part_to_get: -oldest_part_to_merge_to: 20200220_20284_20840_7 -oldest_part_to_mutate_to: -log_max_index: 596273 -log_pointer: 596274 -last_queue_update: 2020-02-20 08:34:32 -absolute_delay: 0 -total_replicas: 2 -active_replicas: 2 -last_queue_update_exception: -zookeeper_exception: -replica_is_active: {'example01-06-1.yandex.ru':1} +database: default +table: test +engine: ReplicatedMergeTree +is_leader: 1 +can_become_leader: 1 +is_readonly: 0 +is_session_expired: 0 +future_parts: 0 +parts_to_check: 0 +zookeeper_path: /test/t +replica_name: 1 +replica_path: /test/t/replicas/1 +columns_version: -1 +queue_size: 0 +inserts_in_queue: 0 +merges_in_queue: 0 +part_mutations_in_queue: 0 +queue_oldest_time: 1970-01-01 03:00:00 +inserts_oldest_time: 1970-01-01 03:00:00 +merges_oldest_time: 1970-01-01 03:00:00 +part_mutations_oldest_time: 1970-01-01 03:00:00 +oldest_part_to_get: +oldest_part_to_merge_to: +oldest_part_to_mutate_to: +log_max_index: 0 +log_pointer: 0 +last_queue_update: 1970-01-01 03:00:00 +absolute_delay: 0 +total_replicas: 1 +active_replicas: 1 +last_queue_update_exception: +zookeeper_exception: +replica_is_active: {'1':1} ``` Columns: From 5bb5e04fb3b4c96b3a0e0d2d44dc0359d4958a67 Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 1 Oct 2021 14:47:05 +0000 Subject: [PATCH 085/919] fix assert --- src/Storages/FileLog/StorageFileLog.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index fe5562d72fb..ded3016c123 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -83,7 +83,6 @@ StorageFileLog::StorageFileLog( #ifndef NDEBUG assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); - assert(file_infos.file_names.size() == file_infos.inode_by_name.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); #endif @@ -670,7 +669,6 @@ bool StorageFileLog::updateFileInfos() { #ifndef NDEBUG assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); - assert(file_infos.file_names.size() == file_infos.inode_by_name.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); assert(file_infos.file_names.size() == 1); #endif @@ -784,7 +782,6 @@ bool StorageFileLog::updateFileInfos() /// These file infos should always have same size(one for one) #ifndef NDEBUG assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); - assert(file_infos.file_names.size() == file_infos.inode_by_name.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); #endif From 7ff092712b804029c5910e7d73f4211b7195a2ca Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 1 Oct 2021 15:43:22 +0300 Subject: [PATCH 086/919] Better --- src/Disks/DiskWebServer.cpp | 46 +----- src/Disks/HDFS/DiskHDFS.cpp | 46 +----- src/Disks/ReadBufferFromRemoteFSGather.cpp | 137 ++++++++++++++++ src/Disks/ReadBufferFromRemoteFSGather.h | 150 ++++++++++++++++++ src/Disks/S3/DiskS3.cpp | 45 +----- ...chronousReadIndirectBufferFromRemoteFS.cpp | 6 +- ...ynchronousReadIndirectBufferFromRemoteFS.h | 6 +- src/IO/ReadBufferFromRemoteFS.cpp | 116 -------------- src/IO/ReadBufferFromRemoteFS.h | 46 ------ .../ReadBufferFromWebServer.cpp} | 14 +- .../ReadBufferFromWebServer.h} | 6 +- .../ReadIndirectBufferFromRemoteFS.cpp | 34 ++-- .../ReadIndirectBufferFromRemoteFS.h | 14 +- src/IO/ThreadPoolRemoteFSReader.cpp | 4 +- src/IO/ThreadPoolRemoteFSReader.h | 6 +- .../WriteIndirectBufferFromRemoteFS.cpp | 0 .../WriteIndirectBufferFromRemoteFS.h | 0 17 files changed, 355 insertions(+), 321 deletions(-) create mode 100644 src/Disks/ReadBufferFromRemoteFSGather.cpp create mode 100644 src/Disks/ReadBufferFromRemoteFSGather.h rename src/{Disks => IO}/AsynchronousReadIndirectBufferFromRemoteFS.cpp (95%) rename src/{Disks => IO}/AsynchronousReadIndirectBufferFromRemoteFS.h (90%) delete mode 100644 src/IO/ReadBufferFromRemoteFS.cpp delete mode 100644 src/IO/ReadBufferFromRemoteFS.h rename src/{Disks/ReadIndirectBufferFromWebServer.cpp => IO/ReadBufferFromWebServer.cpp} (92%) rename src/{Disks/ReadIndirectBufferFromWebServer.h => IO/ReadBufferFromWebServer.h} (84%) rename src/{Disks => IO}/ReadIndirectBufferFromRemoteFS.cpp (61%) rename src/{Disks => IO}/ReadIndirectBufferFromRemoteFS.h (57%) rename src/{Disks => IO}/WriteIndirectBufferFromRemoteFS.cpp (100%) rename src/{Disks => IO}/WriteIndirectBufferFromRemoteFS.h (100%) diff --git a/src/Disks/DiskWebServer.cpp b/src/Disks/DiskWebServer.cpp index 9c4747aefdb..968a67ceead 100644 --- a/src/Disks/DiskWebServer.cpp +++ b/src/Disks/DiskWebServer.cpp @@ -4,15 +4,16 @@ #include #include -#include -#include +#include #include +#include #include #include #include #include -#include +#include +#include #include @@ -107,43 +108,6 @@ private: }; -class ReadBufferFromWebServer final : public ReadBufferFromRemoteFS -{ -public: - ReadBufferFromWebServer( - const String & uri_, - RemoteMetadata metadata_, - ContextPtr context_, - size_t buf_size_, - size_t backoff_threshold_, - size_t max_tries_, - size_t threadpool_read_) - : ReadBufferFromRemoteFS(metadata_) - , uri(uri_) - , context(context_) - , buf_size(buf_size_) - , backoff_threshold(backoff_threshold_) - , max_tries(max_tries_) - , threadpool_read(threadpool_read_) - { - } - - SeekableReadBufferPtr createReadBuffer(const String & path) const override - { - return std::make_unique( - fs::path(uri) / path, context, buf_size, backoff_threshold, max_tries, threadpool_read); - } - -private: - String uri; - ContextPtr context; - size_t buf_size; - size_t backoff_threshold; - size_t max_tries; - bool threadpool_read; -}; - - DiskWebServer::DiskWebServer( const String & disk_name_, const String & url_, @@ -204,7 +168,7 @@ std::unique_ptr DiskWebServer::readFile(const String & p bool threadpool_read = read_settings.remote_fs_method == RemoteFSReadMethod::read_threadpool; - auto web_impl = std::make_unique(url, meta, getContext(), + auto web_impl = std::make_unique(url, meta, getContext(), read_settings.remote_fs_buffer_size, read_settings.remote_fs_backoff_threshold, read_settings.remote_fs_backoff_max_tries, threadpool_read); diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index 50a8806fa7b..ed5ae00539d 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -1,13 +1,12 @@ #include -#include -#include -#include -#include -#include -#include -#include #include +#include +#include +#include +#include +#include +#include namespace DB @@ -49,37 +48,6 @@ private: }; -/// Reads data from HDFS using stored paths in metadata. -class ReadIndirectBufferFromHDFS final : public ReadBufferFromRemoteFS -{ -public: - ReadIndirectBufferFromHDFS( - const Poco::Util::AbstractConfiguration & config_, - const String & hdfs_uri_, - DiskHDFS::Metadata metadata_, - size_t buf_size_) - : ReadBufferFromRemoteFS(metadata_) - , config(config_) - , buf_size(buf_size_) - { - const size_t begin_of_path = hdfs_uri_.find('/', hdfs_uri_.find("//") + 2); - hdfs_directory = hdfs_uri_.substr(begin_of_path); - hdfs_uri = hdfs_uri_.substr(0, begin_of_path); - } - - SeekableReadBufferPtr createReadBuffer(const String & path) const override - { - return std::make_unique(hdfs_uri, hdfs_directory + path, config, buf_size); - } - -private: - const Poco::Util::AbstractConfiguration & config; - String hdfs_uri; - String hdfs_directory; - size_t buf_size; -}; - - DiskHDFS::DiskHDFS( const String & disk_name_, const String & hdfs_root_path_, @@ -103,7 +71,7 @@ std::unique_ptr DiskHDFS::readFile(const String & path, "Read from file by path: {}. Existing HDFS objects: {}", backQuote(metadata_path + path), metadata.remote_fs_objects.size()); - auto hdfs_impl = std::make_unique(config, remote_fs_root_path, metadata, read_settings.remote_fs_buffer_size); + auto hdfs_impl = std::make_unique(config, remote_fs_root_path, metadata, read_settings.remote_fs_buffer_size); if (read_settings.remote_fs_method == RemoteFSReadMethod::read_threadpool) { diff --git a/src/Disks/ReadBufferFromRemoteFSGather.cpp b/src/Disks/ReadBufferFromRemoteFSGather.cpp new file mode 100644 index 00000000000..c923e484ee0 --- /dev/null +++ b/src/Disks/ReadBufferFromRemoteFSGather.cpp @@ -0,0 +1,137 @@ +#include "ReadBufferFromRemoteFSGather.h" + +#include +#include +#include + +#if USE_AWS_S3 +#include +#endif + +#if USE_HDFS +#include +#endif + +#include +#include + +namespace fs = std::filesystem; + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_SEEK_THROUGH_FILE; +} + + +#if USE_AWS_S3 +SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const String & path) const +{ + return std::make_unique(client_ptr, bucket, fs::path(metadata.remote_fs_root_path) / path, max_single_read_retries, buf_size, threadpool_read); +} +#endif + + +SeekableReadBufferPtr ReadBufferFromWebServerGather::createImplementationBuffer(const String & path) const +{ + return std::make_unique(fs::path(uri) / path, context, buf_size, backoff_threshold, max_tries, threadpool_read); +} + + +#if USE_HDFS +SeekableReadBufferPtr ReadBufferFromHDFSGather::createImplementationBuffer(const String & path) const +{ + return std::make_unique(hdfs_uri, fs::path(hdfs_directory) / path, config, buf_size); +} +#endif + + +ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather(const RemoteMetadata & metadata_) + : ReadBuffer(nullptr, 0) + , metadata(metadata_) +{ +} + + +size_t ReadBufferFromRemoteFSGather::readInto(char * data, size_t size, size_t offset) +{ + set(data, size); + absolute_position = offset; + auto result = nextImpl(); + if (result) + return working_buffer.size(); + return 0; +} + + +SeekableReadBufferPtr ReadBufferFromRemoteFSGather::initialize() +{ + /// One clickhouse file can be split into multiple files in remote fs. + auto current_buf_offset = absolute_position; + for (size_t i = 0; i < metadata.remote_fs_objects.size(); ++i) + { + current_buf_idx = i; + const auto & [file_path, size] = metadata.remote_fs_objects[i]; + + if (size > current_buf_offset) + { + auto buf = createImplementationBuffer(file_path); + buf->seek(current_buf_offset, SEEK_SET); + return buf; + } + + current_buf_offset -= size; + } + return nullptr; +} + + +bool ReadBufferFromRemoteFSGather::nextImpl() +{ + /// Find first available buffer that fits to given offset. + if (!current_buf) + current_buf = initialize(); + + /// If current buffer has remaining data - use it. + if (current_buf) + { + if (readImpl()) + return true; + } + else + return false; + + /// If there is no available buffers - nothing to read. + if (current_buf_idx + 1 >= metadata.remote_fs_objects.size()) + return false; + + ++current_buf_idx; + + const auto & path = metadata.remote_fs_objects[current_buf_idx].first; + current_buf = createImplementationBuffer(path); + + return readImpl(); +} + + +bool ReadBufferFromRemoteFSGather::readImpl() +{ + swap(*current_buf); + auto result = current_buf->next(); + swap(*current_buf); + + if (result) + absolute_position += working_buffer.size(); + + return result; +} + + +void ReadBufferFromRemoteFSGather::reset() +{ + current_buf.reset(); +} + +} diff --git a/src/Disks/ReadBufferFromRemoteFSGather.h b/src/Disks/ReadBufferFromRemoteFSGather.h new file mode 100644 index 00000000000..80bdfe4775d --- /dev/null +++ b/src/Disks/ReadBufferFromRemoteFSGather.h @@ -0,0 +1,150 @@ +#pragma once + +#if !defined(ARCADIA_BUILD) +#include +#endif + +#include +#include + +namespace Aws +{ +namespace S3 +{ +class S3Client; +} +} + +namespace DB +{ + +class ReadBufferFromRemoteFSGather : public ReadBuffer +{ +friend class ThreadPoolRemoteFSReader; +friend class ReadIndirectBufferFromRemoteFS; + +public: + explicit ReadBufferFromRemoteFSGather(const RemoteMetadata & metadata_); + + String getFileName() const { return metadata.metadata_file_path; } + + void reset(); + +protected: + size_t readInto(char * data, size_t size, size_t offset); + + virtual SeekableReadBufferPtr createImplementationBuffer(const String & path) const = 0; + + RemoteMetadata metadata; + +private: + bool nextImpl() override; + + SeekableReadBufferPtr initialize(); + + bool readImpl(); + + SeekableReadBufferPtr current_buf; + + size_t current_buf_idx = 0; + + size_t absolute_position = 0; +}; + + +#if USE_AWS_S3 +/// Reads data from S3 using stored paths in metadata. +class ReadBufferFromS3Gather final : public ReadBufferFromRemoteFSGather +{ +public: + ReadBufferFromS3Gather( + std::shared_ptr client_ptr_, + const String & bucket_, + IDiskRemote::Metadata metadata_, + size_t max_single_read_retries_, + size_t buf_size_, + bool threadpool_read_ = false) + : ReadBufferFromRemoteFSGather(metadata_) + , client_ptr(std::move(client_ptr_)) + , bucket(bucket_) + , max_single_read_retries(max_single_read_retries_) + , buf_size(buf_size_) + , threadpool_read(threadpool_read_) + { + } + + SeekableReadBufferPtr createImplementationBuffer(const String & path) const override; + +private: + std::shared_ptr client_ptr; + String bucket; + UInt64 max_single_read_retries; + size_t buf_size; + bool threadpool_read; +}; +#endif + + +class ReadBufferFromWebServerGather final : public ReadBufferFromRemoteFSGather +{ +public: + ReadBufferFromWebServerGather( + const String & uri_, + RemoteMetadata metadata_, + ContextPtr context_, + size_t buf_size_, + size_t backoff_threshold_, + size_t max_tries_, + size_t threadpool_read_) + : ReadBufferFromRemoteFSGather(metadata_) + , uri(uri_) + , context(context_) + , buf_size(buf_size_) + , backoff_threshold(backoff_threshold_) + , max_tries(max_tries_) + , threadpool_read(threadpool_read_) + { + } + + SeekableReadBufferPtr createImplementationBuffer(const String & path) const override; + +private: + String uri; + ContextPtr context; + size_t buf_size; + size_t backoff_threshold; + size_t max_tries; + bool threadpool_read; +}; + + +#if USE_HDFS +/// Reads data from HDFS using stored paths in metadata. +class ReadBufferFromHDFSGather final : public ReadBufferFromRemoteFSGather +{ +public: + ReadBufferFromHDFSGather( + const Poco::Util::AbstractConfiguration & config_, + const String & hdfs_uri_, + IDiskRemote::Metadata metadata_, + size_t buf_size_) + : ReadBufferFromRemoteFSGather(metadata_) + , config(config_) + , buf_size(buf_size_) + { + const size_t begin_of_path = hdfs_uri_.find('/', hdfs_uri_.find("//") + 2); + hdfs_directory = hdfs_uri_.substr(begin_of_path); + hdfs_uri = hdfs_uri_.substr(0, begin_of_path); + } + + SeekableReadBufferPtr createImplementationBuffer(const String & path) const override; + +private: + const Poco::Util::AbstractConfiguration & config; + String hdfs_uri; + String hdfs_directory; + size_t buf_size; +}; +#endif + +} diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 41fb6ceb3bb..3b37f48eecb 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -16,19 +16,17 @@ #include #include -#include -#include - +#include #include - #include +#include #include #include #include #include #include -#include -#include +#include +#include #include // Y_IGNORE #include // Y_IGNORE @@ -128,39 +126,6 @@ void throwIfError(const Aws::Utils::Outcome & response) } } -/// Reads data from S3 using stored paths in metadata. -class ReadIndirectBufferFromS3 final : public ReadBufferFromRemoteFS -{ -public: - ReadIndirectBufferFromS3( - std::shared_ptr client_ptr_, - const String & bucket_, - DiskS3::Metadata metadata_, - size_t max_single_read_retries_, - size_t buf_size_, - bool threadpool_read_ = false) - : ReadBufferFromRemoteFS(metadata_) - , client_ptr(std::move(client_ptr_)) - , bucket(bucket_) - , max_single_read_retries(max_single_read_retries_) - , buf_size(buf_size_) - , threadpool_read(threadpool_read_) - { - } - - SeekableReadBufferPtr createReadBuffer(const String & path) const override - { - return std::make_unique(client_ptr, bucket, fs::path(metadata.remote_fs_root_path) / path, max_single_read_retries, buf_size, threadpool_read); - } - -private: - std::shared_ptr client_ptr; - const String & bucket; - UInt64 max_single_read_retries; - size_t buf_size; - bool threadpool_read; -}; - DiskS3::DiskS3( String name_, String bucket_, @@ -236,7 +201,7 @@ std::unique_ptr DiskS3::readFile(const String & path, co bool threadpool_read = read_settings.remote_fs_method == RemoteFSReadMethod::read_threadpool; - auto s3_impl = std::make_unique( + auto s3_impl = std::make_unique( settings->client, bucket, metadata, settings->s3_max_single_read_retries, read_settings.remote_fs_buffer_size, threadpool_read); diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp similarity index 95% rename from src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp rename to src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp index c6e73c369dd..f12bf0d1760 100644 --- a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -1,10 +1,8 @@ #include "AsynchronousReadIndirectBufferFromRemoteFS.h" -#include -#include -#include #include #include +#include namespace CurrentMetrics @@ -27,7 +25,7 @@ namespace ErrorCodes AsynchronousReadIndirectBufferFromRemoteFS::AsynchronousReadIndirectBufferFromRemoteFS( AsynchronousReaderPtr reader_, Int32 priority_, - std::shared_ptr impl_, size_t buf_size_) + std::shared_ptr impl_, size_t buf_size_) : ReadBufferFromFileBase(buf_size_, nullptr, 0) , reader(reader_) , priority(priority_) diff --git a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h similarity index 90% rename from src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h rename to src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h index afe58ae1c6a..bd5d37fa3da 100644 --- a/src/Disks/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -6,7 +6,7 @@ #include #include -#include +#include #include #include @@ -32,7 +32,7 @@ class AsynchronousReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase public: explicit AsynchronousReadIndirectBufferFromRemoteFS( AsynchronousReaderPtr reader_, Int32 priority_, - std::shared_ptr impl_, + std::shared_ptr impl_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE); ~AsynchronousReadIndirectBufferFromRemoteFS() override; @@ -56,7 +56,7 @@ private: Int32 priority; - std::shared_ptr impl; + std::shared_ptr impl; std::future prefetch_future; diff --git a/src/IO/ReadBufferFromRemoteFS.cpp b/src/IO/ReadBufferFromRemoteFS.cpp deleted file mode 100644 index 06b663dba1c..00000000000 --- a/src/IO/ReadBufferFromRemoteFS.cpp +++ /dev/null @@ -1,116 +0,0 @@ -#include "ReadBufferFromRemoteFS.h" - -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int CANNOT_SEEK_THROUGH_FILE; -} - - -ReadBufferFromRemoteFS::ReadBufferFromRemoteFS(const RemoteMetadata & metadata_) - : metadata(metadata_) -{ -} - - -SeekableReadBufferPtr ReadBufferFromRemoteFS::initialize() -{ - /// One clickhouse file can be split into multiple files in remote fs. - auto current_buf_offset = absolute_position; - for (size_t i = 0; i < metadata.remote_fs_objects.size(); ++i) - { - current_buf_idx = i; - const auto & [file_path, size] = metadata.remote_fs_objects[i]; - - if (size > current_buf_offset) - { - auto buf = createReadBuffer(file_path); - buf->seek(current_buf_offset, SEEK_SET); - return buf; - } - - current_buf_offset -= size; - } - return nullptr; -} - - -bool ReadBufferFromRemoteFS::nextImpl() -{ - /// Find first available buffer that fits to given offset. - if (!current_buf) - current_buf = initialize(); - - /// If current buffer has remaining data - use it. - if (current_buf) - { - if (read()) - return true; - } - else - return false; - - /// If there is no available buffers - nothing to read. - if (current_buf_idx + 1 >= metadata.remote_fs_objects.size()) - return false; - - ++current_buf_idx; - - const auto & path = metadata.remote_fs_objects[current_buf_idx].first; - current_buf = createReadBuffer(path); - - return read(); -} - - -bool ReadBufferFromRemoteFS::read() -{ - /// Transfer current position and working_buffer to actual ReadBuffer - swap(*current_buf); - /// Position and working_buffer will be updated in next() call - auto result = current_buf->next(); - /// Assign result to current buffer. - swap(*current_buf); - - if (result) - absolute_position += working_buffer.size(); - - return result; -} - - -size_t ReadBufferFromRemoteFS::readInto(char * data, size_t size, size_t offset) -{ - set(data, size); - absolute_position = offset; - auto result = nextImpl(); - if (result) - return working_buffer.size(); - return 0; -} - - -off_t ReadBufferFromRemoteFS::seek(off_t offset, int whence) -{ - if (whence != SEEK_SET) - throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Only SEEK_SET is allowed"); - - absolute_position = offset; - current_buf = initialize(); - return absolute_position; -} - - -void ReadBufferFromRemoteFS::reset() -{ - current_buf.reset(); -} - -} diff --git a/src/IO/ReadBufferFromRemoteFS.h b/src/IO/ReadBufferFromRemoteFS.h deleted file mode 100644 index f680093c516..00000000000 --- a/src/IO/ReadBufferFromRemoteFS.h +++ /dev/null @@ -1,46 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -class ReadBufferFromRemoteFS : public ReadBufferFromFileBase -{ -friend class ThreadPoolRemoteFSReader; - -public: - explicit ReadBufferFromRemoteFS(const RemoteMetadata & metadata_); - - off_t seek(off_t offset, int whence) override; - - off_t getPosition() override { return absolute_position - available(); } - - String getFileName() const override { return metadata.metadata_file_path; } - - void reset(); - -protected: - size_t readInto(char * data, size_t size, size_t offset); - - virtual SeekableReadBufferPtr createReadBuffer(const String & path) const = 0; - - RemoteMetadata metadata; - -private: - bool nextImpl() override; - - SeekableReadBufferPtr initialize(); - - bool read(); - - SeekableReadBufferPtr current_buf; - - size_t current_buf_idx = 0; - - size_t absolute_position = 0; -}; - -} diff --git a/src/Disks/ReadIndirectBufferFromWebServer.cpp b/src/IO/ReadBufferFromWebServer.cpp similarity index 92% rename from src/Disks/ReadIndirectBufferFromWebServer.cpp rename to src/IO/ReadBufferFromWebServer.cpp index 02a38c1d546..2bb616ebd84 100644 --- a/src/Disks/ReadIndirectBufferFromWebServer.cpp +++ b/src/IO/ReadBufferFromWebServer.cpp @@ -1,4 +1,4 @@ -#include "ReadIndirectBufferFromWebServer.h" +#include "ReadBufferFromWebServer.h" #include #include @@ -23,11 +23,11 @@ namespace ErrorCodes static const auto WAIT_MS = 10; -ReadIndirectBufferFromWebServer::ReadIndirectBufferFromWebServer( +ReadBufferFromWebServer::ReadBufferFromWebServer( const String & url_, ContextPtr context_, size_t buf_size_, size_t backoff_threshold_, size_t max_tries_, bool use_external_buffer_) : SeekableReadBuffer(nullptr, 0) - , log(&Poco::Logger::get("ReadIndirectBufferFromWebServer")) + , log(&Poco::Logger::get("ReadBufferFromWebServer")) , context(context_) , url(url_) , buf_size(buf_size_) @@ -38,7 +38,7 @@ ReadIndirectBufferFromWebServer::ReadIndirectBufferFromWebServer( } -std::unique_ptr ReadIndirectBufferFromWebServer::initialize() +std::unique_ptr ReadBufferFromWebServer::initialize() { Poco::URI uri(url); @@ -64,7 +64,7 @@ std::unique_ptr ReadIndirectBufferFromWebServer::initialize() } -bool ReadIndirectBufferFromWebServer::nextImpl() +bool ReadBufferFromWebServer::nextImpl() { bool next_result = false, successful_read = false; UInt16 milliseconds_to_wait = WAIT_MS; @@ -148,7 +148,7 @@ bool ReadIndirectBufferFromWebServer::nextImpl() } -off_t ReadIndirectBufferFromWebServer::seek(off_t offset_, int whence) +off_t ReadBufferFromWebServer::seek(off_t offset_, int whence) { if (impl) throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Seek is allowed only before first read attempt from the buffer"); @@ -165,7 +165,7 @@ off_t ReadIndirectBufferFromWebServer::seek(off_t offset_, int whence) } -off_t ReadIndirectBufferFromWebServer::getPosition() +off_t ReadBufferFromWebServer::getPosition() { return offset - available(); } diff --git a/src/Disks/ReadIndirectBufferFromWebServer.h b/src/IO/ReadBufferFromWebServer.h similarity index 84% rename from src/Disks/ReadIndirectBufferFromWebServer.h rename to src/IO/ReadBufferFromWebServer.h index c8ed124dfff..c741ac21fc7 100644 --- a/src/Disks/ReadIndirectBufferFromWebServer.h +++ b/src/IO/ReadBufferFromWebServer.h @@ -11,12 +11,12 @@ namespace DB /* Read buffer, which reads via http, but is used as ReadBufferFromFileBase. * Used to read files, hosted on a web server with static files. * - * Usage: ReadIndirectBufferFromRemoteFS -> SeekAvoidingReadBuffer -> ReadIndirectBufferFromWebServer -> ReadWriteBufferFromHTTP. + * Usage: ReadIndirectBufferFromRemoteFS -> SeekAvoidingReadBuffer -> ReadBufferFromWebServer -> ReadWriteBufferFromHTTP. */ -class ReadIndirectBufferFromWebServer : public SeekableReadBuffer +class ReadBufferFromWebServer : public SeekableReadBuffer { public: - explicit ReadIndirectBufferFromWebServer( + explicit ReadBufferFromWebServer( const String & url_, ContextPtr context_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE, size_t backoff_threshold_ = 10000, size_t max_tries_ = 4, diff --git a/src/Disks/ReadIndirectBufferFromRemoteFS.cpp b/src/IO/ReadIndirectBufferFromRemoteFS.cpp similarity index 61% rename from src/Disks/ReadIndirectBufferFromRemoteFS.cpp rename to src/IO/ReadIndirectBufferFromRemoteFS.cpp index 5ffa744c93d..7aca6b3157c 100644 --- a/src/Disks/ReadIndirectBufferFromRemoteFS.cpp +++ b/src/IO/ReadIndirectBufferFromRemoteFS.cpp @@ -1,5 +1,7 @@ #include "ReadIndirectBufferFromRemoteFS.h" +#include + namespace DB { @@ -11,48 +13,60 @@ namespace ErrorCodes ReadIndirectBufferFromRemoteFS::ReadIndirectBufferFromRemoteFS( - std::shared_ptr impl_) : impl(std::move(impl_)) + std::shared_ptr impl_) : impl(std::move(impl_)) { } +off_t ReadIndirectBufferFromRemoteFS::getPosition() +{ + return impl->absolute_position - available(); +} + + +String ReadIndirectBufferFromRemoteFS::getFileName() const +{ + return impl->getFileName(); +} + + off_t ReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence) { if (whence == SEEK_CUR) { /// If position within current working buffer - shift pos. - if (!working_buffer.empty() && size_t(getPosition() + offset_) < absolute_position) + if (!working_buffer.empty() && size_t(getPosition() + offset_) < impl->absolute_position) { pos += offset_; return getPosition(); } else { - absolute_position += offset_; + impl->absolute_position += offset_; } } else if (whence == SEEK_SET) { /// If position within current working buffer - shift pos. if (!working_buffer.empty() - && size_t(offset_) >= absolute_position - working_buffer.size() - && size_t(offset_) < absolute_position) + && size_t(offset_) >= impl->absolute_position - working_buffer.size() + && size_t(offset_) < impl->absolute_position) { - pos = working_buffer.end() - (absolute_position - offset_); + pos = working_buffer.end() - (impl->absolute_position - offset_); return getPosition(); } else { - absolute_position = offset_; + impl->absolute_position = offset_; } } else throw Exception("Only SEEK_SET or SEEK_CUR modes are allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); - impl->seek(absolute_position, SEEK_SET); + impl->reset(); pos = working_buffer.end(); - return absolute_position; + return impl->absolute_position; } @@ -66,7 +80,7 @@ bool ReadIndirectBufferFromRemoteFS::nextImpl() swap(*impl); if (result) - absolute_position += working_buffer.size(); + impl->absolute_position += working_buffer.size(); return result; } diff --git a/src/Disks/ReadIndirectBufferFromRemoteFS.h b/src/IO/ReadIndirectBufferFromRemoteFS.h similarity index 57% rename from src/Disks/ReadIndirectBufferFromRemoteFS.h rename to src/IO/ReadIndirectBufferFromRemoteFS.h index 4e8027e7319..abd89510882 100644 --- a/src/Disks/ReadIndirectBufferFromRemoteFS.h +++ b/src/IO/ReadIndirectBufferFromRemoteFS.h @@ -5,7 +5,6 @@ #endif #include -#include #include #include @@ -13,26 +12,25 @@ namespace DB { +class ReadBufferFromRemoteFSGather; + /// Reads data from S3/HDFS/Web using stored paths in metadata. class ReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase { -using ImplPtr = std::unique_ptr; public: - explicit ReadIndirectBufferFromRemoteFS(std::shared_ptr impl_); + explicit ReadIndirectBufferFromRemoteFS(std::shared_ptr impl_); off_t seek(off_t offset_, int whence) override; - off_t getPosition() override { return absolute_position - available(); } + off_t getPosition() override; - String getFileName() const override { return impl->getFileName(); } + String getFileName() const override; private: bool nextImpl() override; - std::shared_ptr impl; - - size_t absolute_position = 0; + std::shared_ptr impl; }; } diff --git a/src/IO/ThreadPoolRemoteFSReader.cpp b/src/IO/ThreadPoolRemoteFSReader.cpp index 2fbf6a4ae8b..6affaca2b74 100644 --- a/src/IO/ThreadPoolRemoteFSReader.cpp +++ b/src/IO/ThreadPoolRemoteFSReader.cpp @@ -8,10 +8,11 @@ #include #include -#include +#include #include #include +#include namespace ProfileEvents @@ -62,6 +63,7 @@ std::future ThreadPoolRemoteFSReader::submit(Reques /// ThreadPool is using "bigger is higher priority" instead of "smaller is more priority". pool.scheduleOrThrow([task]{ (*task)(); }, -request.priority); + return future; } } diff --git a/src/IO/ThreadPoolRemoteFSReader.h b/src/IO/ThreadPoolRemoteFSReader.h index 14ba8ad468d..a2d9cbbf779 100644 --- a/src/IO/ThreadPoolRemoteFSReader.h +++ b/src/IO/ThreadPoolRemoteFSReader.h @@ -8,7 +8,7 @@ namespace DB { -class ReadBufferFromRemoteFS; +class ReadBufferFromRemoteFSGather; class ThreadPoolRemoteFSReader : public IAsynchronousReader { @@ -28,12 +28,12 @@ public: struct ThreadPoolRemoteFSReader::RemoteFSFileDescriptor : public IFileDescriptor { public: - RemoteFSFileDescriptor(std::shared_ptr reader_) : reader(reader_) {} + RemoteFSFileDescriptor(std::shared_ptr reader_) : reader(reader_) {} size_t readInto(char * data, size_t size, size_t offset); private: - std::shared_ptr reader; + std::shared_ptr reader; }; } diff --git a/src/Disks/WriteIndirectBufferFromRemoteFS.cpp b/src/IO/WriteIndirectBufferFromRemoteFS.cpp similarity index 100% rename from src/Disks/WriteIndirectBufferFromRemoteFS.cpp rename to src/IO/WriteIndirectBufferFromRemoteFS.cpp diff --git a/src/Disks/WriteIndirectBufferFromRemoteFS.h b/src/IO/WriteIndirectBufferFromRemoteFS.h similarity index 100% rename from src/Disks/WriteIndirectBufferFromRemoteFS.h rename to src/IO/WriteIndirectBufferFromRemoteFS.h From 1146035f1127b71c0f403d4da8f8a006367f1292 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 2 Oct 2021 17:28:12 +0300 Subject: [PATCH 087/919] Small fix --- src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h | 1 + src/IO/ReadIndirectBufferFromRemoteFS.cpp | 3 --- src/IO/ReadIndirectBufferFromRemoteFS.h | 5 ++++- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h index bd5d37fa3da..3b5bcc55a66 100644 --- a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -16,6 +16,7 @@ namespace DB /** * Reads data from S3/HDFS/Web using stored paths in metadata. +* This class is an asynchronous version of ReadIndirectBufferFromRemoteFS. * * Buffers chain for diskS3: * AsynchronousIndirectReadBufferFromRemoteFS -> ReadBufferFromRemoteFS -> diff --git a/src/IO/ReadIndirectBufferFromRemoteFS.cpp b/src/IO/ReadIndirectBufferFromRemoteFS.cpp index 7aca6b3157c..25c4795b8a5 100644 --- a/src/IO/ReadIndirectBufferFromRemoteFS.cpp +++ b/src/IO/ReadIndirectBufferFromRemoteFS.cpp @@ -79,9 +79,6 @@ bool ReadIndirectBufferFromRemoteFS::nextImpl() /// and assigned to current buffer. swap(*impl); - if (result) - impl->absolute_position += working_buffer.size(); - return result; } diff --git a/src/IO/ReadIndirectBufferFromRemoteFS.h b/src/IO/ReadIndirectBufferFromRemoteFS.h index abd89510882..9cd2f12d82f 100644 --- a/src/IO/ReadIndirectBufferFromRemoteFS.h +++ b/src/IO/ReadIndirectBufferFromRemoteFS.h @@ -14,7 +14,10 @@ namespace DB class ReadBufferFromRemoteFSGather; -/// Reads data from S3/HDFS/Web using stored paths in metadata. +/** +* Reads data from S3/HDFS/Web using stored paths in metadata. +* There is asynchronous version of this class -- AsynchronousReadIndirectBufferFromRemoteFS. +*/ class ReadIndirectBufferFromRemoteFS : public ReadBufferFromFileBase { From 3a56cc5141624e2c7d5555cd8fe18fdebc52e318 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 3 Oct 2021 09:31:53 +0000 Subject: [PATCH 088/919] fix update test --- src/Common/ErrorCodes.cpp | 5 - .../FileLog/FileLogDirectoryWatcher.cpp | 12 +- .../FileLog/FileLogDirectoryWatcher.h | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 123 +++++++++--------- .../0_stateless/02023_storage_filelog.sh | 2 +- .../02025_storage_filelog_virtual_col.sh | 2 +- 6 files changed, 70 insertions(+), 76 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 9312fde8baa..ee456f81bcf 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -588,12 +588,7 @@ M(618, LZ4_DECODER_FAILED) \ M(619, POSTGRESQL_REPLICATION_INTERNAL_ERROR) \ M(620, QUERY_NOT_ALLOWED) \ - M(621, PATH_DOESNT_EXIST) \ - M(622, FILE_META_NOT_FOUND) \ - M(623, CANNOT_GET_FILE_STAT) \ M(624, NOT_REGULAR_FILE) \ - M(625, READ_META_FILE_FAILED) \ - M(626, FILE_STREAM_ERROR) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 0055d5d3eb1..6fa6a1605af 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -40,7 +40,7 @@ void FileLogDirectoryWatcher::onItemAdded(const Poco::DirectoryWatcher::Director { std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemAdded"}; - events.emplace(ev.item.path(), info); + events[ev.item.path()].emplace_back(info); } @@ -48,7 +48,7 @@ void FileLogDirectoryWatcher::onItemRemoved(const Poco::DirectoryWatcher::Direct { std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemRemoved"}; - events.emplace(ev.item.path(), info); + events[ev.item.path()].emplace_back(info); } /// Optimize for MODIFY event, during a streamToViews period, since the log files @@ -62,24 +62,24 @@ void FileLogDirectoryWatcher::onItemModified(const Poco::DirectoryWatcher::Direc std::lock_guard lock(mutex); auto event_path = ev.item.path(); /// Already have MODIFY event for this file - if (auto it = events.find(event_path); it != events.end() && it->second.type == ev.event) + if (auto it = events.find(event_path); it != events.end() && it->second.back().type == ev.event) return; EventInfo info{ev.event, "onItemModified"}; - events.emplace(event_path, info); + events[event_path].emplace_back(info); } void FileLogDirectoryWatcher::onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemMovedFrom"}; - events.emplace(ev.item.path(), info); + events[ev.item.path()].emplace_back(info); } void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemMovedTo"}; - events.emplace(ev.item.path(), info); + events[ev.item.path()].emplace_back(info); } void FileLogDirectoryWatcher::onError(const Poco::Exception & e) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index e8ce1a2bbd5..ae1853b7a53 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -18,7 +18,7 @@ public: std::string callback; }; - using Events = std::unordered_map; + using Events = std::unordered_map>; struct Error { diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index ded3016c123..e1218b99332 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -37,10 +37,9 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; - extern const int CANNOT_GET_FILE_STAT; + extern const int CANNOT_STAT; extern const int NOT_REGULAR_FILE; - extern const int READ_META_FILE_FAILED; - extern const int FILE_STREAM_ERROR; + extern const int CANNOT_READ_ALL_DATA; extern const int LOGICAL_ERROR; extern const int TABLE_METADATA_ALREADY_EXISTS; } @@ -262,12 +261,12 @@ void StorageFileLog::deserialize() if (!tryReadIntText(inode, in)) { - throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", dir_entry.path().c_str()); } assertChar('\n', in); if (!tryReadIntText(last_written_pos, in)) { - throw Exception(ErrorCodes::READ_META_FILE_FAILED, "Read meta file {} failed.", dir_entry.path().c_str()); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", dir_entry.path().c_str()); } meta.file_name = dir_entry.path().filename(); @@ -282,7 +281,7 @@ UInt64 StorageFileLog::getInode(const String & file_name) struct stat file_stat; if (stat(file_name.c_str(), &file_stat)) { - throw Exception(ErrorCodes::CANNOT_GET_FILE_STAT, "Can not get stat info of file {}", file_name); + throw Exception(ErrorCodes::CANNOT_STAT, "Can not get stat info of file {}", file_name); } return file_stat.st_ino; } @@ -377,7 +376,7 @@ void StorageFileLog::assertStreamGood(const std::ifstream & reader) { if (!reader.good()) { - throw Exception(ErrorCodes::FILE_STREAM_ERROR, "Stream is in bad state."); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Stream is in bad state."); } } @@ -400,7 +399,7 @@ void StorageFileLog::openFilesAndSetPos() auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); if (meta.last_writen_position > static_cast(file_end)) { - throw Exception(ErrorCodes::FILE_STREAM_ERROR, "File {} has been broken.", file); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "File {} has been broken.", file); } /// update file end at the monment, used in ReadBuffer and serialize meta.last_open_end = file_end; @@ -689,69 +688,69 @@ bool StorageFileLog::updateFileInfos() auto events = directory_watch->getEventsAndReset(); - for (const auto & [event_path, event_info] : events) + for (const auto & [event_path, event_infos] : events) { - switch (event_info.type) + String file_name = std::filesystem::path(event_path).filename(); + for (const auto & event_info : event_infos) { - case Poco::DirectoryWatcher::DW_ITEM_ADDED: + switch (event_info.type) { - LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); - if (std::filesystem::is_regular_file(event_path)) + case Poco::DirectoryWatcher::DW_ITEM_ADDED: { - auto file_name = std::filesystem::path(event_path).filename(); - auto inode = getInode(event_path); + LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); + if (std::filesystem::is_regular_file(event_path)) + { + auto inode = getInode(event_path); + + file_infos.file_names.push_back(file_name); + + file_infos.meta_by_inode.emplace(inode, FileMeta{.file_name = file_name}); + file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); + } + break; + } + + case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: + { + LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); + /// When new file added and appended, it has two event: DW_ITEM_ADDED + /// and DW_ITEM_MODIFIED, since the order of these two events in the + /// sequence is uncentain, so we may can not find it in file_infos, just + /// skip it, the file info will be handled in DW_ITEM_ADDED case. + if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) + { + it->second.status = FileStatus::UPDATED; + } + break; + } + + case Poco::DirectoryWatcher::DW_ITEM_REMOVED: + case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: + { + LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); + if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) + { + it->second.status = FileStatus::REMOVED; + } + break; + } + case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: + { + LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); file_infos.file_names.push_back(file_name); - - file_infos.meta_by_inode.emplace(inode, FileMeta{.file_name = file_name}); + auto inode = getInode(event_path); file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); - } - break; - } - case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: - { - String file_name = std::filesystem::path(event_path).filename(); - LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); - /// When new file added and appended, it has two event: DW_ITEM_ADDED - /// and DW_ITEM_MODIFIED, since the order of these two events in the - /// sequence is uncentain, so we may can not find it in file_infos, just - /// skip it, the file info will be handled in DW_ITEM_ADDED case. - if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) - { - it->second.status = FileStatus::UPDATED; - } - break; - } - - case Poco::DirectoryWatcher::DW_ITEM_REMOVED: - case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: - { - String file_name = std::filesystem::path(event_path).filename(); - LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); - if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) - { - it->second.status = FileStatus::REMOVED; - } - break; - } - case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: - { - auto file_name = std::filesystem::path(event_path).filename(); - LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); - - file_infos.file_names.push_back(file_name); - auto inode = getInode(event_path); - file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); - - /// File has been renamed, we should also rename meta file - if (auto it = file_infos.meta_by_inode.find(inode); it != file_infos.meta_by_inode.end()) - { - auto old_name = it->second.file_name; - it->second.file_name = file_name; - if (std::filesystem::exists(getFullMetaPath(old_name))) + /// File has been renamed, we should also rename meta file + if (auto it = file_infos.meta_by_inode.find(inode); it != file_infos.meta_by_inode.end()) { - std::filesystem::rename(getFullMetaPath(old_name), getFullMetaPath(file_name)); + auto old_name = it->second.file_name; + it->second.file_name = file_name; + if (std::filesystem::exists(getFullMetaPath(old_name))) + { + std::filesystem::rename(getFullMetaPath(old_name), getFullMetaPath(file_name)); + } } } } diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 6554f019a9f..b818093dfdb 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -55,7 +55,7 @@ ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" truncate ${user_files_path}/logs/a.txt --size 0 # exception happend -${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" 2>&1 | grep -q "Code: 626" && echo 'OK' || echo 'FAIL' +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" 2>&1 | grep -q "Code: 33" && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} --query "drop table file_log;" diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index a7293288522..b8b7c980268 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -55,7 +55,7 @@ ${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order truncate ${user_files_path}/logs/a.txt --size 0 # exception happend -${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" 2>&1 | grep -q "Code: 626" && echo 'OK' || echo 'FAIL' +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" 2>&1 | grep -q "Code: 33" && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} --query "drop table file_log;" From 9683513c525ac6c12150d3615758742813953c2e Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 3 Oct 2021 11:03:20 +0000 Subject: [PATCH 089/919] fix build --- src/Storages/FileLog/FileLogDirectoryWatcher.h | 2 +- src/Storages/FileLog/FileLogSource.cpp | 2 +- src/Storages/FileLog/ReadBufferFromFileLog.cpp | 2 +- src/Storages/FileLog/ReadBufferFromFileLog.h | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 2 +- src/Storages/FileLog/StorageFileLog.h | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index ae1853b7a53..d58a23f402b 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -4,7 +4,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index e2223b3d3c2..40114b4f6f6 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 04af2f829ba..2fc0e0885a1 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index c792978a68a..b4102870c3a 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index e1218b99332..b2b11f22aa9 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -26,7 +26,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index f181ad570c3..1ffa04f7c50 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -10,7 +10,7 @@ #include #include -#include +#include #include #include From 8c82e8b9bc032a45cabe52b04eeaeacf36d71be1 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 3 Oct 2021 17:41:08 +0000 Subject: [PATCH 090/919] fix fix --- .../FileLog/FileLogDirectoryWatcher.cpp | 65 ++++++++++++-- src/Storages/FileLog/StorageFileLog.cpp | 89 +++++++++++++------ 2 files changed, 118 insertions(+), 36 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 6fa6a1605af..827d8ec105b 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -39,16 +39,36 @@ const std::string & FileLogDirectoryWatcher::getPath() const void FileLogDirectoryWatcher::onItemAdded(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); + EventInfo info{ev.event, "onItemAdded"}; - events[ev.item.path()].emplace_back(info); + std::string event_path = ev.item.path(); + + if (auto it = events.find(event_path); it != events.end()) + { + it->second.emplace_back(info); + } + else + { + events.emplace(event_path, std::vector{info}); + } } void FileLogDirectoryWatcher::onItemRemoved(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); + EventInfo info{ev.event, "onItemRemoved"}; - events[ev.item.path()].emplace_back(info); + std::string event_path = ev.item.path(); + + if (auto it = events.find(event_path); it != events.end()) + { + it->second.emplace_back(info); + } + else + { + events.emplace(event_path, std::vector{info}); + } } /// Optimize for MODIFY event, during a streamToViews period, since the log files @@ -60,26 +80,55 @@ void FileLogDirectoryWatcher::onItemRemoved(const Poco::DirectoryWatcher::Direct void FileLogDirectoryWatcher::onItemModified(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); + auto event_path = ev.item.path(); - /// Already have MODIFY event for this file - if (auto it = events.find(event_path); it != events.end() && it->second.back().type == ev.event) - return; EventInfo info{ev.event, "onItemModified"}; - events[event_path].emplace_back(info); + /// Already have MODIFY event for this file + if (auto it = events.find(event_path); it != events.end()) + { + if (it->second.back().type == ev.event) + return; + else + it->second.emplace_back(info); + } + else + { + events.emplace(event_path, std::vector{info}); + } } void FileLogDirectoryWatcher::onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); + EventInfo info{ev.event, "onItemMovedFrom"}; - events[ev.item.path()].emplace_back(info); + std::string event_path = ev.item.path(); + + if (auto it = events.find(event_path); it != events.end()) + { + it->second.emplace_back(info); + } + else + { + events.emplace(event_path, std::vector{info}); + } } void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent& ev) { std::lock_guard lock(mutex); + EventInfo info{ev.event, "onItemMovedTo"}; - events[ev.item.path()].emplace_back(info); + std::string event_path = ev.item.path(); + + if (auto it = events.find(event_path); it != events.end()) + { + it->second.emplace_back(info); + } + else + { + events.emplace(event_path, std::vector{info}); + } } void FileLogDirectoryWatcher::onError(const Poco::Exception & e) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index b2b11f22aa9..e0ddd19b1ba 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -81,8 +81,8 @@ StorageFileLog::StorageFileLog( loadFiles(); #ifndef NDEBUG - assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); - assert(file_infos.file_names.size() == file_infos.context_by_name.size()); + assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); + assert(file_infos.file_names.size() == file_infos.context_by_name.size()); #endif if (path_is_directory) @@ -200,14 +200,18 @@ void StorageFileLog::loadFiles() { InodeToFileMeta valid_metas; valid_metas.reserve(file_infos.context_by_name.size()); - for (const auto & it : file_infos.meta_by_inode) + for (const auto & [inode, meta] : file_infos.meta_by_inode) { - auto file_name = it.second.file_name; - if (file_infos.context_by_name.contains(file_name)) - valid_metas.emplace(it); + /// Note, here we need to use inode to judge does the meta file is valid. + /// In the case that when a file deleted, then we create new file with the + /// same name, it will have different inode number with stored meta file, + /// so the stored meta file is invalid + if (auto it = file_infos.context_by_name.find(meta.file_name); + it != file_infos.context_by_name.end() && it->second.inode == inode) + valid_metas.emplace(inode, meta); /// Delete meta file from filesystem else - std::filesystem::remove(getFullMetaPath(file_name)); + std::filesystem::remove(getFullMetaPath(meta.file_name)); } file_infos.meta_by_inode.swap(valid_metas); } @@ -215,6 +219,10 @@ void StorageFileLog::loadFiles() void StorageFileLog::serialize() const { + if (!std::filesystem::exists(root_meta_path)) + { + std::filesystem::create_directories(root_meta_path); + } for (const auto & it : file_infos.meta_by_inode) { auto full_name = getFullMetaPath(it.second.file_name); @@ -231,6 +239,10 @@ void StorageFileLog::serialize() const void StorageFileLog::serialize(UInt64 inode, const FileMeta & file_meta) const { + if (!std::filesystem::exists(root_meta_path)) + { + std::filesystem::create_directories(root_meta_path); + } auto full_name = getFullMetaPath(file_meta.file_name); if (!std::filesystem::exists(full_name)) { @@ -362,14 +374,21 @@ void StorageFileLog::startup() void StorageFileLog::shutdown() { - if (task) + try { - task->stream_cancelled = true; + if (task) + { + task->stream_cancelled = true; - LOG_TRACE(log, "Waiting for cleanup"); - task->holder->deactivate(); + LOG_TRACE(log, "Waiting for cleanup"); + task->holder->deactivate(); + } + closeFilesAndStoreMeta(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); } - closeFilesAndStoreMeta(); } void StorageFileLog::assertStreamGood(const std::ifstream & reader) @@ -698,14 +717,22 @@ bool StorageFileLog::updateFileInfos() case Poco::DirectoryWatcher::DW_ITEM_ADDED: { LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); + /// Check if it is a regular file, and new file may be renamed or removed if (std::filesystem::is_regular_file(event_path)) { auto inode = getInode(event_path); file_infos.file_names.push_back(file_name); - file_infos.meta_by_inode.emplace(inode, FileMeta{.file_name = file_name}); - file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); + if (auto it = file_infos.meta_by_inode.find(inode); it != file_infos.meta_by_inode.end()) + it->second = FileMeta{.file_name = file_name}; + else + file_infos.meta_by_inode.emplace(inode, FileMeta{.file_name = file_name}); + + if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) + it->second = FileContext{.inode = inode}; + else + file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); } break; } @@ -718,9 +745,7 @@ bool StorageFileLog::updateFileInfos() /// sequence is uncentain, so we may can not find it in file_infos, just /// skip it, the file info will be handled in DW_ITEM_ADDED case. if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) - { it->second.status = FileStatus::UPDATED; - } break; } @@ -729,28 +754,36 @@ bool StorageFileLog::updateFileInfos() { LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) - { it->second.status = FileStatus::REMOVED; - } break; } case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: { LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); - file_infos.file_names.push_back(file_name); - auto inode = getInode(event_path); - file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); - - /// File has been renamed, we should also rename meta file - if (auto it = file_infos.meta_by_inode.find(inode); it != file_infos.meta_by_inode.end()) + /// Similar to DW_ITEM_ADDED, but if it removed from an old file + /// should obtain old meta file and rename meta file + if (std::filesystem::is_regular_file(event_path)) { - auto old_name = it->second.file_name; - it->second.file_name = file_name; - if (std::filesystem::exists(getFullMetaPath(old_name))) + file_infos.file_names.push_back(file_name); + auto inode = getInode(event_path); + + if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) + it->second = FileContext{.inode = inode}; + else + file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); + + /// File has been renamed, we should also rename meta file + if (auto it = file_infos.meta_by_inode.find(inode); it != file_infos.meta_by_inode.end()) { - std::filesystem::rename(getFullMetaPath(old_name), getFullMetaPath(file_name)); + auto old_name = it->second.file_name; + it->second.file_name = file_name; + if (std::filesystem::exists(getFullMetaPath(old_name))) + std::filesystem::rename(getFullMetaPath(old_name), getFullMetaPath(file_name)); } + /// May move from other place, adding new meta info + else + file_infos.meta_by_inode.emplace(inode, FileMeta{.file_name = file_name}); } } } From f40a17c300e023fff1845c94f7c17f7030401c82 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 4 Oct 2021 07:43:28 +0000 Subject: [PATCH 091/919] fix bug in updateFileInfos update --- src/Storages/FileLog/StorageFileLog.cpp | 39 +++++++++++++++++-------- 1 file changed, 27 insertions(+), 12 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index e0ddd19b1ba..ffcd9486f7e 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -705,6 +705,12 @@ bool StorageFileLog::updateFileInfos() if (error.has_error) LOG_ERROR(log, "Error happened during watching directory {}: {}", directory_watch->getPath(), error.error_msg); +/// These file infos should always have same size(one for one) before update and after update +#ifndef NDEBUG + assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); + assert(file_infos.file_names.size() == file_infos.context_by_name.size()); +#endif + auto events = directory_watch->getEventsAndReset(); for (const auto & [event_path, event_infos] : events) @@ -794,24 +800,33 @@ bool StorageFileLog::updateFileInfos() /// Remove file infos with REMOVE status for (const auto & file_name : file_infos.file_names) { - if (auto it = file_infos.context_by_name.find(file_name); - it != file_infos.context_by_name.end() && it->second.status == FileStatus::REMOVED) + if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) { - /// Erase meta_by_inode first, otherwise it become invalid - file_infos.meta_by_inode.erase(it->second.inode); - file_infos.context_by_name.erase(it); + if (it->second.status == FileStatus::REMOVED) + { + file_infos.context_by_name.erase(it); - if (std::filesystem::exists(getFullMetaPath(file_name))) - std::filesystem::remove(getFullMetaPath(file_name)); - } - else - { - valid_files.push_back(file_name); + /// We need to check that this ionde does not hold by other file(mv), + /// otherwise, we can not destroy it. + auto inode = it->second.inode; + /// If it's now hold by other file, than the file_name should has + /// been changed during updating file_infos + if (auto meta = file_infos.meta_by_inode.find(inode); + meta != file_infos.meta_by_inode.end() && meta->second.file_name == file_name) + file_infos.meta_by_inode.erase(meta); + + if (std::filesystem::exists(getFullMetaPath(file_name))) + std::filesystem::remove(getFullMetaPath(file_name)); + } + else + { + valid_files.push_back(file_name); + } } } file_infos.file_names.swap(valid_files); - /// These file infos should always have same size(one for one) +/// These file infos should always have same size(one for one) #ifndef NDEBUG assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); From 45a49b399a268bb9421ae05eca468d5aff640482 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 4 Oct 2021 08:23:02 +0000 Subject: [PATCH 092/919] fix typo --- src/Storages/FileLog/StorageFileLog.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index ffcd9486f7e..bf81457995f 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -81,8 +81,8 @@ StorageFileLog::StorageFileLog( loadFiles(); #ifndef NDEBUG - assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); - assert(file_infos.file_names.size() == file_infos.context_by_name.size()); + assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); + assert(file_infos.file_names.size() == file_infos.context_by_name.size()); #endif if (path_is_directory) @@ -806,7 +806,7 @@ bool StorageFileLog::updateFileInfos() { file_infos.context_by_name.erase(it); - /// We need to check that this ionde does not hold by other file(mv), + /// We need to check that this inode does not hold by other file(mv), /// otherwise, we can not destroy it. auto inode = it->second.inode; /// If it's now hold by other file, than the file_name should has From 873805d4da99cc4e7db467f86ddc94336436d68a Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 4 Oct 2021 12:33:05 +0000 Subject: [PATCH 093/919] update poco update fix unbundled build --- contrib/poco | 2 +- src/Storages/FileLog/FileLogDirectoryWatcher.cpp | 2 +- src/Storages/FileLog/FileLogDirectoryWatcher.h | 2 +- src/Storages/FileLog/StorageFileLog.h | 3 ++- 4 files changed, 5 insertions(+), 4 deletions(-) diff --git a/contrib/poco b/contrib/poco index 46c80daf1b0..5966f3a72aa 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 46c80daf1b015aa10474ce82e3d24b578c6ae422 +Subproject commit 5966f3a72aa54f904433a0ffea3f0d7655c2fac2 diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 827d8ec105b..3b4f89d50f5 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -3,7 +3,7 @@ #include FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_) - : path(path_), dw(std::make_shared(path)), log(&Poco::Logger::get("DirectoryIterator (" + path + ")")) + : path(path_), dw(std::make_unique(path)), log(&Poco::Logger::get("DirectoryIterator (" + path + ")")) { /// DW_ITEM_MOVED_FROM and DW_ITEM_MOVED_TO events will only be reported on Linux. /// On other platforms, a file rename or move operation will be reported via a diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index d58a23f402b..158c629b5fd 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -45,7 +45,7 @@ protected: private: const std::string path; - std::shared_ptr dw; + std::unique_ptr dw; Poco::Logger * log; diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 1ffa04f7c50..929be2690e1 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -12,9 +12,10 @@ #include #include -#include #include +#include #include +#include namespace DB { From 6169b180cb9374acc1ab87e57e7975724cbfaf8c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 4 Oct 2021 17:27:44 +0300 Subject: [PATCH 094/919] Fix backward compatibility after #21196. --- src/Core/Defines.h | 4 +- src/Interpreters/ClientInfo.h | 8 +++ src/Interpreters/InterpreterSelectQuery.cpp | 14 +++++ src/Server/TCPHandler.cpp | 5 ++ .../test_aggregate_fixed_key.py | 61 +++++++++++++++++++ 5 files changed, 91 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py diff --git a/src/Core/Defines.h b/src/Core/Defines.h index ce5c9098399..3620c1dfcfe 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -63,7 +63,9 @@ /// Minimum revision with exactly the same set of aggregation methods and rules to select them. /// Two-level (bucketed) aggregation is incompatible if servers are inconsistent in these rules /// (keys will be placed in different buckets and result will not be fully aggregated). -#define DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD 54431 +#define DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD 54456 +#define DBMS_MIN_MAJOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD 21 +#define DBMS_MIN_MINOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD 4 #define DBMS_MIN_REVISION_WITH_COLUMN_DEFAULTS_METADATA 54410 #define DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE 54405 diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index 71570778645..716c66b8b71 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -83,6 +83,14 @@ public: UInt64 client_version_patch = 0; unsigned client_tcp_protocol_version = 0; + /// In case of distributed query, client info for query is actually a client info of client. + /// In order to get a version of server-initiator, use connection_ values. + /// Also for tcp only. + UInt64 connection_client_version_major = 0; + UInt64 connection_client_version_minor = 0; + UInt64 connection_client_version_patch = 0; + unsigned connection_tcp_protocol_version = 0; + /// For http HTTPMethod http_method = HTTPMethod::UNKNOWN; String http_user_agent; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 563e9ba8997..9ef35a774bf 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -75,6 +75,7 @@ #include #include +#include #include #include #include @@ -2609,6 +2610,19 @@ void InterpreterSelectQuery::initSettings() auto & query = getSelectQuery(); if (query.settings()) InterpreterSetQuery(query.settings(), context).executeForCurrentContext(); + + auto & client_info = context->getClientInfo(); + auto min_major = DBMS_MIN_MAJOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD; + auto min_minor = DBMS_MIN_MINOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD; + + if (client_info.query_kind == ClientInfo::QueryKind::SECONDARY_QUERY && + std::forward_as_tuple(client_info.connection_client_version_major, client_info.connection_client_version_minor) < std::forward_as_tuple(min_major, min_minor)) + { + /// Disable two-level aggregation due to version incompatibility. + context->setSetting("group_by_two_level_threshold", Field(0)); + context->setSetting("group_by_two_level_threshold_bytes", Field(0)); + + } } } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 5415bf96443..a139ab5c100 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -944,6 +944,11 @@ void TCPHandler::receiveHello() client_info.client_version_patch = client_version_patch; client_info.client_tcp_protocol_version = client_tcp_protocol_version; + client_info.connection_client_version_major = client_version_major; + client_info.connection_client_version_minor = client_version_minor; + client_info.connection_client_version_patch = client_version_patch; + client_info.connection_tcp_protocol_version = client_tcp_protocol_version; + is_interserver_mode = (user == USER_INTERSERVER_MARKER); if (is_interserver_mode) { diff --git a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py new file mode 100644 index 00000000000..8819be527fd --- /dev/null +++ b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py @@ -0,0 +1,61 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', with_zookeeper=True, image='yandex/clickhouse-server', tag='21.3', with_installed_binary=True) +node2 = cluster.add_instance('node2', with_zookeeper=True, image='yandex/clickhouse-server') +node3 = cluster.add_instance('node3', with_zookeeper=True, image='yandex/clickhouse-server') + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_two_level_merge(start_cluster): + for node in start_cluster.instances.values(): + node.query( + """ + CREATE TABLE IF NOT EXISTS test_two_level_merge(date Date, zone UInt32, number UInt32) + ENGINE = MergeTree() PARTITION BY toUInt64(number / 1000) ORDER BY tuple(); + + INSERT INTO + test_two_level_merge + SELECT + toDate('2021-09-28') - number / 1000, + 249081628, + number + FROM + numbers(15000); + """ + ) + + # covers only the keys64 method + for node in start_cluster.instances.values(): + print(node.query( + """ + SELECT + throwIf(uniqExact(date) != count(), 'group by is borked') + FROM ( + SELECT + date + FROM + remote('node{1,2}', default.test_two_level_merge) + WHERE + date BETWEEN toDate('2021-09-20') AND toDate('2021-09-28') + AND zone = 249081628 + GROUP by date, zone + ) + SETTINGS + group_by_two_level_threshold = 1, + group_by_two_level_threshold_bytes = 1, + max_threads = 2, + prefer_localhost_replica = 0 + """ + )) From 200f655a2f76b3ac23ba10ff54a09da83931af37 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 25 Aug 2021 09:50:30 +0300 Subject: [PATCH 095/919] Fix writing marks in StorageLog. --- src/Storages/StorageLog.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 0fd94bac95a..691e431907a 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -214,8 +214,7 @@ public: , storage(storage_) , metadata_snapshot(metadata_snapshot_) , lock(std::move(lock_)) - , marks_stream( - storage.disk->writeFile(storage.marks_file_path, 4096, WriteMode::Rewrite)) + , marks_stream(storage.disk->writeFile(storage.marks_file_path, 4096, WriteMode::Append)) { if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); From 1ea23230845d5a043d850f892a2d3bd22bed5082 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 5 Oct 2021 06:40:00 +0000 Subject: [PATCH 096/919] update poco --- contrib/poco | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index 5966f3a72aa..39fd359765a 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 5966f3a72aa54f904433a0ffea3f0d7655c2fac2 +Subproject commit 39fd359765a3a77b46d94ec3c5def3c7802a920f From 9ff8a2391d310807807f391275945950274e3d14 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 5 Oct 2021 03:39:15 +0300 Subject: [PATCH 097/919] Add test. --- ...02047_log_family_data_file_sizes.reference | 47 ++++++++++++++++++ .../02047_log_family_data_file_sizes.sh | 48 +++++++++++++++++++ 2 files changed, 95 insertions(+) create mode 100644 tests/queries/0_stateless/02047_log_family_data_file_sizes.reference create mode 100755 tests/queries/0_stateless/02047_log_family_data_file_sizes.sh diff --git a/tests/queries/0_stateless/02047_log_family_data_file_sizes.reference b/tests/queries/0_stateless/02047_log_family_data_file_sizes.reference new file mode 100644 index 00000000000..b802026a0dd --- /dev/null +++ b/tests/queries/0_stateless/02047_log_family_data_file_sizes.reference @@ -0,0 +1,47 @@ +Log: +empty: +1 element: +1 a +__marks.mrk +sizes.json +x.bin +y.bin +3 elements: +1 a +22 bc +333 def +__marks.mrk greater size +sizes.json +x.bin greater size +y.bin greater size + +TinyLog: +empty: +1 element: +1 a +sizes.json +x.bin +y.bin +3 elements: +1 a +22 bc +333 def +sizes.json +x.bin greater size +y.bin greater size + +StripeLog: +empty: +1 element: +1 a +data.bin +index.mrk +sizes.json +3 elements: +1 a +22 bc +333 def +data.bin greater size +index.mrk greater size +sizes.json + diff --git a/tests/queries/0_stateless/02047_log_family_data_file_sizes.sh b/tests/queries/0_stateless/02047_log_family_data_file_sizes.sh new file mode 100755 index 00000000000..e7c1cb5d71e --- /dev/null +++ b/tests/queries/0_stateless/02047_log_family_data_file_sizes.sh @@ -0,0 +1,48 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +declare -a engines=("Log" "TinyLog" "StripeLog") +for engine in "${engines[@]}" +do + echo "$engine:" + + $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS tbl" + $CLICKHOUSE_CLIENT --query="CREATE TABLE tbl(x UInt32, y String) ENGINE=$engine" + data_dir=$($CLICKHOUSE_CLIENT --query="SELECT data_paths[1] FROM system.tables WHERE name='tbl' AND database=currentDatabase()") + + echo "empty:" + find "$data_dir"* 2>/dev/null + + echo "1 element:" + $CLICKHOUSE_CLIENT --query="INSERT INTO tbl VALUES (1, 'a')" + $CLICKHOUSE_CLIENT --query="SELECT * FROM tbl ORDER BY x" + declare -A file_sizes + for name in $(find "$data_dir"* -print0 | xargs -0 -n 1 basename | sort); do + file_path=$data_dir$name + file_size=$(stat -c%s "$file_path") + file_sizes[$name]=$file_size + echo $name + done + + echo "3 elements:" + $CLICKHOUSE_CLIENT --query="INSERT INTO tbl VALUES (22, 'bc'), (333, 'def')" + $CLICKHOUSE_CLIENT --query="SELECT * FROM tbl ORDER BY x" + for name in $(find "$data_dir"* -print0 | xargs -0 -n 1 basename | sort); do + file_path=$data_dir$name + file_size=$(stat -c%s "$file_path") + old_file_size=${file_sizes[$name]} + if [ "$name" == "sizes.json" ]; then + cmp="" + elif (( file_size > old_file_size )); then + cmp="greater size" + else + cmp="unexpected size ($file_size, old_size=$old_file_size)" + fi + echo $name $cmp + done + + echo +done From 0709b2e690df038fc00a671991734081cc606527 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 5 Oct 2021 13:28:45 +0000 Subject: [PATCH 098/919] fix --- src/Storages/FileLog/FileLogSource.cpp | 7 ++++++- src/Storages/FileLog/FileLogSource.h | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index 40114b4f6f6..de46c880301 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -38,7 +38,7 @@ FileLogSource::FileLogSource( buffer = std::make_unique(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_); } -FileLogSource::~FileLogSource() +void FileLogSource::onFinish() { auto & file_infos = storage.getFileInfos(); @@ -56,7 +56,12 @@ FileLogSource::~FileLogSource() Chunk FileLogSource::generate() { if (!buffer || buffer->noRecords()) + { + /// There is no onFinish for ISource, we call it + /// when no records return to close files + onFinish(); return {}; + } MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index d257bb54f06..f57b1096749 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -28,7 +28,7 @@ public: bool noRecords() { return !buffer || buffer->noRecords(); } - virtual ~FileLogSource() override; + void onFinish(); protected: Chunk generate() override; From f2a62ef9eec023bf54c450ff078f7ee7bef08f3a Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 5 Oct 2021 17:38:18 +0000 Subject: [PATCH 099/919] try to fix data race found in DirectoryWatcher --- src/Storages/FileLog/FileLogDirectoryWatcher.h | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index 158c629b5fd..310b5bde388 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -45,13 +45,19 @@ protected: private: const std::string path; + + /// Note, in order to avoid data race found by fuzzer, put events before dw, + /// such that when this class desctruction, dw will be destructed before events. + /// The data race is because dw create a seperate thread to monitor file events + /// and put into events, then if we destruct events first, the monitor thread still + /// running, it may access events during events destruction, leads to data race. + Events events; + std::unique_ptr dw; Poco::Logger * log; std::mutex mutex; - Events events; - Error error; }; From 5a87b6eb4f108b4f9a058bbc5b200dbf415fbe69 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 5 Oct 2021 13:02:26 +0000 Subject: [PATCH 100/919] Predefined configuration for table function remote --- ...OptimizeIfWithConstantConditionVisitor.cpp | 2 +- src/Parsers/ASTFunctionHelpers.h | 16 -- src/Parsers/ASTHelpers.h | 24 ++ .../ExternalDataSourceConfiguration.cpp | 75 ++++-- .../ExternalDataSourceConfiguration.h | 5 +- src/Storages/StorageExternalDistributed.cpp | 5 +- src/Storages/StorageMongoDB.cpp | 5 +- src/Storages/StorageMySQL.cpp | 5 +- src/Storages/StoragePostgreSQL.cpp | 3 +- src/Storages/StorageS3.cpp | 5 +- src/TableFunctions/TableFunctionRemote.cpp | 251 +++++++++++------- src/TableFunctions/TableFunctionS3.cpp | 5 +- tests/config/config.d/named_collection.xml | 13 + ...table_function_named_collections.reference | 2 + ...emote_table_function_named_collections.sql | 11 + 15 files changed, 274 insertions(+), 153 deletions(-) delete mode 100644 src/Parsers/ASTFunctionHelpers.h create mode 100644 src/Parsers/ASTHelpers.h create mode 100644 tests/config/config.d/named_collection.xml create mode 100644 tests/queries/0_stateless/02046_remote_table_function_named_collections.reference create mode 100644 tests/queries/0_stateless/02046_remote_table_function_named_collections.sql diff --git a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp index a9814ce50f5..993c9945039 100644 --- a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp +++ b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Parsers/ASTFunctionHelpers.h b/src/Parsers/ASTFunctionHelpers.h deleted file mode 100644 index 76da2dd1501..00000000000 --- a/src/Parsers/ASTFunctionHelpers.h +++ /dev/null @@ -1,16 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -static bool isFunctionCast(const ASTFunction * function) -{ - if (function) - return function->name == "CAST" || function->name == "_CAST"; - return false; -} - -} diff --git a/src/Parsers/ASTHelpers.h b/src/Parsers/ASTHelpers.h new file mode 100644 index 00000000000..b175049d777 --- /dev/null +++ b/src/Parsers/ASTHelpers.h @@ -0,0 +1,24 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +template +static inline ValueType safeGetFromASTLiteral(const ASTPtr & ast) +{ + return ast->as()->value.safeGet(); +} + +static inline bool isFunctionCast(const ASTFunction * function) +{ + if (function) + return function->name == "CAST" || function->name == "_CAST"; + return false; +} + + +} diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index 00c41069101..d88cd1982e1 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -47,10 +47,11 @@ void ExternalDataSourceConfiguration::set(const ExternalDataSourceConfiguration database = conf.database; table = conf.table; schema = conf.schema; + addresses_expr = conf.addresses_expr; } -std::optional getExternalDataSourceConfiguration(const ASTs & args, ContextPtr context, bool is_database_engine) +std::optional getExternalDataSourceConfiguration(const ASTs & args, ContextPtr context, bool is_database_engine, bool throw_on_no_collection) { if (args.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); @@ -64,7 +65,14 @@ std::optional getExternalDataSourceConfiguration(const const auto & config_prefix = fmt::format("named_collections.{}", collection->name()); if (!config.has(config_prefix)) + { + /// For table function remote we do not throw on no collection, becuase then we consifer first arg + /// as cluster definition from config. + if (!throw_on_no_collection) + return std::nullopt; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", collection->name()); + } configuration.host = config.getString(config_prefix + ".host", ""); configuration.port = config.getInt(config_prefix + ".port", 0); @@ -73,15 +81,19 @@ std::optional getExternalDataSourceConfiguration(const configuration.database = config.getString(config_prefix + ".database", ""); configuration.table = config.getString(config_prefix + ".table", ""); configuration.schema = config.getString(config_prefix + ".schema", ""); + configuration.addresses_expr = config.getString(config_prefix + ".addresses_expr", ""); - if ((args.size() == 1) && (configuration.host.empty() || configuration.port == 0 - || configuration.username.empty() || configuration.password.empty() + if (!configuration.addresses_expr.empty() && !configuration.host.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot have `addresses_expr` and `host`, `port` in configuration at the same time"); + + if ((args.size() == 1) && ((configuration.addresses_expr.empty() && (configuration.host.empty() || configuration.port == 0)) || configuration.database.empty() || (configuration.table.empty() && !is_database_engine))) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Named collection of connection parameters is missing some of the parameters and no key-value arguments are added"); } + /// Check key-value arguments. for (size_t i = 1; i < args.size(); ++i) { if (const auto * ast_function = typeid_cast(args[i].get())) @@ -92,24 +104,40 @@ std::optional getExternalDataSourceConfiguration(const throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument"); auto arg_name = function_args[0]->as()->name(); - auto arg_value = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context)->as()->value; + if (function_args[1]->as()) + { + non_common_args.emplace_back(std::make_pair(arg_name, function_args[1])); + continue; + } - if (arg_name == "host") - configuration.host = arg_value.safeGet(); - else if (arg_name == "port") - configuration.port = arg_value.safeGet(); - else if (arg_name == "user") - configuration.username = arg_value.safeGet(); - else if (arg_name == "password") - configuration.password = arg_value.safeGet(); - else if (arg_name == "database") - configuration.database = arg_value.safeGet(); - else if (arg_name == "table") - configuration.table = arg_value.safeGet(); - else if (arg_name == "schema") - configuration.schema = arg_value.safeGet(); + auto arg_value_ast = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context); + auto arg_value_literal = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context)->as(); + if (arg_value_literal) + { + auto arg_value = arg_value_literal->value; + if (arg_name == "host") + configuration.host = arg_value.safeGet(); + else if (arg_name == "port") + configuration.port = arg_value.safeGet(); + else if (arg_name == "user") + configuration.username = arg_value.safeGet(); + else if (arg_name == "password") + configuration.password = arg_value.safeGet(); + else if (arg_name == "database") + configuration.database = arg_value.safeGet(); + else if (arg_name == "table") + configuration.table = arg_value.safeGet(); + else if (arg_name == "schema") + configuration.schema = arg_value.safeGet(); + else if (arg_name == "addresses_expr") + configuration.addresses_expr = arg_value.safeGet(); + else + non_common_args.emplace_back(std::make_pair(arg_name, arg_value_ast)); + } else - non_common_args.emplace_back(std::make_pair(arg_name, arg_value)); + { + non_common_args.emplace_back(std::make_pair(arg_name, arg_value_ast)); + } } else { @@ -269,9 +297,13 @@ std::optional getURLBasedDataSourceConfiguration(const } } else - non_common_args.emplace_back(std::make_pair(key, config.getString(config_prefix + '.' + key))); + { + auto value = config.getString(config_prefix + '.' + key); + non_common_args.emplace_back(std::make_pair(key, std::make_shared(value))); + } } + /// Check key-value arguments. for (size_t i = 1; i < args.size(); ++i) { if (const auto * ast_function = typeid_cast(args[i].get())) @@ -282,6 +314,7 @@ std::optional getURLBasedDataSourceConfiguration(const throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected key-value defined argument"); auto arg_name = function_args[0]->as()->name(); + auto arg_value_ast = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context); auto arg_value = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context)->as()->value; if (arg_name == "url") @@ -293,7 +326,7 @@ std::optional getURLBasedDataSourceConfiguration(const else if (arg_name == "structure") configuration.structure = arg_value.safeGet(); else - non_common_args.emplace_back(std::make_pair(arg_name, arg_value)); + non_common_args.emplace_back(std::make_pair(arg_name, arg_value_ast)); } else { diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index 5f88653807b..e3b589b1bb4 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -19,6 +19,7 @@ struct ExternalDataSourceConfiguration String schema; std::vector> addresses; /// Failover replicas. + String addresses_expr; String toString() const; @@ -45,7 +46,7 @@ struct StorageMongoDBConfiguration : ExternalDataSourceConfiguration }; -using StorageSpecificArgs = std::vector>; +using StorageSpecificArgs = std::vector>; struct ExternalDataSourceConfig { @@ -63,7 +64,7 @@ struct ExternalDataSourceConfig * Any key-value engine argument except common (`host`, `port`, `username`, `password`, `database`) * is returned in EngineArgs struct. */ -std::optional getExternalDataSourceConfiguration(const ASTs & args, ContextPtr context, bool is_database_engine = false); +std::optional getExternalDataSourceConfiguration(const ASTs & args, ContextPtr context, bool is_database_engine = false, bool throw_on_no_collection = true); std::optional getExternalDataSourceConfiguration( const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context); diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index ce76ab4a723..67071bf5bc5 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -234,7 +235,7 @@ void registerStorageExternalDistributed(StorageFactory & factory) for (const auto & [name, value] : storage_specific_args) { if (name == "description") - cluster_description = value.safeGet(); + cluster_description = safeGetFromASTLiteral(value); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown key-value argument {} for table engine URL", name); @@ -280,7 +281,7 @@ void registerStorageExternalDistributed(StorageFactory & factory) for (const auto & [name, value] : storage_specific_args) { if (name == "description") - cluster_description = value.safeGet(); + cluster_description = safeGetFromASTLiteral(value); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown key-value argument {} for table function URL", name); diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 2a1f7cc2aa9..69a3211b468 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB { @@ -115,9 +116,9 @@ StorageMongoDBConfiguration StorageMongoDB::getConfiguration(ASTs engine_args, C for (const auto & [arg_name, arg_value] : storage_specific_args) { if (arg_name == "collection") - configuration.collection = arg_value.safeGet(); + configuration.collection = safeGetFromASTLiteral(arg_value); else if (arg_name == "options") - configuration.options = arg_value.safeGet(); + configuration.options = safeGetFromASTLiteral(arg_value); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected key-value argument." diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index f2907c0d707..ce4fd14ca55 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -251,9 +252,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.safeGet(); + configuration.replace_query = safeGetFromASTLiteral(arg_value); else if (arg_name == "on_duplicate_clause") - configuration.on_duplicate_clause = arg_value.safeGet(); + configuration.on_duplicate_clause = safeGetFromASTLiteral(arg_value); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected key-value argument." diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 47a3e4091f5..0842b824ef1 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -39,6 +39,7 @@ #include #include +#include namespace DB @@ -399,7 +400,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.safeGet(); + configuration.on_conflict = safeGetFromASTLiteral(arg_value); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected key-value argument." diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index ce1b049758e..d62854a52be 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include @@ -746,9 +747,9 @@ StorageS3Configuration StorageS3::getConfiguration(ASTs & engine_args, ContextPt for (const auto & [arg_name, arg_value] : storage_specific_args) { if (arg_name == "access_key_id") - configuration.access_key_id = arg_value.safeGet(); + configuration.access_key_id = safeGetFromASTLiteral(arg_value); else if (arg_name == "secret_access_key") - configuration.secret_access_key = arg_value.safeGet(); + configuration.secret_access_key = safeGetFromASTLiteral(arg_value); 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].", diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 296076344a4..5fcca1d876c 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -13,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -32,126 +34,171 @@ namespace ErrorCodes void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr context) { ASTs & args_func = ast_function->children; + ExternalDataSourceConfiguration configuration; + + String cluster_name; + String cluster_description; if (args_func.size() != 1) throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); ASTs & args = args_func.at(0)->children; - const size_t max_args = is_cluster_function ? 4 : 6; - if (args.size() < 2 || args.size() > max_args) - throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - String cluster_name; - String cluster_description; - String remote_database; - String remote_table; - String username; - String password; - - size_t arg_num = 0; - - auto get_string_literal = [](const IAST & node, String & res) + /** + * Number of arguments for remote function is 4. + * Number of arguments for cluster function is 6. + * For now named collection can be used only for remote as cluster does not require credentials. + */ + size_t max_args = is_cluster_function ? 4 : 6; + auto named_collection = getExternalDataSourceConfiguration(args, context, false, false); + if (named_collection) { - const auto * lit = node.as(); - if (!lit) - return false; + if (is_cluster_function) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Named collection cannot be used for table function cluster"); - if (lit->value.getType() != Field::Types::String) - return false; + /** + * Common arguments: database, table, username, password, addresses_expr. + * Specific args (remote): sharding_key, or database (in case it is not ASTLiteral). + * None of the common arguments is empty at this point, it is checked in getExternalDataSourceConfiguration. + */ + auto [common_configuration, storage_specific_args] = named_collection.value(); + configuration.set(common_configuration); - res = safeGet(lit->value); - return true; - }; - - if (is_cluster_function) - { - args[arg_num] = evaluateConstantExpressionOrIdentifierAsLiteral(args[arg_num], context); - cluster_name = args[arg_num]->as().value.safeGet(); - } - else - { - if (!tryGetIdentifierNameInto(args[arg_num], cluster_name)) + for (const auto & [arg_name, arg_value] : storage_specific_args) { - if (!get_string_literal(*args[arg_num], cluster_description)) - throw Exception("Hosts pattern must be string literal (in single quotes).", ErrorCodes::BAD_ARGUMENTS); - } - } - ++arg_num; - - const auto * function = args[arg_num]->as(); - - if (function && TableFunctionFactory::instance().isTableFunctionName(function->name)) - { - remote_table_function_ptr = args[arg_num]; - ++arg_num; - } - else - { - args[arg_num] = evaluateConstantExpressionForDatabaseName(args[arg_num], context); - remote_database = args[arg_num]->as().value.safeGet(); - - ++arg_num; - - auto qualified_name = QualifiedTableName::parseFromString(remote_database); - if (qualified_name.database.empty()) - { - if (arg_num >= args.size()) + if (arg_name == "sharding_key") { - throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + sharding_key = arg_value; + } + else if (arg_name == "database") + { + const auto * function = arg_value->as(); + if (function && TableFunctionFactory::instance().isTableFunctionName(function->name)) + { + remote_table_function_ptr = arg_value; + } + else + { + auto database_literal = evaluateConstantExpressionOrIdentifierAsLiteral(arg_value, context); + configuration.database = safeGetFromASTLiteral(database_literal); + } } else + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Unexpected key-value argument." + "Got: {}, but expected: sharding_key", arg_name); + } + cluster_description = configuration.addresses_expr; + if (cluster_description.empty()) + cluster_description = configuration.port ? configuration.host + ':' + toString(configuration.port) : configuration.host; + } + else + { + if (args.size() < 2 || args.size() > max_args) + throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + size_t arg_num = 0; + auto get_string_literal = [](const IAST & node, String & res) + { + const auto * lit = node.as(); + if (!lit) + return false; + + if (lit->value.getType() != Field::Types::String) + return false; + + res = safeGet(lit->value); + return true; + }; + + if (is_cluster_function) + { + args[arg_num] = evaluateConstantExpressionOrIdentifierAsLiteral(args[arg_num], context); + cluster_name = args[arg_num]->as().value.safeGet(); + } + else + { + if (!tryGetIdentifierNameInto(args[arg_num], cluster_name)) { - 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(); + if (!get_string_literal(*args[arg_num], cluster_description)) + throw Exception("Hosts pattern must be string literal (in single quotes).", ErrorCodes::BAD_ARGUMENTS); + } + } + + ++arg_num; + const auto * function = args[arg_num]->as(); + if (function && TableFunctionFactory::instance().isTableFunctionName(function->name)) + { + remote_table_function_ptr = args[arg_num]; + ++arg_num; + } + else + { + args[arg_num] = evaluateConstantExpressionForDatabaseName(args[arg_num], context); + configuration.database = args[arg_num]->as().value.safeGet(); + + ++arg_num; + + auto qualified_name = QualifiedTableName::parseFromString(configuration.database); + if (qualified_name.database.empty()) + { + if (arg_num >= args.size()) + { + throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + else + { + 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(); + ++arg_num; + } + } + + configuration.database = std::move(qualified_name.database); + configuration.table = std::move(qualified_name.table); + + /// Cluster function may have sharding key for insert + if (is_cluster_function && arg_num < args.size()) + { + sharding_key = args[arg_num]; ++arg_num; } } - remote_database = std::move(qualified_name.database); - remote_table = std::move(qualified_name.table); - } + /// Username and password parameters are prohibited in cluster version of the function + if (!is_cluster_function) + { + if (arg_num < args.size()) + { + if (!get_string_literal(*args[arg_num], configuration.username)) + { + configuration.username = "default"; + sharding_key = args[arg_num]; + } + ++arg_num; + } - /// Cluster function may have sharding key for insert - if (is_cluster_function && arg_num < args.size()) - { - sharding_key = args[arg_num]; - ++arg_num; - } + if (arg_num < args.size() && !sharding_key) + { + if (!get_string_literal(*args[arg_num], configuration.password)) + { + sharding_key = args[arg_num]; + } + ++arg_num; + } + + if (arg_num < args.size() && !sharding_key) + { + sharding_key = args[arg_num]; + ++arg_num; + } + } - /// Username and password parameters are prohibited in cluster version of the function - if (!is_cluster_function) - { if (arg_num < args.size()) - { - if (!get_string_literal(*args[arg_num], username)) - { - username = "default"; - sharding_key = args[arg_num]; - } - ++arg_num; - } - - if (arg_num < args.size() && !sharding_key) - { - if (!get_string_literal(*args[arg_num], password)) - { - sharding_key = args[arg_num]; - } - ++arg_num; - } - - if (arg_num < args.size() && !sharding_key) - { - sharding_key = args[arg_num]; - ++arg_num; - } + throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } - if (arg_num < args.size()) - throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (!cluster_name.empty()) { /// Use an existing cluster from the main config @@ -197,19 +244,19 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, ContextPtr cluster = std::make_shared( context->getSettings(), names, - username, - password, + configuration.username, + configuration.password, (secure ? (maybe_secure_port ? *maybe_secure_port : DBMS_DEFAULT_SECURE_PORT) : context->getTCPPort()), treat_local_as_remote, treat_local_port_as_remote, secure); } - if (!remote_table_function_ptr && remote_table.empty()) + if (!remote_table_function_ptr && configuration.table.empty()) throw Exception("The name of remote table cannot be empty", ErrorCodes::BAD_ARGUMENTS); - remote_table_id.database_name = remote_database; - remote_table_id.table_name = remote_table; + remote_table_id.database_name = configuration.database; + remote_table_id.table_name = configuration.table; } StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 4faf3f15aa4..6bcf7271de6 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include "registerTableFunctions.h" @@ -48,9 +49,9 @@ void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr con for (const auto & [arg_name, arg_value] : storage_specific_args) { if (arg_name == "access_key_id") - configuration.access_key_id = arg_value.safeGet(); + configuration.access_key_id = safeGetFromASTLiteral(arg_value); else if (arg_name == "secret_access_key") - configuration.secret_access_key = arg_value.safeGet(); + configuration.secret_access_key = safeGetFromASTLiteral(arg_value); else throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Unknown key-value argument `{}` for StorageS3, expected: " diff --git a/tests/config/config.d/named_collection.xml b/tests/config/config.d/named_collection.xml new file mode 100644 index 00000000000..e48bac7d4ef --- /dev/null +++ b/tests/config/config.d/named_collection.xml @@ -0,0 +1,13 @@ + + + + 127.0.0.2 + 9000 + remote_test
+
+ + 127.0.0.{1,2} + remote_test
+
+
+
diff --git a/tests/queries/0_stateless/02046_remote_table_function_named_collections.reference b/tests/queries/0_stateless/02046_remote_table_function_named_collections.reference new file mode 100644 index 00000000000..de78180725a --- /dev/null +++ b/tests/queries/0_stateless/02046_remote_table_function_named_collections.reference @@ -0,0 +1,2 @@ +4 +8 diff --git a/tests/queries/0_stateless/02046_remote_table_function_named_collections.sql b/tests/queries/0_stateless/02046_remote_table_function_named_collections.sql new file mode 100644 index 00000000000..a198c388781 --- /dev/null +++ b/tests/queries/0_stateless/02046_remote_table_function_named_collections.sql @@ -0,0 +1,11 @@ +-- Tags: shard, no-fasttest + +DROP TABLE IF EXISTS remote_test; +CREATE TABLE remote_test(a1 UInt8) ENGINE=Memory; +INSERT INTO FUNCTION remote(remote1, database=currentDatabase()) VALUES(1); +INSERT INTO FUNCTION remote(remote1, database=currentDatabase()) VALUES(2); +INSERT INTO FUNCTION remote(remote1, database=currentDatabase()) VALUES(3); +INSERT INTO FUNCTION remote(remote1, database=currentDatabase()) VALUES(4); +SELECT COUNT(*) FROM remote(remote1, database=currentDatabase()); +SELECT count(*) FROM remote(remote2, database=merge(currentDatabase(), '^remote_test')); +DROP TABLE remote_test; From b9f8647353ea9eba64df1fdc782f421d4d737bc5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 6 Oct 2021 11:59:41 +0300 Subject: [PATCH 101/919] Fix build with master --- src/Interpreters/InterpreterKillQueryQuery.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 13108e297f5..df31ed75374 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -355,7 +355,8 @@ BlockIO InterpreterKillQueryQuery::execute() ErrorCodes::ACCESS_DENIED); - res_io.in = std::make_shared(header.cloneWithColumns(std::move(res_columns))); + + res_io.pipeline = QueryPipeline(Pipe(std::make_shared(header.cloneWithColumns(std::move(res_columns))))); break; } From acdb8233c3426b78d3c0fc075fd2da7b1b723d92 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 6 Oct 2021 08:08:49 +0000 Subject: [PATCH 102/919] fix fix fix --- .../FileLog/FileLogDirectoryWatcher.h | 4 +- src/Storages/FileLog/FileLogSource.cpp | 26 ++--- src/Storages/FileLog/FileLogSource.h | 11 +++ src/Storages/FileLog/StorageFileLog.cpp | 96 ++++++++++++++++--- src/Storages/FileLog/StorageFileLog.h | 9 +- 5 files changed, 120 insertions(+), 26 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index 310b5bde388..00ee7ddfc40 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -47,8 +47,8 @@ private: const std::string path; /// Note, in order to avoid data race found by fuzzer, put events before dw, - /// such that when this class desctruction, dw will be destructed before events. - /// The data race is because dw create a seperate thread to monitor file events + /// such that when this class destruction, dw will be destructed before events. + /// The data race is because dw create a separate thread to monitor file events /// and put into events, then if we destruct events first, the monitor thread still /// running, it may access events during events destruction, leads to data race. Events events; diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index de46c880301..eb8cb2b28f9 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -36,21 +36,17 @@ FileLogSource::FileLogSource( metadata_snapshot->getSampleBlockForColumns(storage.getVirtualColumnNames(), storage.getVirtuals(), storage.getStorageID())) { buffer = std::make_unique(storage, max_block_size, poll_time_out, context, stream_number_, max_streams_number_); + + const auto & file_infos = storage.getFileInfos(); + + size_t files_per_stream = file_infos.file_names.size() / max_streams_number; + start = stream_number * files_per_stream; + end = stream_number == max_streams_number - 1 ? file_infos.file_names.size() : (stream_number + 1) * files_per_stream; } void FileLogSource::onFinish() { - auto & file_infos = storage.getFileInfos(); - - size_t files_per_stream = file_infos.file_names.size() / max_streams_number; - size_t start = stream_number * files_per_stream; - size_t end = stream_number == max_streams_number - 1 ? file_infos.file_names.size() : (stream_number + 1) * files_per_stream; - - /// Each stream responsible for close it's files and store meta - for (size_t i = start; i < end; ++i) - { - storage.closeFileAndStoreMeta(file_infos.file_names[i]); - } + storage.closeFilesAndStoreMeta(start, end); } Chunk FileLogSource::generate() @@ -60,6 +56,7 @@ Chunk FileLogSource::generate() /// There is no onFinish for ISource, we call it /// when no records return to close files onFinish(); + finished = true; return {}; } @@ -105,7 +102,11 @@ Chunk FileLogSource::generate() } if (total_rows == 0) + { + onFinish(); + finished = true; return {}; + } auto result_block = non_virtual_header.cloneWithColumns(executor.getResultColumns()); auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); @@ -121,6 +122,9 @@ Chunk FileLogSource::generate() auto converting_actions = std::make_shared(std::move(converting_dag)); converting_actions->execute(result_block); + /// After generate each block, store metas into disk + storage.storeMetas(start, end); + return Chunk(result_block.getColumns(), result_block.rows()); } diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index f57b1096749..6f94ef2470a 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -30,6 +30,12 @@ public: void onFinish(); + virtual ~FileLogSource() override + { + if (!finished) + onFinish(); + } + protected: Chunk generate() override; @@ -49,6 +55,11 @@ private: Block non_virtual_header; Block virtual_header; + + /// The start pos and end pos of files responsible by this stream, + /// does not includind end + int start; + int end; }; } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index bf81457995f..431d88b9f7b 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -223,17 +223,26 @@ void StorageFileLog::serialize() const { std::filesystem::create_directories(root_meta_path); } - for (const auto & it : file_infos.meta_by_inode) + for (const auto & [inode, meta] : file_infos.meta_by_inode) { - auto full_name = getFullMetaPath(it.second.file_name); + auto full_name = getFullMetaPath(meta.file_name); if (!std::filesystem::exists(full_name)) { FS::createFile(full_name); } + else + { + auto last_pos = getLastWrittenPos(meta.file_name); + if (last_pos > meta.last_writen_position) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Last stored last written pos is bigger than current last written pos need to store for meta file {}.", + full_name); + } WriteBufferFromFile out(full_name); - writeIntText(it.first, out); + writeIntText(inode, out); writeChar('\n', out); - writeIntText(it.second.last_writen_position, out); + writeIntText(meta.last_writen_position, out); } } @@ -248,6 +257,15 @@ void StorageFileLog::serialize(UInt64 inode, const FileMeta & file_meta) const { FS::createFile(full_name); } + else + { + auto last_pos = getLastWrittenPos(file_meta.file_name); + if (last_pos > file_meta.last_writen_position) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Last stored last written pos is bigger than current last written pos need to store for meta file {}.", + full_name); + } WriteBufferFromFile out(full_name); writeIntText(inode, out); writeChar('\n', out); @@ -288,6 +306,23 @@ void StorageFileLog::deserialize() } } +UInt64 StorageFileLog::getLastWrittenPos(const String & file_name) const +{ + ReadBufferFromFile in(getFullMetaPath(file_name)); + UInt64 _, last_written_pos; + + if (!tryReadIntText(_, in)) + { + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", getFullMetaPath(file_name)); + } + assertChar('\n', in); + if (!tryReadIntText(last_written_pos, in)) + { + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", getFullMetaPath(file_name)); + } + return last_written_pos; +} + UInt64 StorageFileLog::getInode(const String & file_name) { struct stat file_stat; @@ -307,7 +342,19 @@ Pipe StorageFileLog::read( size_t /* max_block_size */, unsigned /* num_streams */) { - /// We need this lock, in case read and streamToViews execute at the same time + auto table_id = getStorageID(); + size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); + /// If there are MVs depended on this table, we just forbid reading + if (dependencies_count) + { + throw Exception( + ErrorCodes::CANNOT_READ_ALL_DATA, + "Can not read from table {}, because it has been depended by other tables.", + table_id.getTableName()); + } + + /// We need this lock, in case read and streamToViews execute at the same time. + /// In case of MV attached during reading std::lock_guard lock(status_mutex); updateFileInfos(); @@ -440,14 +487,41 @@ void StorageFileLog::closeFilesAndStoreMeta() serialize(); } -void StorageFileLog::closeFileAndStoreMeta(const String & file_name) +void StorageFileLog::closeFilesAndStoreMeta(int start, int end) { - auto & file_ctx = findInMap(file_infos.context_by_name, file_name); - if (file_ctx.reader.is_open()) - file_ctx.reader.close(); +#ifndef NDEBUG + assert(start >= 0); + assert(start < end); + assert(end <= file_infos.file_names.size()); +#endif - auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); - serialize(file_ctx.inode, meta); + for (int i = start; i < end; ++i) + { + auto & file_ctx = findInMap(file_infos.context_by_name, file_infos.file_names[i]); + + if (file_ctx.reader.is_open()) + file_ctx.reader.close(); + + auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); + serialize(file_ctx.inode, meta); + } +} + +void StorageFileLog::storeMetas(int start, int end) +{ +#ifndef NDEBUG + assert(start >= 0); + assert(start < end); + assert(end <= file_infos.file_names.size()); +#endif + + for (int i = start; i < end; ++i) + { + auto & file_ctx = findInMap(file_infos.context_by_name, file_infos.file_names[i]); + + auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); + serialize(file_ctx.inode, meta); + } } size_t StorageFileLog::getMaxBlockSize() const diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 929be2690e1..0251544f33f 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -97,10 +97,14 @@ public: static UInt64 getInode(const String & file_name); void openFilesAndSetPos(); + /// Used in shutdown() void closeFilesAndStoreMeta(); - /// Used in FileSource - void closeFileAndStoreMeta(const String & file_name); + /// Used in FileLogSource when finish generating all blocks + void closeFilesAndStoreMeta(int start, int end); + + /// Used in FileLogSource after generating every block + void storeMetas(int start, int end); static void assertStreamGood(const std::ifstream & reader); @@ -183,6 +187,7 @@ private: void serialize(UInt64 inode, const FileMeta & file_meta) const; void deserialize(); + UInt64 getLastWrittenPos(const String & file_name) const; }; } From dd067f80cd13d25073d73b5e4433dd85e8a533ae Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Wed, 6 Oct 2021 12:40:24 +0300 Subject: [PATCH 103/919] update RU --- docs/ru/operations/system-tables/replicas.md | 67 +++++++++++--------- 1 file changed, 37 insertions(+), 30 deletions(-) diff --git a/docs/ru/operations/system-tables/replicas.md b/docs/ru/operations/system-tables/replicas.md index ff58355145d..c7bc9c5982e 100644 --- a/docs/ru/operations/system-tables/replicas.md +++ b/docs/ru/operations/system-tables/replicas.md @@ -13,38 +13,43 @@ FORMAT Vertical ``` ``` text +Query id: dc6dcbcb-dc28-4df9-ae27-4354f5b3b13e + Row 1: ────── -database: merge -table: visits -engine: ReplicatedCollapsingMergeTree -is_leader: 1 -can_become_leader: 1 -is_readonly: 0 -is_session_expired: 0 -future_parts: 1 -parts_to_check: 0 -zookeeper_path: /clickhouse/tables/01-06/visits -replica_name: example01-06-1.yandex.ru -replica_path: /clickhouse/tables/01-06/visits/replicas/example01-06-1.yandex.ru -columns_version: 9 -queue_size: 1 -inserts_in_queue: 0 -merges_in_queue: 1 -part_mutations_in_queue: 0 -queue_oldest_time: 2020-02-20 08:34:30 -inserts_oldest_time: 0000-00-00 00:00:00 -merges_oldest_time: 2020-02-20 08:34:30 -part_mutations_oldest_time: 0000-00-00 00:00:00 -oldest_part_to_get: -oldest_part_to_merge_to: 20200220_20284_20840_7 -oldest_part_to_mutate_to: -log_max_index: 596273 -log_pointer: 596274 -last_queue_update: 2020-02-20 08:34:32 -absolute_delay: 0 -total_replicas: 2 -active_replicas: 2 +database: default +table: test +engine: ReplicatedMergeTree +is_leader: 1 +can_become_leader: 1 +is_readonly: 0 +is_session_expired: 0 +future_parts: 0 +parts_to_check: 0 +zookeeper_path: /test/t +replica_name: 1 +replica_path: /test/t/replicas/1 +columns_version: -1 +queue_size: 0 +inserts_in_queue: 0 +merges_in_queue: 0 +part_mutations_in_queue: 0 +queue_oldest_time: 1970-01-01 03:00:00 +inserts_oldest_time: 1970-01-01 03:00:00 +merges_oldest_time: 1970-01-01 03:00:00 +part_mutations_oldest_time: 1970-01-01 03:00:00 +oldest_part_to_get: +oldest_part_to_merge_to: +oldest_part_to_mutate_to: +log_max_index: 0 +log_pointer: 0 +last_queue_update: 1970-01-01 03:00:00 +absolute_delay: 0 +total_replicas: 1 +active_replicas: 1 +last_queue_update_exception: +zookeeper_exception: +replica_is_active: {'1':1} ``` Столбцы: @@ -82,6 +87,8 @@ active_replicas: 2 - `absolute_delay` (`UInt64`) - задержка (в секундах) для текущей реплики. - `total_replicas` (`UInt8`) - общее число известных реплик этой таблицы. - `active_replicas` (`UInt8`) - число реплик этой таблицы, имеющих сессию в ZK; то есть, число работающих реплик. +- `last_queue_update_exception` (`String`) - если в очереди есть битые записи. Особенно важно, когда в ClickHouse нарушается обратная совместимость между версиями, а записи журнала, сделанные более новыми версиями, не могут быть проанализированы старыми версиями. +- `zookeeper_exception` (`String`) - последнее сообщение об исключении. Появляется, если ошибка произошла при получении информации из ZooKeeper. - `replica_is_active` ([Map(String, UInt8)](../../sql-reference/data-types/map.md)) — соответствие между именем реплики и признаком активности реплики. Если запрашивать все столбцы, то таблица может работать слегка медленно, так как на каждую строчку делается несколько чтений из ZK. From ddca7e071292d754cd22a7c22d20bd9c062a45ba Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 6 Oct 2021 12:51:33 +0300 Subject: [PATCH 104/919] Update InterpreterKillQueryQuery.cpp --- src/Interpreters/InterpreterKillQueryQuery.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index df31ed75374..2b949266c17 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -354,8 +354,6 @@ BlockIO InterpreterKillQueryQuery::execute() "Not allowed to kill move partition. To execute this query it's necessary to have the grant " + required_access_rights.toString(), ErrorCodes::ACCESS_DENIED); - - res_io.pipeline = QueryPipeline(Pipe(std::make_shared(header.cloneWithColumns(std::move(res_columns))))); break; From c801fa65dfb319c7b2e91df975e592b1ab0a1c08 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 6 Oct 2021 10:37:58 +0000 Subject: [PATCH 105/919] fix and add new test fix fix --- src/Databases/DatabaseAtomic.cpp | 3 + src/Storages/FileLog/FileLogSource.h | 4 +- src/Storages/FileLog/StorageFileLog.cpp | 58 ++++++++----------- src/Storages/FileLog/StorageFileLog.h | 12 +++- src/Storages/IStorage.h | 3 + .../02026_storage_filelog_largefile.reference | 3 + .../02026_storage_filelog_largefile.sh | 45 ++++++++++++++ 7 files changed, 90 insertions(+), 38 deletions(-) create mode 100644 tests/queries/0_stateless/02026_storage_filelog_largefile.reference create mode 100755 tests/queries/0_stateless/02026_storage_filelog_largefile.sh diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 5c75f6f1036..ae90f1a6900 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -140,6 +140,9 @@ void DatabaseAtomic::dropTable(ContextPtr local_context, const String & table_na if (table->storesDataOnDisk()) tryRemoveSymlink(table_name); + if (table->dropTableImmediately()) + table->drop(); + /// Notify DatabaseCatalog that table was dropped. It will remove table data in background. /// Cleanup is performed outside of database to allow easily DROP DATABASE without waiting for cleanup to complete. DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop, no_delay); diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index 6f94ef2470a..0ff71c1af87 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -58,8 +58,8 @@ private: /// The start pos and end pos of files responsible by this stream, /// does not includind end - int start; - int end; + size_t start; + size_t end; }; } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 431d88b9f7b..112e68acd08 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -232,12 +232,7 @@ void StorageFileLog::serialize() const } else { - auto last_pos = getLastWrittenPos(meta.file_name); - if (last_pos > meta.last_writen_position) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Last stored last written pos is bigger than current last written pos need to store for meta file {}.", - full_name); + checkOffsetIsValid(full_name, meta.last_writen_position); } WriteBufferFromFile out(full_name); writeIntText(inode, out); @@ -259,12 +254,7 @@ void StorageFileLog::serialize(UInt64 inode, const FileMeta & file_meta) const } else { - auto last_pos = getLastWrittenPos(file_meta.file_name); - if (last_pos > file_meta.last_writen_position) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Last stored last written pos is bigger than current last written pos need to store for meta file {}.", - full_name); + checkOffsetIsValid(full_name, file_meta.last_writen_position); } WriteBufferFromFile out(full_name); writeIntText(inode, out); @@ -306,23 +296,6 @@ void StorageFileLog::deserialize() } } -UInt64 StorageFileLog::getLastWrittenPos(const String & file_name) const -{ - ReadBufferFromFile in(getFullMetaPath(file_name)); - UInt64 _, last_written_pos; - - if (!tryReadIntText(_, in)) - { - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", getFullMetaPath(file_name)); - } - assertChar('\n', in); - if (!tryReadIntText(last_written_pos, in)) - { - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", getFullMetaPath(file_name)); - } - return last_written_pos; -} - UInt64 StorageFileLog::getInode(const String & file_name) { struct stat file_stat; @@ -487,7 +460,7 @@ void StorageFileLog::closeFilesAndStoreMeta() serialize(); } -void StorageFileLog::closeFilesAndStoreMeta(int start, int end) +void StorageFileLog::closeFilesAndStoreMeta(size_t start, size_t end) { #ifndef NDEBUG assert(start >= 0); @@ -495,7 +468,7 @@ void StorageFileLog::closeFilesAndStoreMeta(int start, int end) assert(end <= file_infos.file_names.size()); #endif - for (int i = start; i < end; ++i) + for (size_t i = start; i < end; ++i) { auto & file_ctx = findInMap(file_infos.context_by_name, file_infos.file_names[i]); @@ -507,7 +480,7 @@ void StorageFileLog::closeFilesAndStoreMeta(int start, int end) } } -void StorageFileLog::storeMetas(int start, int end) +void StorageFileLog::storeMetas(size_t start, size_t end) { #ifndef NDEBUG assert(start >= 0); @@ -515,7 +488,7 @@ void StorageFileLog::storeMetas(int start, int end) assert(end <= file_infos.file_names.size()); #endif - for (int i = start; i < end; ++i) + for (size_t i = start; i < end; ++i) { auto & file_ctx = findInMap(file_infos.context_by_name, file_infos.file_names[i]); @@ -524,6 +497,25 @@ void StorageFileLog::storeMetas(int start, int end) } } +void StorageFileLog::checkOffsetIsValid(const String & full_name, UInt64 offset) +{ + ReadBufferFromFile in(full_name); + UInt64 _, last_written_pos; + + if (!tryReadIntText(_, in)) + { + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", full_name); + } + assertChar('\n', in); + if (!tryReadIntText(last_written_pos, in)) + { + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", full_name); + } + if (last_written_pos > offset) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Last stored last_written_pos in meta file {} is bigger than current last_written_pos.", full_name); +} + size_t StorageFileLog::getMaxBlockSize() const { return filelog_settings->max_block_size.changed ? filelog_settings->max_block_size.value diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 0251544f33f..f33f6320e1a 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -50,6 +50,12 @@ public: void drop() override; + /// We need to call drop() immediately to remove meta data directory, + /// otherwise, if another filelog table with same name created before + /// the table be dropped finally, then its meta data directory will + /// be deleted by this table drop finally + bool dropTableImmediately() override { return true; } + const auto & getFormatName() const { return format_name; } enum class FileStatus @@ -101,10 +107,10 @@ public: /// Used in shutdown() void closeFilesAndStoreMeta(); /// Used in FileLogSource when finish generating all blocks - void closeFilesAndStoreMeta(int start, int end); + void closeFilesAndStoreMeta(size_t start, size_t end); /// Used in FileLogSource after generating every block - void storeMetas(int start, int end); + void storeMetas(size_t start, size_t end); static void assertStreamGood(const std::ifstream & reader); @@ -187,7 +193,7 @@ private: void serialize(UInt64 inode, const FileMeta & file_meta) const; void deserialize(); - UInt64 getLastWrittenPos(const String & file_name) const; + static void checkOffsetIsValid(const String & full_name, UInt64 offset); }; } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 6ce17552ba1..38f24e0ce29 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -578,6 +578,9 @@ public: /// Does not takes underlying Storage (if any) into account. virtual std::optional lifetimeBytes() const { return {}; } + /// Do we should call table->drop immediately when drop table + virtual bool dropTableImmediately() { return false; } + private: /// Lock required for alter queries (lockForAlter). Always taken for write /// (actually can be replaced with std::mutex, but for consistency we use diff --git a/tests/queries/0_stateless/02026_storage_filelog_largefile.reference b/tests/queries/0_stateless/02026_storage_filelog_largefile.reference new file mode 100644 index 00000000000..95240890a95 --- /dev/null +++ b/tests/queries/0_stateless/02026_storage_filelog_largefile.reference @@ -0,0 +1,3 @@ +2000000 +2000000 +2000000 diff --git a/tests/queries/0_stateless/02026_storage_filelog_largefile.sh b/tests/queries/0_stateless/02026_storage_filelog_largefile.sh new file mode 100755 index 00000000000..8ee6f2e6a0c --- /dev/null +++ b/tests/queries/0_stateless/02026_storage_filelog_largefile.sh @@ -0,0 +1,45 @@ +#!/usr/bin/env bash +# Tags: long, no-parallel + +set -eu + +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')" +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 ${user_files_path}/logs/ + +rm -rf ${user_files_path}/logs/* + +for i in {1..200} +do + ${CLICKHOUSE_CLIENT} --query "insert into function file('${user_files_path}/logs/test$i.csv', 'CSV', 'k UInt32, v UInt32') select number, number from numbers(10000);" +done + +${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt32, v UInt32) engine=FileLog('${user_files_path}/logs/', 'CSV');" + +${CLICKHOUSE_CLIENT} --query "select count() from file_log " + +for i in {201..400} +do + ${CLICKHOUSE_CLIENT} --query "insert into function file('${user_files_path}/logs/test$i.csv', 'CSV', 'k UInt32, v UInt32') select number, number from numbers(10000);" +done + +${CLICKHOUSE_CLIENT} --query "select count() from file_log " + +for i in {401..600} +do + ${CLICKHOUSE_CLIENT} --query "insert into function file('${user_files_path}/logs/test$i.csv', 'CSV', 'k UInt32, v UInt32') select number, number from numbers(10000);" +done + +${CLICKHOUSE_CLIENT} --query "select count() from file_log " + +${CLICKHOUSE_CLIENT} --query "drop table file_log;" + +rm -rf ${user_files_path}/logs From 1f91e2de469e9ad02a8d525de0b2adb489b49084 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 6 Oct 2021 12:06:40 +0000 Subject: [PATCH 106/919] Update install.sh --- src/Storages/ExternalDataSourceConfiguration.cpp | 2 +- tests/config/install.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index d88cd1982e1..c0a663bcb6f 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -66,7 +66,7 @@ std::optional getExternalDataSourceConfiguration(const if (!config.has(config_prefix)) { - /// For table function remote we do not throw on no collection, becuase then we consifer first arg + /// For table function remote we do not throw on no collection, because then we consider first arg /// as cluster definition from config. if (!throw_on_no_collection) return std::nullopt; diff --git a/tests/config/install.sh b/tests/config/install.sh index 05db883c691..325a08a715f 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -39,6 +39,7 @@ ln -sf $SRC_PATH/config.d/top_level_domains_path.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/encryption.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/zookeeper_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/logger.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/named_collection.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/users.d/log_queries.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/readonly.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/access_management.xml $DEST_SERVER_PATH/users.d/ @@ -46,7 +47,6 @@ ln -sf $SRC_PATH/users.d/database_atomic_drop_detach_sync.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/users.d/opentelemetry.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/remote_queries.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/session_log_test.xml $DEST_SERVER_PATH/users.d/ -ln -sf $SRC_PATH/users.d/memory_profiler.xml $DEST_SERVER_PATH/users.d/ # FIXME DataPartsExchange may hang for http_send_timeout seconds # when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"), From 28f8e1c297738c79446d52f835192de52cd9fd14 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 6 Oct 2021 15:48:23 +0000 Subject: [PATCH 107/919] fix update test --- src/Storages/FileLog/StorageFileLog.cpp | 15 ++++----------- src/Storages/FileLog/StorageFileLog.h | 5 ++--- .../0_stateless/02022_storage_filelog_one_file.sh | 2 +- .../queries/0_stateless/02023_storage_filelog.sh | 2 +- .../02025_storage_filelog_virtual_col.sh | 2 +- .../02026_storage_filelog_largefile.sh | 2 ++ 6 files changed, 11 insertions(+), 17 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 112e68acd08..6489c9270ca 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -403,7 +403,10 @@ void StorageFileLog::shutdown() LOG_TRACE(log, "Waiting for cleanup"); task->holder->deactivate(); } - closeFilesAndStoreMeta(); + /// If no reading call and threadFunc, the log files will never + /// be opened, also just leave the work of close files and + /// store meta to streams. because if we close files in here, + /// may result in data race with unfinishing reading pipeline } catch (...) { @@ -450,16 +453,6 @@ void StorageFileLog::openFilesAndSetPos() serialize(); } -void StorageFileLog::closeFilesAndStoreMeta() -{ - for (auto & it : file_infos.context_by_name) - { - if (it.second.reader.is_open()) - it.second.reader.close(); - } - serialize(); -} - void StorageFileLog::closeFilesAndStoreMeta(size_t start, size_t end) { #ifndef NDEBUG diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index f33f6320e1a..82c7a2dc9f9 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -104,9 +104,8 @@ public: void openFilesAndSetPos(); - /// Used in shutdown() - void closeFilesAndStoreMeta(); - /// Used in FileLogSource when finish generating all blocks + /// Used in FileLogSource when finish generating all blocks. + /// Each stream responsible for close its files and store meta. void closeFilesAndStoreMeta(size_t start, size_t end); /// Used in FileLogSource after generating every block diff --git a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh index d1ed3e5a142..600e537a352 100755 --- a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: no-parallel set -eu diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index b818093dfdb..b695c270835 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: no-parallel set -eu diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index b8b7c980268..781253c2184 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: no-parallel set -eu diff --git a/tests/queries/0_stateless/02026_storage_filelog_largefile.sh b/tests/queries/0_stateless/02026_storage_filelog_largefile.sh index 8ee6f2e6a0c..a6c5d19287e 100755 --- a/tests/queries/0_stateless/02026_storage_filelog_largefile.sh +++ b/tests/queries/0_stateless/02026_storage_filelog_largefile.sh @@ -16,6 +16,8 @@ mkdir -p ${user_files_path}/logs/ rm -rf ${user_files_path}/logs/* +chmod 777 ${user_files_path}/logs/ + for i in {1..200} do ${CLICKHOUSE_CLIENT} --query "insert into function file('${user_files_path}/logs/test$i.csv', 'CSV', 'k UInt32, v UInt32') select number, number from numbers(10000);" From d9959a4f22bec685f084a5be160ff8de1470d786 Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 7 Oct 2021 05:56:09 +0000 Subject: [PATCH 108/919] fix --- src/Storages/FileLog/FileLogSource.cpp | 2 ++ src/Storages/FileLog/FileLogSource.h | 1 + src/Storages/FileLog/StorageFileLog.cpp | 16 ++++++++++++++++ src/Storages/FileLog/StorageFileLog.h | 9 +++++++++ 4 files changed, 28 insertions(+) diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index eb8cb2b28f9..0c266178142 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -42,6 +42,8 @@ FileLogSource::FileLogSource( size_t files_per_stream = file_infos.file_names.size() / max_streams_number; start = stream_number * files_per_stream; end = stream_number == max_streams_number - 1 ? file_infos.file_names.size() : (stream_number + 1) * files_per_stream; + + storage.increaseStreams(); } void FileLogSource::onFinish() diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index 0ff71c1af87..cdf60cd4fc1 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -34,6 +34,7 @@ public: { if (!finished) onFinish(); + storage.reduceStreams(); } protected: diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 6489c9270ca..fb13137ce31 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -42,6 +42,7 @@ namespace ErrorCodes extern const int CANNOT_READ_ALL_DATA; extern const int LOGICAL_ERROR; extern const int TABLE_METADATA_ALREADY_EXISTS; + extern const int CANNOT_SELECT; } namespace @@ -326,6 +327,11 @@ Pipe StorageFileLog::read( table_id.getTableName()); } + if (running_streams.load(std::memory_order_relaxed)) + { + throw Exception("Another select query is running on this table, need to wait it finish.", ErrorCodes::CANNOT_SELECT); + } + /// We need this lock, in case read and streamToViews execute at the same time. /// In case of MV attached during reading std::lock_guard lock(status_mutex); @@ -364,6 +370,16 @@ Pipe StorageFileLog::read( return Pipe::unitePipes(std::move(pipes)); } +void StorageFileLog::increaseStreams() +{ + running_streams.fetch_add(1, std::memory_order_relaxed); +} + +void StorageFileLog::reduceStreams() +{ + running_streams.fetch_sub(1, std::memory_order_relaxed); +} + void StorageFileLog::drop() { try diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 82c7a2dc9f9..bdab85866c0 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -122,6 +122,9 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "The key {} doesn't exist.", key); } + void increaseStreams(); + void reduceStreams(); + protected: StorageFileLog( const StorageID & table_id_, @@ -169,6 +172,12 @@ private: }; std::shared_ptr task; + /// In order to avoid data race, using a naive trick to forbid execute two select + /// simultaneously, although read is not useful in this engine. Using an atomic + /// variable to records current unfinishing streams, then if have unfinishing streams, + /// later select should forbid to execute. + std::atomic running_streams = 0; + using TaskThread = BackgroundSchedulePool::TaskHolder; void loadFiles(); From 127a0f35910a097b5240d68cc60f5e30b9e697ac Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 7 Oct 2021 07:31:41 +0000 Subject: [PATCH 109/919] fix --- src/Storages/FileLog/StorageFileLog.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index fb13137ce31..63337b9b5c2 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -879,8 +879,6 @@ bool StorageFileLog::updateFileInfos() { if (it->second.status == FileStatus::REMOVED) { - file_infos.context_by_name.erase(it); - /// We need to check that this inode does not hold by other file(mv), /// otherwise, we can not destroy it. auto inode = it->second.inode; @@ -892,6 +890,7 @@ bool StorageFileLog::updateFileInfos() if (std::filesystem::exists(getFullMetaPath(file_name))) std::filesystem::remove(getFullMetaPath(file_name)); + file_infos.context_by_name.erase(it); } else { From bd1ea5cd1630c2b8d73c8234c840cdf84649a903 Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 7 Oct 2021 12:15:36 +0000 Subject: [PATCH 110/919] try fix undefine in ifstream --- .../FileLog/ReadBufferFromFileLog.cpp | 20 +++++++++++----- src/Storages/FileLog/StorageFileLog.cpp | 24 +++++++++++-------- src/Storages/FileLog/StorageFileLog.h | 3 ++- 3 files changed, 30 insertions(+), 17 deletions(-) diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 2fc0e0885a1..cf52b2a2ccc 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -10,6 +10,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_READ_ALL_DATA; +} ReadBufferFromFileLog::ReadBufferFromFileLog( StorageFileLog & storage_, @@ -102,23 +106,27 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ auto & file_meta = StorageFileLog::findInMap(file_infos.meta_by_inode, file_ctx.inode); + if (!file_ctx.reader) + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Ifstream for file {} does not initialized.", file_meta.file_name); + + auto & reader = file_ctx.reader.value(); Record record; - while (read_records_size < need_records_size && static_cast(file_ctx.reader.tellg()) < file_meta.last_open_end) + while (read_records_size < need_records_size && static_cast(reader.tellg()) < file_meta.last_open_end) { /// Need to get offset before reading record from stream - record.offset = file_ctx.reader.tellg(); + record.offset = reader.tellg(); record.file_name = file_name; - StorageFileLog::assertStreamGood(file_ctx.reader); + StorageFileLog::assertStreamGood(reader); - std::getline(file_ctx.reader, record.data); + std::getline(reader, record.data); new_records.emplace_back(record); ++read_records_size; } - UInt64 current_position = file_ctx.reader.tellg(); - StorageFileLog::assertStreamGood(file_ctx.reader); + UInt64 current_position = reader.tellg(); + StorageFileLog::assertStreamGood(reader); file_meta.last_writen_position = current_position; diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 63337b9b5c2..c9b29a5a9d1 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -445,14 +445,15 @@ void StorageFileLog::openFilesAndSetPos() auto & file_ctx = findInMap(file_infos.context_by_name, file); if (file_ctx.status != FileStatus::NO_CHANGE) { - file_ctx.reader.open(getFullDataPath(file)); - assertStreamGood(file_ctx.reader); + file_ctx.reader.emplace(getFullDataPath(file)); + auto & reader = file_ctx.reader.value(); + assertStreamGood(reader); - file_ctx.reader.seekg(0, file_ctx.reader.end); - assertStreamGood(file_ctx.reader); + reader.seekg(0, reader.end); + assertStreamGood(reader); - auto file_end = file_ctx.reader.tellg(); - assertStreamGood(file_ctx.reader); + auto file_end = reader.tellg(); + assertStreamGood(reader); auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); if (meta.last_writen_position > static_cast(file_end)) @@ -462,8 +463,8 @@ void StorageFileLog::openFilesAndSetPos() /// update file end at the monment, used in ReadBuffer and serialize meta.last_open_end = file_end; - file_ctx.reader.seekg(meta.last_writen_position); - assertStreamGood(file_ctx.reader); + reader.seekg(meta.last_writen_position); + assertStreamGood(reader); } } serialize(); @@ -481,8 +482,11 @@ void StorageFileLog::closeFilesAndStoreMeta(size_t start, size_t end) { auto & file_ctx = findInMap(file_infos.context_by_name, file_infos.file_names[i]); - if (file_ctx.reader.is_open()) - file_ctx.reader.close(); + if (file_ctx.reader) + { + if ((*file_ctx.reader).is_open()) + (*file_ctx.reader).close(); + } auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); serialize(file_ctx.inode, meta); diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index bdab85866c0..f3b85d47313 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB { @@ -70,7 +71,7 @@ public: { FileStatus status = FileStatus::OPEN; UInt64 inode{}; - std::ifstream reader{}; + std::optional reader = std::nullopt; }; struct FileMeta From 4233e2cca452a3f987e774d1066194a68769e207 Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 7 Oct 2021 13:23:25 +0000 Subject: [PATCH 111/919] fix --- src/Storages/FileLog/FileLogSource.cpp | 2 -- src/Storages/FileLog/StorageFileLog.cpp | 5 ++--- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index 0c266178142..7beee42c99c 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -1,5 +1,3 @@ -#include -#include #include #include #include diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index c9b29a5a9d1..a75c9b0190e 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include @@ -484,8 +483,8 @@ void StorageFileLog::closeFilesAndStoreMeta(size_t start, size_t end) if (file_ctx.reader) { - if ((*file_ctx.reader).is_open()) - (*file_ctx.reader).close(); + if (file_ctx.reader->is_open()) + file_ctx.reader->close(); } auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); From 757f49bacbd9ced52d3f0b333601fbe6f75e6ead Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 8 Oct 2021 11:15:47 +0000 Subject: [PATCH 112/919] test poco --- .gitmodules | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.gitmodules b/.gitmodules index 74d1049ce01..7f5c60d2b0c 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,6 +1,6 @@ [submodule "contrib/poco"] path = contrib/poco - url = https://github.com/ClickHouse-Extras/poco.git + url = https://github.com/ucasfl/poco.git branch = clickhouse [submodule "contrib/zstd"] path = contrib/zstd From f3f820c6e027d332f1da8262d1764dca7553bf25 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 7 Oct 2021 13:39:54 +0000 Subject: [PATCH 113/919] HTTP read buffer retries --- src/Bridge/LibraryBridgeHelper.cpp | 1 + src/Core/Settings.h | 8 +- src/Dictionaries/HTTPDictionarySource.cpp | 4 + src/Disks/DiskWebServer.cpp | 2 +- src/Disks/ReadIndirectBufferFromWebServer.cpp | 52 ++------- src/Disks/ReadIndirectBufferFromWebServer.h | 3 - src/IO/ReadSettings.h | 9 +- src/IO/ReadWriteBufferFromHTTP.h | 101 ++++++++++++++++-- src/Interpreters/Context.cpp | 9 +- src/Storages/StorageURL.cpp | 1 + 10 files changed, 129 insertions(+), 61 deletions(-) diff --git a/src/Bridge/LibraryBridgeHelper.cpp b/src/Bridge/LibraryBridgeHelper.cpp index 7f3e3f8009c..d9b067ba743 100644 --- a/src/Bridge/LibraryBridgeHelper.cpp +++ b/src/Bridge/LibraryBridgeHelper.cpp @@ -257,6 +257,7 @@ Pipe LibraryBridgeHelper::loadBase(const Poco::URI & uri, ReadWriteBufferFromHTT 0, Poco::Net::HTTPBasicCredentials{}, DBMS_DEFAULT_BUFFER_SIZE, + getContext()->getReadSettings(), ReadWriteBufferFromHTTP::HTTPHeaderEntries{}); auto input_stream = getContext()->getInputFormat(LibraryBridgeHelper::DEFAULT_FORMAT, *read_buf_ptr, sample_block, DEFAULT_BLOCK_SIZE); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f55f10c0267..3cc5f5f3d6c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -75,7 +75,6 @@ class IColumn; M(UInt64, s3_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \ M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \ M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \ - M(UInt64, http_max_single_read_retries, 4, "The maximum number of retries during single http read.", 0) \ M(UInt64, hsts_max_age, 0, "Expired time for hsts. 0 means disable HSTS.", 0) \ M(Bool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \ M(Bool, use_uncompressed_cache, false, "Whether to use the cache of uncompressed blocks.", 0) \ @@ -520,9 +519,14 @@ class IColumn; M(Milliseconds, async_insert_busy_timeout_ms, 200, "Maximum time to wait before dumping collected data per query since the first data appeared", 0) \ M(Milliseconds, async_insert_stale_timeout_ms, 0, "Maximum time to wait before dumping collected data per query since the last data appeared. Zero means no timeout at all", 0) \ \ - M(Int64, remote_fs_read_backoff_threshold, 10000, "Max wait time when trying to read data for remote disk", 0) \ + M(Int64, remote_fs_read_max_backoff_ms, 10000, "Max wait time when trying to read data for remote disk", 0) \ M(Int64, remote_fs_read_backoff_max_tries, 5, "Max attempts to read with backoff", 0) \ \ + M(Int64, http_max_tries, 1, "Max attempts to read via http.", 0) \ + M(Int64, http_retry_initial_backoff_ms, 100, "Min milliseconds for backoff, when retrying read via http", 0) \ + M(Int64, http_retry_max_backoff_ms, 10000, "Max milliseconds for backoff, when retrying read via http", 0) \ + M(Bool, http_retriable_read, true, "Allow to resume reading via http if some error occurred. Reading will continue starting from last read byte (with `range` header)", 0) \ + \ /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index d6a0c25e9f1..a05f1278313 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -103,6 +103,7 @@ Pipe HTTPDictionarySource::loadAll() 0, credentials, DBMS_DEFAULT_BUFFER_SIZE, + context->getReadSettings(), configuration.header_entries); return createWrappedBuffer(std::move(in_ptr)); @@ -121,6 +122,7 @@ Pipe HTTPDictionarySource::loadUpdatedAll() 0, credentials, DBMS_DEFAULT_BUFFER_SIZE, + context->getReadSettings(), configuration.header_entries); return createWrappedBuffer(std::move(in_ptr)); @@ -148,6 +150,7 @@ Pipe HTTPDictionarySource::loadIds(const std::vector & ids) 0, credentials, DBMS_DEFAULT_BUFFER_SIZE, + context->getReadSettings(), configuration.header_entries); return createWrappedBuffer(std::move(in_ptr)); @@ -175,6 +178,7 @@ Pipe HTTPDictionarySource::loadKeys(const Columns & key_columns, const std::vect 0, credentials, DBMS_DEFAULT_BUFFER_SIZE, + context->getReadSettings(), configuration.header_entries); return createWrappedBuffer(std::move(in_ptr)); diff --git a/src/Disks/DiskWebServer.cpp b/src/Disks/DiskWebServer.cpp index 347ece5a78d..9d7b286c81c 100644 --- a/src/Disks/DiskWebServer.cpp +++ b/src/Disks/DiskWebServer.cpp @@ -197,7 +197,7 @@ std::unique_ptr DiskWebServer::readFile(const String & p meta.remote_fs_objects.emplace_back(std::make_pair(remote_path, iter->second.size)); auto reader = std::make_unique(url, meta, getContext(), - read_settings.remote_fs_buffer_size, read_settings.remote_fs_backoff_threshold, read_settings.remote_fs_backoff_max_tries); + read_settings.remote_fs_buffer_size, read_settings.remote_fs_read_max_backoff_ms, read_settings.remote_fs_read_backoff_max_tries); return std::make_unique(std::move(reader), min_bytes_for_seek); } diff --git a/src/Disks/ReadIndirectBufferFromWebServer.cpp b/src/Disks/ReadIndirectBufferFromWebServer.cpp index f4c01784542..a88f9d51751 100644 --- a/src/Disks/ReadIndirectBufferFromWebServer.cpp +++ b/src/Disks/ReadIndirectBufferFromWebServer.cpp @@ -20,18 +20,13 @@ namespace ErrorCodes extern const int NETWORK_ERROR; } -static const auto WAIT_MS = 10; - - ReadIndirectBufferFromWebServer::ReadIndirectBufferFromWebServer( - const String & url_, ContextPtr context_, size_t buf_size_, size_t backoff_threshold_, size_t max_tries_) + const String & url_, ContextPtr context_, size_t buf_size_, size_t, size_t) : BufferWithOwnMemory(buf_size_) , log(&Poco::Logger::get("ReadIndirectBufferFromWebServer")) , context(context_) , url(url_) , buf_size(buf_size_) - , backoff_threshold_ms(backoff_threshold_) - , max_tries(max_tries_) { } @@ -59,65 +54,32 @@ std::unique_ptr ReadIndirectBufferFromWebServer::initialize() 0, Poco::Net::HTTPBasicCredentials{}, buf_size, + context->getReadSettings(), headers); } bool ReadIndirectBufferFromWebServer::nextImpl() { - bool next_result = false, successful_read = false; - UInt16 milliseconds_to_wait = WAIT_MS; - if (impl) { /// Restore correct position at the needed offset. impl->position() = position(); assert(!impl->hasPendingData()); } - - WriteBufferFromOwnString error_msg; - for (size_t i = 0; (i < max_tries) && !successful_read && !next_result; ++i) + else { - while (milliseconds_to_wait < backoff_threshold_ms) - { - try - { - if (!impl) - { - impl = initialize(); - next_result = impl->hasPendingData(); - if (next_result) - break; - } - - next_result = impl->next(); - successful_read = true; - break; - } - catch (const Poco::Exception & e) - { - LOG_WARNING(log, "Read attempt failed for url: {}. Error: {}", url, e.what()); - error_msg << fmt::format("Error: {}\n", e.what()); - - sleepForMilliseconds(milliseconds_to_wait); - milliseconds_to_wait *= 2; - impl.reset(); - } - } - milliseconds_to_wait = WAIT_MS; + impl = initialize(); } - if (!successful_read) - throw Exception(ErrorCodes::NETWORK_ERROR, - "All read attempts failed for url: {}. Reason:\n{}", url, error_msg.str()); - - if (next_result) + auto result = impl->next(); + if (result) { BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset()); offset += working_buffer.size(); } - return next_result; + return result; } diff --git a/src/Disks/ReadIndirectBufferFromWebServer.h b/src/Disks/ReadIndirectBufferFromWebServer.h index 04bb155f83b..aa0b9eafbe8 100644 --- a/src/Disks/ReadIndirectBufferFromWebServer.h +++ b/src/Disks/ReadIndirectBufferFromWebServer.h @@ -39,9 +39,6 @@ private: std::unique_ptr impl; off_t offset = 0; - - size_t backoff_threshold_ms; - size_t max_tries; }; } diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index 379b7bc2216..8c7fbd473ab 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -66,8 +66,13 @@ struct ReadSettings /// For 'pread_threadpool' method. Lower is more priority. size_t priority = 0; - size_t remote_fs_backoff_threshold = 10000; - size_t remote_fs_backoff_max_tries = 4; + size_t remote_fs_read_max_backoff_ms = 10000; + size_t remote_fs_read_backoff_max_tries = 4; + + bool http_retriable_read = true; + size_t http_max_tries = 1; + size_t http_retry_initial_backoff_ms = 100; + size_t http_retry_max_backoff_ms = 10000; ReadSettings adjustBufferSize(size_t file_size) const { diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index a42404cc631..3908e64c4a0 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -2,10 +2,12 @@ #include #include +#include #include #include #include #include +#include #include #include #include @@ -17,6 +19,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) # include @@ -101,6 +104,15 @@ namespace detail RemoteHostFilter remote_host_filter; std::function next_callback; + size_t buffer_size; + + size_t bytes_read = 0; + size_t start_byte = 0; + std::optional total_bytes_to_read; + + ReadSettings settings; + bool retry_partially_read = true; + std::istream * call(Poco::URI uri_, Poco::Net::HTTPResponse & response) { // With empty path poco will send "POST HTTP/1.1" its bug. @@ -118,6 +130,9 @@ namespace detail request.set(std::get<0>(http_header_entry), std::get<1>(http_header_entry)); } + if (bytes_read && retry_partially_read) + request.set("Range", fmt::format("bytes={}-", start_byte + bytes_read)); + if (!credentials.getUsername().empty()) credentials.authenticate(request); @@ -159,6 +174,7 @@ namespace detail OutStreamCallback out_stream_callback_ = {}, const Poco::Net::HTTPBasicCredentials & credentials_ = {}, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, + const ReadSettings & settings_ = {}, HTTPHeaderEntries http_header_entries_ = {}, const RemoteHostFilter & remote_host_filter_ = {}) : ReadBuffer(nullptr, 0) @@ -169,9 +185,29 @@ namespace detail , credentials {credentials_} , http_header_entries {http_header_entries_} , remote_host_filter {remote_host_filter_} + , buffer_size {buffer_size_} + , settings {settings_} { - Poco::Net::HTTPResponse response; + /** + * Get first byte from `bytes=offset-`, `bytes=offset-end`. + * Now there are two places, where it can be set: 1. in DiskWeb (offset), 2. via config as part of named-collection. + * Also header can be `bytes=-k` (read last k bytes), for this case retries in the middle of reading are disabled. + */ + auto range_header = std::find_if(http_header_entries_.begin(), http_header_entries_.end(), + [&](const HTTPHeaderEntry & header) { return std::get<0>(header) == "Range"; }); + if (range_header != http_header_entries_.end()) + { + auto range = std::get<1>(*range_header).substr(std::strlen("bytes=")); + auto [ptr, ec] = std::from_chars(range.data(), range.data() + range.size(), start_byte); + if (ec != std::errc()) + retry_partially_read = false; + } + } + void initialize() + { + + Poco::Net::HTTPResponse response; istr = call(uri, response); while (isRedirect(response.getStatus())) @@ -184,9 +220,19 @@ namespace detail istr = call(uri_redirect, response); } + /// If it is the very first initialization. + if (!bytes_read && !total_bytes_to_read) + { + /// If we do not know total size, disable retries in the middle of reading. + if (response.hasContentLength()) + total_bytes_to_read = response.getContentLength(); + else + retry_partially_read = false; + } + try { - impl = std::make_unique(*istr, buffer_size_); + impl = std::make_unique(*istr, buffer_size); } catch (const Poco::Exception & e) { @@ -202,12 +248,54 @@ namespace detail { if (next_callback) next_callback(count()); - if (!working_buffer.empty()) - impl->position() = position(); - if (!impl->next()) + + if (total_bytes_to_read && bytes_read == total_bytes_to_read.value()) return false; + + if (impl && !working_buffer.empty()) + impl->position() = position(); + + bool result = false; + bool successful_read = false; + size_t milliseconds_to_wait = settings.http_retry_initial_backoff_ms; + + /// Default http_max_tries = 1. + for (size_t i = 0; (i < settings.http_max_tries) && !successful_read; ++i) + { + while (milliseconds_to_wait < settings.http_retry_max_backoff_ms) + { + try + { + if (!impl) + initialize(); + + result = impl->next(); + successful_read = true; + break; + } + catch (const Poco::Exception &) + { + if (!settings.http_retriable_read + || i == settings.http_max_tries - 1 + || (bytes_read && !retry_partially_read)) + throw; + + tryLogCurrentException(__PRETTY_FUNCTION__); + impl.reset(); + + sleepForMilliseconds(milliseconds_to_wait); + milliseconds_to_wait *= 2; + } + } + milliseconds_to_wait = settings.http_retry_initial_backoff_ms; + } + + if (!result) + return false; + internal_buffer = impl->buffer(); working_buffer = internal_buffer; + bytes_read += working_buffer.size(); return true; } @@ -270,10 +358,11 @@ public: const UInt64 max_redirects = 0, const Poco::Net::HTTPBasicCredentials & credentials_ = {}, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, + const ReadSettings & settings_ = {}, const HTTPHeaderEntries & http_header_entries_ = {}, const RemoteHostFilter & remote_host_filter_ = {}) : Parent(std::make_shared(uri_, timeouts, max_redirects), - uri_, method_, out_stream_callback_, credentials_, buffer_size_, http_header_entries_, remote_host_filter_) + uri_, method_, out_stream_callback_, credentials_, buffer_size_, settings_, http_header_entries_, remote_host_filter_) { } }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 1a95a642e18..6e668d410a0 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2929,14 +2929,19 @@ ReadSettings Context::getReadSettings() const res.local_fs_prefetch = settings.local_filesystem_read_prefetch; res.remote_fs_prefetch = settings.remote_filesystem_read_prefetch; - res.remote_fs_backoff_threshold = settings.remote_fs_read_backoff_threshold; - res.remote_fs_backoff_max_tries = settings.remote_fs_read_backoff_max_tries; + res.remote_fs_read_max_backoff_ms = settings.remote_fs_read_max_backoff_ms; + res.remote_fs_read_backoff_max_tries = settings.remote_fs_read_backoff_max_tries; res.local_fs_buffer_size = settings.max_read_buffer_size; res.direct_io_threshold = settings.min_bytes_to_use_direct_io; res.mmap_threshold = settings.min_bytes_to_use_mmap_io; res.priority = settings.read_priority; + res.http_retriable_read = settings.http_retriable_read; + res.http_max_tries = settings.http_max_tries; + res.http_retry_initial_backoff_ms = settings.http_retry_initial_backoff_ms; + res.http_retry_max_backoff_ms = settings.http_retry_max_backoff_ms; + res.mmap_cache = getMMappedFileCache().get(); return res; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index cbd935f7909..8813a9fddbf 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -124,6 +124,7 @@ namespace context->getSettingsRef().max_http_get_redirects, Poco::Net::HTTPBasicCredentials{}, DBMS_DEFAULT_BUFFER_SIZE, + context->getReadSettings(), headers, context->getRemoteHostFilter()), chooseCompressionMethod(request_uri.getPath(), compression_method)); From 40434519e864300d9ff077597e658cb36ed07c98 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 8 Oct 2021 15:54:24 +0300 Subject: [PATCH 114/919] Fix disk web --- src/Disks/DiskWebServer.cpp | 16 +++++++--------- src/Disks/ReadIndirectBufferFromWebServer.cpp | 5 +++-- src/Disks/ReadIndirectBufferFromWebServer.h | 4 +++- src/IO/ReadWriteBufferFromHTTP.h | 12 +++++------- 4 files changed, 18 insertions(+), 19 deletions(-) diff --git a/src/Disks/DiskWebServer.cpp b/src/Disks/DiskWebServer.cpp index 9d7b286c81c..5d53dd8dea7 100644 --- a/src/Disks/DiskWebServer.cpp +++ b/src/Disks/DiskWebServer.cpp @@ -113,28 +113,25 @@ public: RemoteMetadata metadata_, ContextPtr context_, size_t buf_size_, - size_t backoff_threshold_, - size_t max_tries_) + const ReadSettings & settings_) : ReadIndirectBufferFromRemoteFS(metadata_) , uri(uri_) , context(context_) , buf_size(buf_size_) - , backoff_threshold(backoff_threshold_) - , max_tries(max_tries_) + , settings(settings_) { } std::unique_ptr createReadBuffer(const String & path) override { - return std::make_unique(fs::path(uri) / path, context, buf_size, backoff_threshold, max_tries); + return std::make_unique(fs::path(uri) / path, context, buf_size, settings); } private: String uri; ContextPtr context; size_t buf_size; - size_t backoff_threshold; - size_t max_tries; + ReadSettings settings; }; @@ -196,8 +193,9 @@ std::unique_ptr DiskWebServer::readFile(const String & p RemoteMetadata meta(path, remote_path); meta.remote_fs_objects.emplace_back(std::make_pair(remote_path, iter->second.size)); - auto reader = std::make_unique(url, meta, getContext(), - read_settings.remote_fs_buffer_size, read_settings.remote_fs_read_max_backoff_ms, read_settings.remote_fs_read_backoff_max_tries); + auto reader = std::make_unique(url, meta, getContext(), read_settings.remote_fs_buffer_size, + read_settings); + return std::make_unique(std::move(reader), min_bytes_for_seek); } diff --git a/src/Disks/ReadIndirectBufferFromWebServer.cpp b/src/Disks/ReadIndirectBufferFromWebServer.cpp index a88f9d51751..57ee0614563 100644 --- a/src/Disks/ReadIndirectBufferFromWebServer.cpp +++ b/src/Disks/ReadIndirectBufferFromWebServer.cpp @@ -21,12 +21,13 @@ namespace ErrorCodes } ReadIndirectBufferFromWebServer::ReadIndirectBufferFromWebServer( - const String & url_, ContextPtr context_, size_t buf_size_, size_t, size_t) + const String & url_, ContextPtr context_, size_t buf_size_, const ReadSettings & settings_) : BufferWithOwnMemory(buf_size_) , log(&Poco::Logger::get("ReadIndirectBufferFromWebServer")) , context(context_) , url(url_) , buf_size(buf_size_) + , read_settings(settings_) { } @@ -54,7 +55,7 @@ std::unique_ptr ReadIndirectBufferFromWebServer::initialize() 0, Poco::Net::HTTPBasicCredentials{}, buf_size, - context->getReadSettings(), + read_settings, headers); } diff --git a/src/Disks/ReadIndirectBufferFromWebServer.h b/src/Disks/ReadIndirectBufferFromWebServer.h index aa0b9eafbe8..c65b7d91821 100644 --- a/src/Disks/ReadIndirectBufferFromWebServer.h +++ b/src/Disks/ReadIndirectBufferFromWebServer.h @@ -2,6 +2,7 @@ #include #include +#include #include @@ -19,7 +20,7 @@ public: explicit ReadIndirectBufferFromWebServer(const String & url_, ContextPtr context_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE, - size_t backoff_threshold_ = 10000, size_t max_tries_ = 4); + const ReadSettings & settings_ = {}); bool nextImpl() override; @@ -39,6 +40,7 @@ private: std::unique_ptr impl; off_t offset = 0; + ReadSettings read_settings; }; } diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 3908e64c4a0..d57c25adbe0 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -111,7 +111,6 @@ namespace detail std::optional total_bytes_to_read; ReadSettings settings; - bool retry_partially_read = true; std::istream * call(Poco::URI uri_, Poco::Net::HTTPResponse & response) { @@ -130,7 +129,7 @@ namespace detail request.set(std::get<0>(http_header_entry), std::get<1>(http_header_entry)); } - if (bytes_read && retry_partially_read) + if (bytes_read && settings.http_retriable_read) request.set("Range", fmt::format("bytes={}-", start_byte + bytes_read)); if (!credentials.getUsername().empty()) @@ -200,7 +199,7 @@ namespace detail auto range = std::get<1>(*range_header).substr(std::strlen("bytes=")); auto [ptr, ec] = std::from_chars(range.data(), range.data() + range.size(), start_byte); if (ec != std::errc()) - retry_partially_read = false; + settings.http_retriable_read = false; } } @@ -227,7 +226,7 @@ namespace detail if (response.hasContentLength()) total_bytes_to_read = response.getContentLength(); else - retry_partially_read = false; + settings.http_retriable_read = false; } try @@ -275,9 +274,8 @@ namespace detail } catch (const Poco::Exception &) { - if (!settings.http_retriable_read - || i == settings.http_max_tries - 1 - || (bytes_read && !retry_partially_read)) + if (i == settings.http_max_tries - 1 + || (bytes_read && !settings.http_retriable_read)) throw; tryLogCurrentException(__PRETTY_FUNCTION__); From 24feb6674c11b690c3722b4f0fc1aff492da9479 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 8 Oct 2021 15:38:33 +0000 Subject: [PATCH 115/919] May be move back initialize to constructor --- src/IO/ReadWriteBufferFromHTTP.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index d57c25adbe0..b7ecd81286a 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -201,6 +201,8 @@ namespace detail if (ec != std::errc()) settings.http_retriable_read = false; } + + initialize(); } void initialize() From ddf9e524cc1012271f433cb526bc921acd7c5376 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 9 Oct 2021 05:14:33 +0000 Subject: [PATCH 116/919] test poco --- contrib/poco | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index 39fd359765a..c273106994a 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 39fd359765a3a77b46d94ec3c5def3c7802a920f +Subproject commit c273106994a57b916a446de6afdc8489867f67fb From aae07e1947c27412ac9ea13c29cfd58d48fb8153 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 9 Oct 2021 10:34:57 +0000 Subject: [PATCH 117/919] run test again --- contrib/poco | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index c273106994a..5e822a5015a 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit c273106994a57b916a446de6afdc8489867f67fb +Subproject commit 5e822a5015a431a288c4b364fd82511d95731f78 From 0e8d2f33d76d0644c9aa673a76248c7be67f76d5 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 9 Oct 2021 10:51:10 +0000 Subject: [PATCH 118/919] update poco --- contrib/poco | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index 5e822a5015a..ab0b711c68d 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 5e822a5015a431a288c4b364fd82511d95731f78 +Subproject commit ab0b711c68d27feabc2e82c840bd5225b185ef4a From fe12404cb017e2f41ae9d46c9d679cde765156ba Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 9 Oct 2021 15:14:23 +0300 Subject: [PATCH 119/919] Fix tests test_log_family_s3, test_log_family_hdfs. --- tests/integration/test_log_family_hdfs/test.py | 14 ++++++++++++-- tests/integration/test_log_family_s3/test.py | 14 ++++++++++++-- 2 files changed, 24 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_log_family_hdfs/test.py b/tests/integration/test_log_family_hdfs/test.py index a38b067358e..7bb9cdfeaf5 100644 --- a/tests/integration/test_log_family_hdfs/test.py +++ b/tests/integration/test_log_family_hdfs/test.py @@ -31,10 +31,20 @@ def assert_objects_count(started_cluster, objects_count, path='data/'): hdfs_objects = fs.listdir('/clickhouse') assert objects_count == len(hdfs_objects) - +# TinyLog: files: id.bin, sizes.json +# INSERT overwrites 1 file (`sizes.json`) and appends 1 file (`id.bin`), so +# files_overhead=1, files_overhead_per_insert=1 +# +# Log: files: id.bin, __marks.mrk, sizes.json +# INSERT overwrites 1 file (`sizes.json`), and appends 2 files (`id.bin`, `__marks.mrk`), so +# files_overhead=1, files_overhead_per_insert=2 +# +# StripeLog: files: data.bin, index.mrk, sizes.json +# INSERT overwrites 1 file (`sizes.json`), and appends 2 files (`index.mrk`, `data.bin`), so +# files_overhead=1, files_overhead_per_insert=2 @pytest.mark.parametrize( "log_engine,files_overhead,files_overhead_per_insert", - [("TinyLog", 1, 1), ("Log", 2, 1), ("StripeLog", 1, 2)]) + [("TinyLog", 1, 1), ("Log", 1, 2), ("StripeLog", 1, 2)]) def test_log_family_hdfs(started_cluster, log_engine, files_overhead, files_overhead_per_insert): node = started_cluster.instances["node"] diff --git a/tests/integration/test_log_family_s3/test.py b/tests/integration/test_log_family_s3/test.py index 71d47a8a2e8..8531edd635f 100644 --- a/tests/integration/test_log_family_s3/test.py +++ b/tests/integration/test_log_family_s3/test.py @@ -30,10 +30,20 @@ def assert_objects_count(cluster, objects_count, path='data/'): logging.info("Existing S3 object: %s", str(object_meta)) assert objects_count == len(s3_objects) - +# TinyLog: files: id.bin, sizes.json +# INSERT overwrites 1 file (`sizes.json`) and appends 1 file (`id.bin`), so +# files_overhead=1, files_overhead_per_insert=1 +# +# Log: files: id.bin, __marks.mrk, sizes.json +# INSERT overwrites 1 file (`sizes.json`), and appends 2 files (`id.bin`, `__marks.mrk`), so +# files_overhead=1, files_overhead_per_insert=2 +# +# StripeLog: files: data.bin, index.mrk, sizes.json +# INSERT overwrites 1 file (`sizes.json`), and appends 2 files (`index.mrk`, `data.bin`), so +# files_overhead=1, files_overhead_per_insert=2 @pytest.mark.parametrize( "log_engine,files_overhead,files_overhead_per_insert", - [("TinyLog", 1, 1), ("Log", 2, 1), ("StripeLog", 1, 2)]) + [("TinyLog", 1, 1), ("Log", 1, 2), ("StripeLog", 1, 2)]) def test_log_family_s3(cluster, log_engine, files_overhead, files_overhead_per_insert): node = cluster.instances["node"] From 764b17787b7c260b71662f703b90094d130063c6 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 9 Oct 2021 14:25:04 +0000 Subject: [PATCH 120/919] update poco --- contrib/poco | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index ab0b711c68d..237b0184424 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit ab0b711c68d27feabc2e82c840bd5225b185ef4a +Subproject commit 237b018442435f732911e9fc6dd1d4d5328968a5 From 3329b668d6f4f08aac17fb1ba904a73cf107ad63 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 10 Oct 2021 15:51:17 +0000 Subject: [PATCH 121/919] Fix style check --- src/Disks/ReadIndirectBufferFromWebServer.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Disks/ReadIndirectBufferFromWebServer.cpp b/src/Disks/ReadIndirectBufferFromWebServer.cpp index 57ee0614563..079c5c1d98a 100644 --- a/src/Disks/ReadIndirectBufferFromWebServer.cpp +++ b/src/Disks/ReadIndirectBufferFromWebServer.cpp @@ -17,7 +17,6 @@ namespace ErrorCodes { extern const int CANNOT_SEEK_THROUGH_FILE; extern const int SEEK_POSITION_OUT_OF_BOUND; - extern const int NETWORK_ERROR; } ReadIndirectBufferFromWebServer::ReadIndirectBufferFromWebServer( From 045704274cc1d7042a76608418d29cbc9c36929b Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 10 Oct 2021 16:31:32 +0000 Subject: [PATCH 122/919] implement directory watcher --- .gitmodules | 2 +- src/Common/ErrorCodes.cpp | 1 + src/Storages/FileLog/DirectoryWatcherBase.cpp | 134 ++++++++++++++++++ src/Storages/FileLog/DirectoryWatcherBase.h | 100 +++++++++++++ .../FileLog/FileLogDirectoryWatcher.cpp | 39 +++-- .../FileLog/FileLogDirectoryWatcher.h | 25 ++-- src/Storages/FileLog/StorageFileLog.cpp | 10 +- src/Storages/FileLog/StorageFileLog.h | 4 - 8 files changed, 270 insertions(+), 45 deletions(-) create mode 100644 src/Storages/FileLog/DirectoryWatcherBase.cpp create mode 100644 src/Storages/FileLog/DirectoryWatcherBase.h diff --git a/.gitmodules b/.gitmodules index 7f5c60d2b0c..74d1049ce01 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,6 +1,6 @@ [submodule "contrib/poco"] path = contrib/poco - url = https://github.com/ucasfl/poco.git + url = https://github.com/ClickHouse-Extras/poco.git branch = clickhouse [submodule "contrib/zstd"] path = contrib/zstd diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index ee456f81bcf..a8c3ca62e30 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -589,6 +589,7 @@ M(619, POSTGRESQL_REPLICATION_INTERNAL_ERROR) \ M(620, QUERY_NOT_ALLOWED) \ M(624, NOT_REGULAR_FILE) \ + M(625, IO_ERROR) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp new file mode 100644 index 00000000000..ed263767ad2 --- /dev/null +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -0,0 +1,134 @@ +#include +#include +#include + +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int FILE_DOESNT_EXIST; + extern const int DIRECTORY_DOESNT_EXIST; + extern const int IO_ERROR; +} + +DirectoryWatcherBase::DirectoryWatcherBase(FileLogDirectoryWatcher & owner_, const std::string & path_, int event_mask_) + : owner(owner_), path(path_), event_mask(event_mask_) +{ + if (!std::filesystem::exists(path)) + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "The path {} does not exist.", path); + + if (!std::filesystem::is_directory(path)) + throw Exception(ErrorCodes::DIRECTORY_DOESNT_EXIST, "The path {} does not a directory.", path); + + fd = inotify_init(); + if (fd == -1) + throw Exception("cannot initialize inotify", ErrorCodes::IO_ERROR); + watch_task = getContext()->getMessageBrokerSchedulePool().createTask("directory_watch", [this] { watchFunc(); }); + start(); +} + +void DirectoryWatcherBase::watchFunc() +{ + int mask = 0; + if (eventMask() & DirectoryWatcherBase::DW_ITEM_ADDED) + mask |= IN_CREATE; + if (eventMask() & DirectoryWatcherBase::DW_ITEM_REMOVED) + mask |= IN_DELETE; + if (eventMask() & DirectoryWatcherBase::DW_ITEM_MODIFIED) + mask |= IN_MODIFY; + if (eventMask() & DirectoryWatcherBase::DW_ITEM_MOVED_FROM) + mask |= IN_MOVED_FROM; + if (eventMask() & DirectoryWatcherBase::DW_ITEM_MOVED_TO) + mask |= IN_MOVED_TO; + + int wd = inotify_add_watch(fd, path.c_str(), mask); + if (wd == -1) + { + owner.onError(Exception(ErrorCodes::IO_ERROR, "Watch directory {} failed.", path)); + } + + std::string buffer; + buffer.resize(4096); + fd_set fds; + while (true) + { + FD_ZERO(&fds); + FD_SET(fd, &fds); + + struct timeval tv; + tv.tv_sec = 0; + tv.tv_usec = 200000; + + if (select(fd + 1, &fds, nullptr, nullptr, &tv) == 1) + { + int n = read(fd, buffer.data(), buffer.size()); + int i = 0; + if (n > 0) + { + while (n > 0) + { + struct inotify_event * p_event = reinterpret_cast(buffer.data() + i); + + if (p_event->len > 0) + { + if ((p_event->mask & IN_CREATE) && (eventMask() & DirectoryWatcherBase::DW_ITEM_ADDED)) + { + DirectoryWatcherBase::DirectoryEvent ev(p_event->name, DirectoryWatcherBase::DW_ITEM_ADDED); + owner.onItemAdded(ev); + } + if ((p_event->mask & IN_DELETE) && (eventMask() & DirectoryWatcherBase::DW_ITEM_REMOVED)) + { + DirectoryWatcherBase::DirectoryEvent ev(p_event->name, DirectoryWatcherBase::DW_ITEM_REMOVED); + owner.onItemRemoved(ev); + } + if ((p_event->mask & IN_MODIFY) && (eventMask() & DirectoryWatcherBase::DW_ITEM_MODIFIED)) + { + DirectoryWatcherBase::DirectoryEvent ev(p_event->name, DirectoryWatcherBase::DW_ITEM_MODIFIED); + owner.onItemModified(ev); + } + if ((p_event->mask & IN_MOVED_FROM) && (eventMask() & DirectoryWatcherBase::DW_ITEM_MOVED_FROM)) + { + DirectoryWatcherBase::DirectoryEvent ev(p_event->name, DirectoryWatcherBase::DW_ITEM_MOVED_FROM); + owner.onItemMovedFrom(ev); + } + if ((p_event->mask & IN_MOVED_TO) && (eventMask() & DirectoryWatcherBase::DW_ITEM_MOVED_TO)) + { + DirectoryWatcherBase::DirectoryEvent ev(p_event->name, DirectoryWatcherBase::DW_ITEM_MOVED_TO); + owner.onItemMovedTo(ev); + } + } + + i += sizeof(inotify_event) + p_event->len; + n -= sizeof(inotify_event) + p_event->len; + } + } + } + } +} + + +DirectoryWatcherBase::~DirectoryWatcherBase() +{ + stop(); + close(fd); +} + +void DirectoryWatcherBase::start() +{ + if (watch_task) + watch_task->activateAndSchedule(); +} + + +void DirectoryWatcherBase::stop() +{ + if (watch_task) + watch_task->deactivate(); +} + +} diff --git a/src/Storages/FileLog/DirectoryWatcherBase.h b/src/Storages/FileLog/DirectoryWatcherBase.h new file mode 100644 index 00000000000..53fb3862a41 --- /dev/null +++ b/src/Storages/FileLog/DirectoryWatcherBase.h @@ -0,0 +1,100 @@ +#pragma once + +#include + +#include +#include +#include + +namespace DB +{ +class FileLogDirectoryWatcher; + +class DirectoryWatcherBase : WithContext +{ + /// This class is used to get notifications about changes + /// to the filesystem, more specifically, to a specific + /// directory. Changes to a directory are reported via + /// events. + /// + /// A thread will be created that watches the specified + /// directory for changes. Events are reported in the context + /// of this thread. + /// + /// Note that changes to files in subdirectories of the watched + /// directory are not reported. Separate DirectoryWatcher objects + /// must be created for these directories if they should be watched. +public: + enum DirectoryEventType + { + DW_ITEM_ADDED = 1, + /// A new item has been created and added to the directory. + + DW_ITEM_REMOVED = 2, + /// An item has been removed from the directory. + + DW_ITEM_MODIFIED = 4, + /// An item has been modified. + + DW_ITEM_MOVED_FROM = 8, + /// An item has been renamed or moved. This event delivers the old name. + + DW_ITEM_MOVED_TO = 16 + /// An item has been renamed or moved. This event delivers the new name. + }; + + enum DirectoryEventMask + { + /// Enables all event types. + DW_FILTER_ENABLE_ALL = 31, + + /// Disables all event types. + DW_FILTER_DISABLE_ALL = 0 + }; + + struct DirectoryEvent + { + DirectoryEvent(const std::string & f, DirectoryEventType ev) : path(f), event(ev) { } + + /// The directory or file that has been changed. + const std::string & path; + /// The kind of event. + DirectoryEventType event; + }; + + + DirectoryWatcherBase() = delete; + DirectoryWatcherBase(const DirectoryWatcherBase &) = delete; + DirectoryWatcherBase & operator=(const DirectoryWatcherBase &) = delete; + + /// Creates a DirectoryWatcher for the directory given in path. + /// To enable only specific events, an eventMask can be specified by + /// OR-ing the desired event IDs (e.g., DW_ITEM_ADDED | DW_ITEM_MODIFIED). + explicit DirectoryWatcherBase(FileLogDirectoryWatcher & owner_, const std::string & path_, int event_mask_ = DW_FILTER_ENABLE_ALL); + + ~DirectoryWatcherBase(); + + /// Returns the value of the eventMask passed to the constructor. + int eventMask() const { return event_mask; } + + /// Returns the directory being watched. + const std::string & directory() const; + + [[noreturn]] void watchFunc(); + +protected: + void start(); + void stop(); + +private: + FileLogDirectoryWatcher & owner; + + using TaskThread = BackgroundSchedulePool::TaskHolder; + TaskThread watch_task; + + const std::string path; + int event_mask; + int fd; +}; + +} diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 3b4f89d50f5..b0fbf5d535b 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -1,18 +1,10 @@ #include -#include -#include -FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_) - : path(path_), dw(std::make_unique(path)), log(&Poco::Logger::get("DirectoryIterator (" + path + ")")) +namespace DB +{ +FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_) + : path(path_), dw(std::make_unique(*this, path)), log(&Poco::Logger::get("DirectoryIterator (" + path + ")")) { - /// DW_ITEM_MOVED_FROM and DW_ITEM_MOVED_TO events will only be reported on Linux. - /// On other platforms, a file rename or move operation will be reported via a - /// DW_ITEM_REMOVED and a DW_ITEM_ADDED event. The order of these two events is not defined. - dw->itemAdded += Poco::delegate(this, &FileLogDirectoryWatcher::onItemAdded); - dw->itemRemoved += Poco::delegate(this, &FileLogDirectoryWatcher::onItemRemoved); - dw->itemModified += Poco::delegate(this, &FileLogDirectoryWatcher::onItemModified); - dw->itemMovedFrom += Poco::delegate(this, &FileLogDirectoryWatcher::onItemMovedFrom); - dw->itemMovedTo += Poco::delegate(this, &FileLogDirectoryWatcher::onItemMovedTo); } FileLogDirectoryWatcher::Events FileLogDirectoryWatcher::getEventsAndReset() @@ -36,12 +28,12 @@ const std::string & FileLogDirectoryWatcher::getPath() const return path; } -void FileLogDirectoryWatcher::onItemAdded(const Poco::DirectoryWatcher::DirectoryEvent& ev) +void FileLogDirectoryWatcher::onItemAdded(const DirectoryWatcherBase::DirectoryEvent & ev) { std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemAdded"}; - std::string event_path = ev.item.path(); + std::string event_path = ev.path; if (auto it = events.find(event_path); it != events.end()) { @@ -54,12 +46,12 @@ void FileLogDirectoryWatcher::onItemAdded(const Poco::DirectoryWatcher::Director } -void FileLogDirectoryWatcher::onItemRemoved(const Poco::DirectoryWatcher::DirectoryEvent& ev) +void FileLogDirectoryWatcher::onItemRemoved(const DirectoryWatcherBase::DirectoryEvent & ev) { std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemRemoved"}; - std::string event_path = ev.item.path(); + std::string event_path = ev.path; if (auto it = events.find(event_path); it != events.end()) { @@ -77,11 +69,11 @@ void FileLogDirectoryWatcher::onItemRemoved(const Poco::DirectoryWatcher::Direct /// So, if we record all of these events, it will use a lot of memory, and then we /// need to handle it one by one in StorageFileLog::updateFileInfos, this is unnecessary /// because it is equal to just record and handle one MODIY event -void FileLogDirectoryWatcher::onItemModified(const Poco::DirectoryWatcher::DirectoryEvent& ev) +void FileLogDirectoryWatcher::onItemModified(const DirectoryWatcherBase::DirectoryEvent & ev) { std::lock_guard lock(mutex); - auto event_path = ev.item.path(); + auto event_path = ev.path; EventInfo info{ev.event, "onItemModified"}; /// Already have MODIFY event for this file if (auto it = events.find(event_path); it != events.end()) @@ -97,12 +89,12 @@ void FileLogDirectoryWatcher::onItemModified(const Poco::DirectoryWatcher::Direc } } -void FileLogDirectoryWatcher::onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent& ev) +void FileLogDirectoryWatcher::onItemMovedFrom(const DirectoryWatcherBase::DirectoryEvent & ev) { std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemMovedFrom"}; - std::string event_path = ev.item.path(); + std::string event_path = ev.path; if (auto it = events.find(event_path); it != events.end()) { @@ -114,12 +106,12 @@ void FileLogDirectoryWatcher::onItemMovedFrom(const Poco::DirectoryWatcher::Dire } } -void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent& ev) +void FileLogDirectoryWatcher::onItemMovedTo(const DirectoryWatcherBase::DirectoryEvent & ev) { std::lock_guard lock(mutex); EventInfo info{ev.event, "onItemMovedTo"}; - std::string event_path = ev.item.path(); + std::string event_path = ev.path; if (auto it = events.find(event_path); it != events.end()) { @@ -131,10 +123,11 @@ void FileLogDirectoryWatcher::onItemMovedTo(const Poco::DirectoryWatcher::Direct } } -void FileLogDirectoryWatcher::onError(const Poco::Exception & e) +void FileLogDirectoryWatcher::onError(const Exception & e) { std::lock_guard lock(mutex); LOG_ERROR(log, "Error happened during watching directory {}: {}", path, error.error_msg); error.has_error = true; error.error_msg = e.message(); } +} diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index 00ee7ddfc40..c7216fa4872 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -1,20 +1,21 @@ #pragma once -#include -#include -#include +#include #include #include #include +namespace DB +{ + class FileLogDirectoryWatcher { public: struct EventInfo { - Poco::DirectoryWatcher::DirectoryEventType type; + DirectoryWatcherBase::DirectoryEventType type; std::string callback; }; @@ -35,13 +36,12 @@ public: const std::string & getPath() const; -protected: - void onItemAdded(const Poco::DirectoryWatcher::DirectoryEvent& ev); - void onItemRemoved(const Poco::DirectoryWatcher::DirectoryEvent & ev); - void onItemModified(const Poco::DirectoryWatcher::DirectoryEvent& ev); - void onItemMovedFrom(const Poco::DirectoryWatcher::DirectoryEvent & ev); - void onItemMovedTo(const Poco::DirectoryWatcher::DirectoryEvent & ev); - void onError(const Poco::Exception &); + void onItemAdded(const DirectoryWatcherBase::DirectoryEvent & ev); + void onItemRemoved(const DirectoryWatcherBase::DirectoryEvent & ev); + void onItemModified(const DirectoryWatcherBase::DirectoryEvent & ev); + void onItemMovedFrom(const DirectoryWatcherBase::DirectoryEvent & ev); + void onItemMovedTo(const DirectoryWatcherBase::DirectoryEvent & ev); + void onError(const Exception &); private: const std::string path; @@ -53,7 +53,7 @@ private: /// running, it may access events during events destruction, leads to data race. Events events; - std::unique_ptr dw; + std::unique_ptr dw; Poco::Logger * log; @@ -61,3 +61,4 @@ private: Error error; }; +} diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index a75c9b0190e..18515304c0c 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -798,7 +798,7 @@ bool StorageFileLog::updateFileInfos() { switch (event_info.type) { - case Poco::DirectoryWatcher::DW_ITEM_ADDED: + case DirectoryWatcherBase::DW_ITEM_ADDED: { LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); /// Check if it is a regular file, and new file may be renamed or removed @@ -821,7 +821,7 @@ bool StorageFileLog::updateFileInfos() break; } - case Poco::DirectoryWatcher::DW_ITEM_MODIFIED: + case DirectoryWatcherBase::DW_ITEM_MODIFIED: { LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); /// When new file added and appended, it has two event: DW_ITEM_ADDED @@ -833,15 +833,15 @@ bool StorageFileLog::updateFileInfos() break; } - case Poco::DirectoryWatcher::DW_ITEM_REMOVED: - case Poco::DirectoryWatcher::DW_ITEM_MOVED_FROM: + case DirectoryWatcherBase::DW_ITEM_REMOVED: + case DirectoryWatcherBase::DW_ITEM_MOVED_FROM: { LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) it->second.status = FileStatus::REMOVED; break; } - case Poco::DirectoryWatcher::DW_ITEM_MOVED_TO: + case DirectoryWatcherBase::DW_ITEM_MOVED_TO: { LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index f3b85d47313..1b1d5f6671a 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -8,8 +8,6 @@ #include #include -#include -#include #include #include @@ -179,8 +177,6 @@ private: /// later select should forbid to execute. std::atomic running_streams = 0; - using TaskThread = BackgroundSchedulePool::TaskHolder; - void loadFiles(); void loadMetaFiles(bool attach); From e3d2942aa60b99ef7c81fb7bc88dc6cae6652340 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 10 Oct 2021 20:24:36 +0300 Subject: [PATCH 123/919] Correct merge --- src/Disks/DiskWebServer.cpp | 4 +- src/Disks/ReadBufferFromRemoteFSGather.cpp | 2 +- src/Disks/ReadBufferFromRemoteFSGather.h | 15 ++---- src/IO/ReadBufferFromWebServer.cpp | 19 +++++++- src/IO/ReadBufferFromWebServer.h | 4 +- src/IO/ReadWriteBufferFromHTTP.h | 55 +++++++++++++--------- 6 files changed, 61 insertions(+), 38 deletions(-) diff --git a/src/Disks/DiskWebServer.cpp b/src/Disks/DiskWebServer.cpp index f663229f689..4c98df561f1 100644 --- a/src/Disks/DiskWebServer.cpp +++ b/src/Disks/DiskWebServer.cpp @@ -168,9 +168,7 @@ std::unique_ptr DiskWebServer::readFile(const String & p bool threadpool_read = read_settings.remote_fs_method == RemoteFSReadMethod::read_threadpool; - auto web_impl = std::make_unique(url, meta, getContext(), - read_settings, - threadpool_read); + auto web_impl = std::make_unique(url, meta, getContext(), threadpool_read, read_settings); if (threadpool_read) { diff --git a/src/Disks/ReadBufferFromRemoteFSGather.cpp b/src/Disks/ReadBufferFromRemoteFSGather.cpp index c923e484ee0..5038698d1e7 100644 --- a/src/Disks/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/ReadBufferFromRemoteFSGather.cpp @@ -36,7 +36,7 @@ SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const S SeekableReadBufferPtr ReadBufferFromWebServerGather::createImplementationBuffer(const String & path) const { - return std::make_unique(fs::path(uri) / path, context, buf_size, backoff_threshold, max_tries, threadpool_read); + return std::make_unique(fs::path(uri) / path, context, settings, threadpool_read); } diff --git a/src/Disks/ReadBufferFromRemoteFSGather.h b/src/Disks/ReadBufferFromRemoteFSGather.h index 80bdfe4775d..28ea347352a 100644 --- a/src/Disks/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/ReadBufferFromRemoteFSGather.h @@ -6,6 +6,7 @@ #include #include +#include namespace Aws { @@ -92,17 +93,13 @@ public: const String & uri_, RemoteMetadata metadata_, ContextPtr context_, - size_t buf_size_, - size_t backoff_threshold_, - size_t max_tries_, - size_t threadpool_read_) + size_t threadpool_read_, + const ReadSettings & settings_) : ReadBufferFromRemoteFSGather(metadata_) , uri(uri_) , context(context_) - , buf_size(buf_size_) - , backoff_threshold(backoff_threshold_) - , max_tries(max_tries_) , threadpool_read(threadpool_read_) + , settings(settings_) { } @@ -111,10 +108,8 @@ public: private: String uri; ContextPtr context; - size_t buf_size; - size_t backoff_threshold; - size_t max_tries; bool threadpool_read; + ReadSettings settings; }; diff --git a/src/IO/ReadBufferFromWebServer.cpp b/src/IO/ReadBufferFromWebServer.cpp index 693a2842a08..9a7874997ef 100644 --- a/src/IO/ReadBufferFromWebServer.cpp +++ b/src/IO/ReadBufferFromWebServer.cpp @@ -20,6 +20,8 @@ namespace ErrorCodes } +static constexpr size_t HTTP_MAX_TRIES = 10; + ReadBufferFromWebServer::ReadBufferFromWebServer( const String & url_, ContextPtr context_, const ReadSettings & settings_, bool use_external_buffer_) : SeekableReadBuffer(nullptr, 0) @@ -93,7 +95,22 @@ bool ReadBufferFromWebServer::nextImpl() } else { - impl = initialize(); + /// Initialize impl with retry. + auto num_tries = std::max(read_settings.http_max_tries, HTTP_MAX_TRIES); + for (size_t i = 0; i < num_tries; ++i) + { + try + { + impl = initialize(); + } + catch (Poco::Exception &) + { + if (i == num_tries - 1) + throw; + + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } } auto result = impl->next(); diff --git a/src/IO/ReadBufferFromWebServer.h b/src/IO/ReadBufferFromWebServer.h index c7970f9f856..61e5ebfbb51 100644 --- a/src/IO/ReadBufferFromWebServer.h +++ b/src/IO/ReadBufferFromWebServer.h @@ -41,9 +41,9 @@ private: off_t offset = 0; - bool use_external_buffer; - ReadSettings read_settings; + + bool use_external_buffer; }; } diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 8f3fde11eaa..c61aebd040c 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -239,6 +239,16 @@ namespace detail try { impl = std::make_unique(*istr, buffer_size); + + if (use_external_buffer) + { + /** + * See comment 30 lines lower. + */ + impl->set(internal_buffer.begin(), internal_buffer.size()); + assert(working_buffer.begin() != nullptr); + assert(!internal_buffer.empty()); + } } catch (const Poco::Exception & e) { @@ -255,28 +265,31 @@ namespace detail if (next_callback) next_callback(count()); - if (use_external_buffer) + if (impl) { - /** - * use_external_buffer -- means we read into the buffer which - * was passed to us from somewhere else. We do not check whether - * previously returned buffer was read or not, because this branch - * means we are prefetching data, each nextImpl() call we can fill - * a different buffer. - */ - impl->set(internal_buffer.begin(), internal_buffer.size()); - assert(working_buffer.begin() != nullptr); - assert(!internal_buffer.empty()); - } - else - { - /** - * impl was initialized before, pass position() to it to make - * sure there is no pending data which was not read, becuase - * this branch means we read sequentially. - */ - if (!working_buffer.empty()) - impl->position() = position(); + if (use_external_buffer) + { + /** + * use_external_buffer -- means we read into the buffer which + * was passed to us from somewhere else. We do not check whether + * previously returned buffer was read or not, because this branch + * means we are prefetching data, each nextImpl() call we can fill + * a different buffer. + */ + impl->set(internal_buffer.begin(), internal_buffer.size()); + assert(working_buffer.begin() != nullptr); + assert(!internal_buffer.empty()); + } + else + { + /** + * impl was initialized before, pass position() to it to make + * sure there is no pending data which was not read, becuase + * this branch means we read sequentially. + */ + if (!working_buffer.empty()) + impl->position() = position(); + } } if (total_bytes_to_read && bytes_read == total_bytes_to_read.value()) From 7c570f2e48f6f7a360810c493f03597fd5c5e066 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 10 Oct 2021 22:13:49 +0300 Subject: [PATCH 124/919] Add retries for http read init --- src/IO/ReadBufferFromWebServer.cpp | 64 ++++++++++++++++++++++-------- src/IO/ReadBufferFromWebServer.h | 2 + 2 files changed, 49 insertions(+), 17 deletions(-) diff --git a/src/IO/ReadBufferFromWebServer.cpp b/src/IO/ReadBufferFromWebServer.cpp index 9a7874997ef..31b8f7da1c7 100644 --- a/src/IO/ReadBufferFromWebServer.cpp +++ b/src/IO/ReadBufferFromWebServer.cpp @@ -40,9 +40,12 @@ std::unique_ptr ReadBufferFromWebServer::initialize() Poco::URI uri(url); ReadWriteBufferFromHTTP::HTTPHeaderEntries headers; + + // read_settings.remote_read_max_bytes = 100000000; headers.emplace_back(std::make_pair("Range", fmt::format("bytes={}-", offset))); - const auto & settings = context->getSettingsRef(); LOG_DEBUG(log, "Reading from offset: {}", offset); + + const auto & settings = context->getSettingsRef(); const auto & config = context->getConfigRef(); Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 20), 0}; @@ -65,6 +68,48 @@ std::unique_ptr ReadBufferFromWebServer::initialize() } +void ReadBufferFromWebServer::initializeWithRetry() +{ + /// Initialize impl with retry. + auto num_tries = std::max(read_settings.http_max_tries, HTTP_MAX_TRIES); + size_t milliseconds_to_wait = read_settings.http_retry_initial_backoff_ms; + bool initialized = false; + for (size_t i = 0; (i < num_tries) && !initialized; ++i) + { + while (milliseconds_to_wait < read_settings.http_retry_max_backoff_ms) + { + try + { + impl = initialize(); + + if (use_external_buffer) + { + /** + * See comment at line 120. + */ + impl->set(internal_buffer.begin(), internal_buffer.size()); + assert(working_buffer.begin() != nullptr); + assert(!internal_buffer.empty()); + } + + initialized = true; + break; + } + catch (Poco::Exception & e) + { + if (i == num_tries - 1) + throw; + + LOG_ERROR(&Poco::Logger::get("ReadBufferFromWeb"), e.what()); + sleepForMilliseconds(milliseconds_to_wait); + milliseconds_to_wait *= 2; + } + } + milliseconds_to_wait = read_settings.http_retry_initial_backoff_ms; + } +} + + bool ReadBufferFromWebServer::nextImpl() { if (impl) @@ -95,22 +140,7 @@ bool ReadBufferFromWebServer::nextImpl() } else { - /// Initialize impl with retry. - auto num_tries = std::max(read_settings.http_max_tries, HTTP_MAX_TRIES); - for (size_t i = 0; i < num_tries; ++i) - { - try - { - impl = initialize(); - } - catch (Poco::Exception &) - { - if (i == num_tries - 1) - throw; - - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } + initializeWithRetry(); } auto result = impl->next(); diff --git a/src/IO/ReadBufferFromWebServer.h b/src/IO/ReadBufferFromWebServer.h index 61e5ebfbb51..780a4b16442 100644 --- a/src/IO/ReadBufferFromWebServer.h +++ b/src/IO/ReadBufferFromWebServer.h @@ -31,6 +31,8 @@ public: private: std::unique_ptr initialize(); + void initializeWithRetry(); + Poco::Logger * log; ContextPtr context; From 4c99f6da5198db26ca61fcabc471dba81061816b Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 10 Oct 2021 22:15:21 +0300 Subject: [PATCH 125/919] http allow to know total bytes to read --- src/IO/ReadSettings.h | 2 + src/IO/ReadWriteBufferFromHTTP.h | 83 ++++++++++++++++++++++++-------- 2 files changed, 64 insertions(+), 21 deletions(-) diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index edd5463bd7c..2ad73823285 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -77,6 +77,8 @@ struct ReadSettings size_t remote_fs_read_max_backoff_ms = 10000; size_t remote_fs_read_backoff_max_tries = 4; + size_t remote_read_max_bytes = 0; /// Zero means no upper bound. + bool http_retriable_read = true; size_t http_max_tries = 1; size_t http_retry_initial_backoff_ms = 100; diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index c61aebd040c..452dae9f88f 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -112,13 +112,13 @@ namespace detail ReadSettings settings; - std::istream * call(Poco::URI uri_, Poco::Net::HTTPResponse & response) + std::istream * call(Poco::URI uri_, Poco::Net::HTTPResponse & response, const std::string & method_) { // With empty path poco will send "POST HTTP/1.1" its bug. if (uri_.getPath().empty()) uri_.setPath("/"); - Poco::Net::HTTPRequest request(method, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + Poco::Net::HTTPRequest request(method_, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request.setHost(uri_.getHost()); // use original, not resolved host name in header if (out_stream_callback) @@ -162,6 +162,37 @@ namespace detail } } + std::optional getTotalSizeToRead() + { + if (total_bytes_to_read) + return total_bytes_to_read; + + Poco::Net::HTTPResponse response; + call(uri, response, Poco::Net::HTTPRequest::HTTP_HEAD); + + while (isRedirect(response.getStatus())) + { + Poco::URI uri_redirect(response.get("Location")); + remote_host_filter.checkURL(uri_redirect); + + session->updateSession(uri_redirect); + + istr = call(uri_redirect, response, method); + } + + /// If it is the very first initialization. + if (!bytes_read && !total_bytes_to_read) + { + /// If we do not know total size, disable retries in the middle of reading. + if (response.hasContentLength()) + total_bytes_to_read = response.getContentLength(); + else + settings.http_retriable_read = false; + } + + return total_bytes_to_read; + } + private: bool use_external_buffer; @@ -192,29 +223,39 @@ namespace detail , settings {settings_} , use_external_buffer {use_external_buffer_} { - /** - * Get first byte from `bytes=offset-`, `bytes=offset-end`. - * Now there are two places, where it can be set: 1. in DiskWeb (offset), 2. via config as part of named-collection. - * Also header can be `bytes=-k` (read last k bytes), for this case retries in the middle of reading are disabled. - */ - auto range_header = std::find_if(http_header_entries_.begin(), http_header_entries_.end(), - [&](const HTTPHeaderEntry & header) { return std::get<0>(header) == "Range"; }); - if (range_header != http_header_entries_.end()) - { - auto range = std::get<1>(*range_header).substr(std::strlen("bytes=")); - auto [ptr, ec] = std::from_chars(range.data(), range.data() + range.size(), start_byte); - if (ec != std::errc()) - settings.http_retriable_read = false; - } - initialize(); } void initialize() { + /** + * Get first byte from `bytes=offset-`, `bytes=offset-end`. + * Now there are two places, where it can be set: 1. in DiskWeb (offset), 2. via config as part of named-collection. + * Also header can be `bytes=-k` (read last k bytes), for this case retries in the middle of reading are disabled. + */ + auto range_header = std::find_if(http_header_entries.begin(), http_header_entries.end(), + [&](const HTTPHeaderEntry & header) { return std::get<0>(header) == "Range"; }); + if (range_header != http_header_entries.end()) + { + auto & value = std::get<1>(*range_header); + auto range = value.substr(std::strlen("bytes=")); + auto [ptr, ec] = std::from_chars(range.data(), range.data() + range.size(), start_byte); + if (ec != std::errc()) + settings.http_retriable_read = false; + + if (settings.remote_read_max_bytes) + { + auto file_size = total_bytes_to_read ? total_bytes_to_read.value() : getTotalSizeToRead(); + if (file_size) + { + value = fmt::format("Range={}-{}", start_byte, std::min(settings.remote_read_max_bytes, *file_size)); + std::cerr << "\n\nRange header: " << value << std::endl; + } + } + } Poco::Net::HTTPResponse response; - istr = call(uri, response); + istr = call(uri, response, method); while (isRedirect(response.getStatus())) { @@ -223,7 +264,7 @@ namespace detail session->updateSession(uri_redirect); - istr = call(uri_redirect, response); + istr = call(uri_redirect, response, method); } /// If it is the very first initialization. @@ -316,13 +357,13 @@ namespace detail successful_read = true; break; } - catch (const Poco::Exception &) + catch (const Poco::Exception & e) { if (i == settings.http_max_tries - 1 || (bytes_read && !settings.http_retriable_read)) throw; - tryLogCurrentException(__PRETTY_FUNCTION__); + LOG_ERROR(&Poco::Logger::get("ReadBufferFromHTTP"), e.what()); impl.reset(); sleepForMilliseconds(milliseconds_to_wait); From eb5cb8627118564b4f2464a64f53c204a48f2ce5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 10 Oct 2021 23:20:00 +0300 Subject: [PATCH 126/919] More profile events --- src/Common/ProfileEvents.cpp | 11 +++++++++ ...chronousReadIndirectBufferFromRemoteFS.cpp | 23 +++++++++++++++++++ ...ynchronousReadIndirectBufferFromRemoteFS.h | 2 ++ src/IO/ReadIndirectBufferFromRemoteFS.cpp | 8 +++++++ 4 files changed, 44 insertions(+) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index cba194fe8d6..41ae41b38d4 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -252,6 +252,17 @@ M(RemoteFSReadMicroseconds, "Time of reading from remote filesystem.") \ M(RemoteFSReadBytes, "Read bytes from remote filesystem.") \ \ + M(RemoteFSSeeks, "Total number of seeks for async buffer") \ + M(RemoteFSPrefetches, "Total number of prefetches") \ + M(RemoteFSSeekCancelledPrefetches, "Number of cancelled prefecthes because of seek") \ + M(RemoteFSUnusedCancelledPrefetches, "Number of prefetches prending in buffer desctructor") \ + M(RemoteFSPrefetchReads, "Total number of reads from prefecthed buffer") \ + M(RemoteFSAsyncBufferReads, "Number of nextImpl() calls for async buffer") \ + M(RemoteFSSimpleBufferReads, "Number of nextImpl() calls for non-async buffer") \ + M(RemoteFSNewReaders, "Number of created impl objects") \ + M(RemoteFSAsyncBuffers, "Total number of AsycnhronousReadIndirectBufferFromREmoteFS buffers") \ + M(RemoteFSSimpleBuffers, "Total number of ReadIndirectBufferFromREmoteFS buffers") \ + \ M(SleepFunctionCalls, "Number of times a sleep function (sleep, sleepEachRow) has been called.") \ M(SleepFunctionMicroseconds, "Time spent sleeping due to a sleep function call.") \ \ diff --git a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp index f25707b70a0..0327223dbb9 100644 --- a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -8,9 +8,17 @@ namespace CurrentMetrics { extern const Metric AsynchronousReadWait; } + namespace ProfileEvents { extern const Event AsynchronousReadWaitMicroseconds; + extern const Event RemoteFSSeeks; + extern const Event RemoteFSPrefetches; + extern const Event RemoteFSSeekCancelledPrefetches; + extern const Event RemoteFSUnusedCancelledPrefetches; + extern const Event RemoteFSPrefetchReads; + extern const Event RemoteFSAsyncBufferReads; + extern const Event RemoteFSAsyncBuffers; } namespace DB @@ -52,15 +60,21 @@ void AsynchronousReadIndirectBufferFromRemoteFS::prefetch() return; prefetch_future = readInto(prefetch_buffer.data(), prefetch_buffer.size()); + ProfileEvents::increment(ProfileEvents::RemoteFSPrefetches); + buffer_events += "-- Prefetch --"; } bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() { + ProfileEvents::increment(ProfileEvents::RemoteFSAsyncBufferReads); size_t size = 0; if (prefetch_future.valid()) { + ProfileEvents::increment(ProfileEvents::RemoteFSPrefetchReads); + buffer_events += "-- Read from prefetch --"; + CurrentMetrics::Increment metric_increment{CurrentMetrics::AsynchronousReadWait}; Stopwatch watch; { @@ -78,6 +92,7 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() } else { + buffer_events += "-- Read without prefetch --"; size = readInto(memory.data(), memory.size()).get(); if (size) { @@ -87,6 +102,7 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() } } + buffer_events += " + " + toString(size) + " + "; prefetch_future = {}; return size; } @@ -94,6 +110,9 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence) { + ProfileEvents::increment(ProfileEvents::RemoteFSSeeks); + buffer_events += "-- Seek to " + toString(offset_) + " --"; + if (whence == SEEK_CUR) { /// If position within current working buffer - shift pos. @@ -131,6 +150,7 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence if (prefetch_future.valid()) { + ProfileEvents::increment(ProfileEvents::RemoteFSSeekCancelledPrefetches); prefetch_future.wait(); prefetch_future = {}; } @@ -144,8 +164,11 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence void AsynchronousReadIndirectBufferFromRemoteFS::finalize() { + std::cerr << "\n\n\nBuffer events: " << buffer_events << std::endl; + if (prefetch_future.valid()) { + ProfileEvents::increment(ProfileEvents::RemoteFSUnusedCancelledPrefetches); prefetch_future.wait(); prefetch_future = {}; } diff --git a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h index 3b5bcc55a66..ed94f72fd69 100644 --- a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -64,6 +64,8 @@ private: size_t absolute_position = 0; Memory<> prefetch_buffer; + + String buffer_events; }; } diff --git a/src/IO/ReadIndirectBufferFromRemoteFS.cpp b/src/IO/ReadIndirectBufferFromRemoteFS.cpp index 25c4795b8a5..d70d280871a 100644 --- a/src/IO/ReadIndirectBufferFromRemoteFS.cpp +++ b/src/IO/ReadIndirectBufferFromRemoteFS.cpp @@ -3,6 +3,12 @@ #include +namespace ProfileEvents +{ + extern const Event RemoteFSSimpleBufferReads; + extern const Event RemoteFSSimpleBuffers; +} + namespace DB { @@ -15,6 +21,7 @@ namespace ErrorCodes ReadIndirectBufferFromRemoteFS::ReadIndirectBufferFromRemoteFS( std::shared_ptr impl_) : impl(std::move(impl_)) { + ProfileEvents::increment(ProfileEvents::RemoteFSSimpleBuffers); } @@ -72,6 +79,7 @@ off_t ReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence) bool ReadIndirectBufferFromRemoteFS::nextImpl() { + ProfileEvents::increment(ProfileEvents::RemoteFSSimpleBufferReads); /// Transfer current position and working_buffer to actual ReadBuffer swap(*impl); /// Position and working_buffer will be updated in next() call From aa4b79780845a78388571c241645de5ab194d881 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 10 Oct 2021 23:22:58 +0300 Subject: [PATCH 127/919] Turn off tasks stealing for remote disk --- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 ++ src/Storages/MergeTree/MergeTreeDataPartCompact.cpp | 5 +++++ src/Storages/MergeTree/MergeTreeDataPartCompact.h | 2 ++ src/Storages/MergeTree/MergeTreeDataPartInMemory.h | 1 + src/Storages/MergeTree/MergeTreeDataPartWide.cpp | 5 +++++ src/Storages/MergeTree/MergeTreeDataPartWide.h | 2 ++ src/Storages/MergeTree/MergeTreeReadPool.cpp | 4 ++++ 7 files changed, 21 insertions(+) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index be48aed5c8b..f02b7e98f02 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -95,6 +95,8 @@ public: virtual bool isStoredOnDisk() const = 0; + virtual bool isStoredOnRemoteDisk() const = 0; + virtual bool supportsVerticalMerge() const { return false; } /// NOTE: Returns zeros if column files are not found in checksums. diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index ad1c2abeee7..c4c2e65547b 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -182,6 +182,11 @@ void MergeTreeDataPartCompact::checkConsistency(bool require_part_metadata) cons } } +bool MergeTreeDataPartCompact::isStoredOnRemoteDisk() const +{ + return volume->getDisk()->isRemote(); +} + MergeTreeDataPartCompact::~MergeTreeDataPartCompact() { removeIfNeeded(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 2fcc7b7034b..38bfa11652a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -56,6 +56,8 @@ public: bool isStoredOnDisk() const override { return true; } + bool isStoredOnRemoteDisk() const override; + bool hasColumnFiles(const NameAndTypePair & column) const override; String getFileNameForColumn(const NameAndTypePair & /* column */) const override { return DATA_FILE_NAME; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index 118340f0233..d1a0344859d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -44,6 +44,7 @@ public: const MergeTreeIndexGranularity & computed_index_granularity) const override; bool isStoredOnDisk() const override { return false; } + bool isStoredOnRemoteDisk() const override { return false; } bool hasColumnFiles(const NameAndTypePair & column) const override { return !!getColumnPosition(column.name); } String getFileNameForColumn(const NameAndTypePair & /* column */) const override { return ""; } void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 11e080fda6c..6ac4919329d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -142,6 +142,11 @@ void MergeTreeDataPartWide::loadIndexGranularity() index_granularity.setInitialized(); } +bool MergeTreeDataPartWide::isStoredOnRemoteDisk() const +{ + return volume->getDisk()->isRemote(); +} + MergeTreeDataPartWide::~MergeTreeDataPartWide() { removeIfNeeded(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index a43396f8cd5..4796143e11d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -50,6 +50,8 @@ public: bool isStoredOnDisk() const override { return true; } + bool isStoredOnRemoteDisk() const override; + bool supportsVerticalMerge() const override { return true; } String getFileNameForColumn(const NameAndTypePair & column) const override; diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index d08cec24184..f23c58bc2b3 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -221,6 +221,10 @@ std::vector MergeTreeReadPool::fillPerPartInfo( { const auto & part = parts[i]; + /// Turn off tasks stealing in case there is remote disk. + if (part.data_part->isStoredOnRemoteDisk()) + do_not_steal_tasks = true; + /// Read marks for every data part. size_t sum_marks = 0; for (const auto & range : part.ranges) From 6044725baa2ffa525a4e723fa9aa05347bee5835 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 10 Oct 2021 23:37:10 +0300 Subject: [PATCH 128/919] Revert "http allow to know total bytes to read" This reverts commit 4c99f6da5198db26ca61fcabc471dba81061816b. --- src/IO/ReadSettings.h | 2 - src/IO/ReadWriteBufferFromHTTP.h | 71 +++++++------------------------- 2 files changed, 15 insertions(+), 58 deletions(-) diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index 2ad73823285..edd5463bd7c 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -77,8 +77,6 @@ struct ReadSettings size_t remote_fs_read_max_backoff_ms = 10000; size_t remote_fs_read_backoff_max_tries = 4; - size_t remote_read_max_bytes = 0; /// Zero means no upper bound. - bool http_retriable_read = true; size_t http_max_tries = 1; size_t http_retry_initial_backoff_ms = 100; diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 452dae9f88f..c61aebd040c 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -112,13 +112,13 @@ namespace detail ReadSettings settings; - std::istream * call(Poco::URI uri_, Poco::Net::HTTPResponse & response, const std::string & method_) + std::istream * call(Poco::URI uri_, Poco::Net::HTTPResponse & response) { // With empty path poco will send "POST HTTP/1.1" its bug. if (uri_.getPath().empty()) uri_.setPath("/"); - Poco::Net::HTTPRequest request(method_, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + Poco::Net::HTTPRequest request(method, uri_.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request.setHost(uri_.getHost()); // use original, not resolved host name in header if (out_stream_callback) @@ -162,37 +162,6 @@ namespace detail } } - std::optional getTotalSizeToRead() - { - if (total_bytes_to_read) - return total_bytes_to_read; - - Poco::Net::HTTPResponse response; - call(uri, response, Poco::Net::HTTPRequest::HTTP_HEAD); - - while (isRedirect(response.getStatus())) - { - Poco::URI uri_redirect(response.get("Location")); - remote_host_filter.checkURL(uri_redirect); - - session->updateSession(uri_redirect); - - istr = call(uri_redirect, response, method); - } - - /// If it is the very first initialization. - if (!bytes_read && !total_bytes_to_read) - { - /// If we do not know total size, disable retries in the middle of reading. - if (response.hasContentLength()) - total_bytes_to_read = response.getContentLength(); - else - settings.http_retriable_read = false; - } - - return total_bytes_to_read; - } - private: bool use_external_buffer; @@ -222,40 +191,30 @@ namespace detail , buffer_size {buffer_size_} , settings {settings_} , use_external_buffer {use_external_buffer_} - { - initialize(); - } - - void initialize() { /** * Get first byte from `bytes=offset-`, `bytes=offset-end`. * Now there are two places, where it can be set: 1. in DiskWeb (offset), 2. via config as part of named-collection. * Also header can be `bytes=-k` (read last k bytes), for this case retries in the middle of reading are disabled. */ - auto range_header = std::find_if(http_header_entries.begin(), http_header_entries.end(), + auto range_header = std::find_if(http_header_entries_.begin(), http_header_entries_.end(), [&](const HTTPHeaderEntry & header) { return std::get<0>(header) == "Range"; }); - if (range_header != http_header_entries.end()) + if (range_header != http_header_entries_.end()) { - auto & value = std::get<1>(*range_header); - auto range = value.substr(std::strlen("bytes=")); + auto range = std::get<1>(*range_header).substr(std::strlen("bytes=")); auto [ptr, ec] = std::from_chars(range.data(), range.data() + range.size(), start_byte); if (ec != std::errc()) settings.http_retriable_read = false; - - if (settings.remote_read_max_bytes) - { - auto file_size = total_bytes_to_read ? total_bytes_to_read.value() : getTotalSizeToRead(); - if (file_size) - { - value = fmt::format("Range={}-{}", start_byte, std::min(settings.remote_read_max_bytes, *file_size)); - std::cerr << "\n\nRange header: " << value << std::endl; - } - } } + initialize(); + } + + void initialize() + { + Poco::Net::HTTPResponse response; - istr = call(uri, response, method); + istr = call(uri, response); while (isRedirect(response.getStatus())) { @@ -264,7 +223,7 @@ namespace detail session->updateSession(uri_redirect); - istr = call(uri_redirect, response, method); + istr = call(uri_redirect, response); } /// If it is the very first initialization. @@ -357,13 +316,13 @@ namespace detail successful_read = true; break; } - catch (const Poco::Exception & e) + catch (const Poco::Exception &) { if (i == settings.http_max_tries - 1 || (bytes_read && !settings.http_retriable_read)) throw; - LOG_ERROR(&Poco::Logger::get("ReadBufferFromHTTP"), e.what()); + tryLogCurrentException(__PRETTY_FUNCTION__); impl.reset(); sleepForMilliseconds(milliseconds_to_wait); From 1f8b449bc512505ed2ba11e0680dc2516472b31a Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Oct 2021 00:51:43 +0300 Subject: [PATCH 129/919] Lazy seek avoiding --- src/Disks/DiskWebServer.cpp | 3 +- src/Disks/ReadBufferFromRemoteFSGather.cpp | 47 ++++++++++++++++--- src/Disks/ReadBufferFromRemoteFSGather.h | 14 ++++-- ...chronousReadIndirectBufferFromRemoteFS.cpp | 31 ++++++++++-- ...ynchronousReadIndirectBufferFromRemoteFS.h | 6 ++- src/IO/AsynchronousReader.h | 1 + src/IO/ThreadPoolRemoteFSReader.cpp | 6 +-- src/IO/ThreadPoolRemoteFSReader.h | 2 +- 8 files changed, 88 insertions(+), 22 deletions(-) diff --git a/src/Disks/DiskWebServer.cpp b/src/Disks/DiskWebServer.cpp index 4c98df561f1..d6fae0aa7dc 100644 --- a/src/Disks/DiskWebServer.cpp +++ b/src/Disks/DiskWebServer.cpp @@ -173,8 +173,7 @@ std::unique_ptr DiskWebServer::readFile(const String & p if (threadpool_read) { auto reader = IDiskRemote::getThreadPoolReader(); - auto buf = std::make_unique(reader, read_settings.priority, std::move(web_impl)); - return std::make_unique(std::move(buf), min_bytes_for_seek); + return std::make_unique(reader, read_settings.priority, std::move(web_impl)); } else { diff --git a/src/Disks/ReadBufferFromRemoteFSGather.cpp b/src/Disks/ReadBufferFromRemoteFSGather.cpp index 5038698d1e7..4a886ff37a7 100644 --- a/src/Disks/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/ReadBufferFromRemoteFSGather.cpp @@ -55,18 +55,28 @@ ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather(const RemoteMetadata } -size_t ReadBufferFromRemoteFSGather::readInto(char * data, size_t size, size_t offset) +size_t ReadBufferFromRemoteFSGather::readInto(char * data, size_t size, size_t offset, size_t ignore) { + /** + * Set `data` to current working and internal buffers. + * Internal buffer with size `size`. Working buffer with size 0. + */ set(data, size); + absolute_position = offset; + bytes_to_ignore = ignore; + auto result = nextImpl(); + bytes_to_ignore = 0; + if (result) return working_buffer.size(); + return 0; } -SeekableReadBufferPtr ReadBufferFromRemoteFSGather::initialize() +void ReadBufferFromRemoteFSGather::initialize() { /// One clickhouse file can be split into multiple files in remote fs. auto current_buf_offset = absolute_position; @@ -77,14 +87,20 @@ SeekableReadBufferPtr ReadBufferFromRemoteFSGather::initialize() if (size > current_buf_offset) { - auto buf = createImplementationBuffer(file_path); - buf->seek(current_buf_offset, SEEK_SET); - return buf; + /// Do not create a new buffer if we already have what we need. + if (!current_buf || buf_idx != i) + { + current_buf = createImplementationBuffer(file_path); + buf_idx = i; + } + + current_buf->seek(current_buf_offset, SEEK_SET); + return; } current_buf_offset -= size; } - return nullptr; + current_buf = nullptr; } @@ -92,7 +108,7 @@ bool ReadBufferFromRemoteFSGather::nextImpl() { /// Find first available buffer that fits to given offset. if (!current_buf) - current_buf = initialize(); + initialize(); /// If current buffer has remaining data - use it. if (current_buf) @@ -119,7 +135,17 @@ bool ReadBufferFromRemoteFSGather::nextImpl() bool ReadBufferFromRemoteFSGather::readImpl() { swap(*current_buf); + + /** + * Lazy seek is performed here. + * In asynchronous buffer when seeking to offset in range [pos, pos + min_bytes_for_seek] + * we save how many bytes need to be ignored (new_offset - position() bytes). + */ + if (bytes_to_ignore) + current_buf->ignore(bytes_to_ignore); + auto result = current_buf->next(); + swap(*current_buf); if (result) @@ -129,6 +155,13 @@ bool ReadBufferFromRemoteFSGather::readImpl() } +void ReadBufferFromRemoteFSGather::seek(off_t offset) +{ + absolute_position = offset; + initialize(); +} + + void ReadBufferFromRemoteFSGather::reset() { current_buf.reset(); diff --git a/src/Disks/ReadBufferFromRemoteFSGather.h b/src/Disks/ReadBufferFromRemoteFSGather.h index 28ea347352a..7a858b0e126 100644 --- a/src/Disks/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/ReadBufferFromRemoteFSGather.h @@ -13,9 +13,7 @@ namespace Aws namespace S3 { class S3Client; -} -} - +}} namespace DB { @@ -31,8 +29,10 @@ public: void reset(); + void seek(off_t offset); /// SEEK_SET only. + protected: - size_t readInto(char * data, size_t size, size_t offset); + size_t readInto(char * data, size_t size, size_t offset, size_t ignore = 0); virtual SeekableReadBufferPtr createImplementationBuffer(const String & path) const = 0; @@ -41,7 +41,7 @@ protected: private: bool nextImpl() override; - SeekableReadBufferPtr initialize(); + void initialize(); bool readImpl(); @@ -50,6 +50,10 @@ private: size_t current_buf_idx = 0; size_t absolute_position = 0; + + size_t buf_idx = 0; + + size_t bytes_to_ignore = 0; }; diff --git a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp index 0327223dbb9..c69eb893663 100644 --- a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -31,13 +31,17 @@ namespace ErrorCodes AsynchronousReadIndirectBufferFromRemoteFS::AsynchronousReadIndirectBufferFromRemoteFS( - AsynchronousReaderPtr reader_, Int32 priority_, - std::shared_ptr impl_, size_t buf_size_) + AsynchronousReaderPtr reader_, + Int32 priority_, + std::shared_ptr impl_, + size_t buf_size_, + size_t min_bytes_for_seek_) : ReadBufferFromFileBase(buf_size_, nullptr, 0) , reader(reader_) , priority(priority_) , impl(impl_) , prefetch_buffer(buf_size_) + , min_bytes_for_seek(min_bytes_for_seek_) { } @@ -50,12 +54,21 @@ std::future AsynchronousReadIndirectBufferFromRemot request.size = size; request.offset = absolute_position; request.priority = priority; + + if (bytes_to_ignore) + { + request.ignore = bytes_to_ignore; + bytes_to_ignore = 0; + } return reader->submit(request); } void AsynchronousReadIndirectBufferFromRemoteFS::prefetch() { + if (hasPendingData()) + return; + if (prefetch_future.valid()) return; @@ -156,7 +169,19 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence } pos = working_buffer.end(); - impl->reset(); + + if (static_cast(absolute_position) >= getPosition() + && static_cast(absolute_position) < getPosition() + static_cast(min_bytes_for_seek)) + { + /** + * Lazy ignore. Save number of bytes to ignore and ignore it either for prefetch buffer or current buffer. + */ + bytes_to_ignore = absolute_position - getPosition(); + } + else + { + impl->seek(absolute_position); /// SEEK_SET. + } return absolute_position; } diff --git a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h index ed94f72fd69..3c600562252 100644 --- a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -34,7 +34,8 @@ public: explicit AsynchronousReadIndirectBufferFromRemoteFS( AsynchronousReaderPtr reader_, Int32 priority_, std::shared_ptr impl_, - size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE); + size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE, + size_t min_bytes_for_seek = 1024 * 1024); ~AsynchronousReadIndirectBufferFromRemoteFS() override; @@ -66,6 +67,9 @@ private: Memory<> prefetch_buffer; String buffer_events; + + size_t min_bytes_for_seek; + size_t bytes_to_ignore = 0; }; } diff --git a/src/IO/AsynchronousReader.h b/src/IO/AsynchronousReader.h index 77b4a2f5b22..e4a81623205 100644 --- a/src/IO/AsynchronousReader.h +++ b/src/IO/AsynchronousReader.h @@ -46,6 +46,7 @@ public: size_t size = 0; char * buf = nullptr; int64_t priority = 0; + size_t ignore = 0; }; /// Less than requested amount of data can be returned. diff --git a/src/IO/ThreadPoolRemoteFSReader.cpp b/src/IO/ThreadPoolRemoteFSReader.cpp index ce750f1898a..258d20d62e9 100644 --- a/src/IO/ThreadPoolRemoteFSReader.cpp +++ b/src/IO/ThreadPoolRemoteFSReader.cpp @@ -28,9 +28,9 @@ namespace CurrentMetrics namespace DB { -size_t ThreadPoolRemoteFSReader::RemoteFSFileDescriptor::readInto(char * data, size_t size, size_t offset) +size_t ThreadPoolRemoteFSReader::RemoteFSFileDescriptor::readInto(char * data, size_t size, size_t offset, size_t ignore) { - return reader->readInto(data, size, offset); + return reader->readInto(data, size, offset, ignore); } @@ -49,7 +49,7 @@ std::future ThreadPoolRemoteFSReader::submit(Reques auto * remote_fs_fd = assert_cast(request.descriptor.get()); Stopwatch watch(CLOCK_MONOTONIC); - auto bytes_read = remote_fs_fd->readInto(request.buf, request.size, request.offset); + auto bytes_read = remote_fs_fd->readInto(request.buf, request.size, request.offset, request.ignore); watch.stop(); ProfileEvents::increment(ProfileEvents::RemoteFSReadMicroseconds, watch.elapsedMicroseconds()); diff --git a/src/IO/ThreadPoolRemoteFSReader.h b/src/IO/ThreadPoolRemoteFSReader.h index a2d9cbbf779..c300162e214 100644 --- a/src/IO/ThreadPoolRemoteFSReader.h +++ b/src/IO/ThreadPoolRemoteFSReader.h @@ -30,7 +30,7 @@ struct ThreadPoolRemoteFSReader::RemoteFSFileDescriptor : public IFileDescriptor public: RemoteFSFileDescriptor(std::shared_ptr reader_) : reader(reader_) {} - size_t readInto(char * data, size_t size, size_t offset); + size_t readInto(char * data, size_t size, size_t offset, size_t ignore = 0); private: std::shared_ptr reader; From 448d63a0fce5985aa14a493d02dc9eccc98e57d4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Oct 2021 01:53:51 +0300 Subject: [PATCH 130/919] Range read with upper bound offset --- src/Disks/IDiskRemote.cpp | 2 +- src/Disks/ReadBufferFromRemoteFSGather.cpp | 3 ++- src/Disks/ReadBufferFromRemoteFSGather.h | 6 +++--- src/Disks/S3/DiskS3.cpp | 10 +++++---- src/Disks/S3/DiskS3.h | 3 +++ src/Disks/S3/registerDiskS3.cpp | 1 + src/IO/ReadBufferFromS3.cpp | 21 +++++++++++++------ src/IO/ReadBufferFromS3.h | 17 ++++++++------- src/IO/ReadBufferFromWebServer.cpp | 16 ++++++++++---- src/IO/ReadSettings.h | 2 ++ src/IO/ReadWriteBufferFromHTTP.h | 4 ++-- .../MergeTree/MergeTreeReaderStream.cpp | 9 ++++++++ src/Storages/StorageS3.cpp | 2 +- 13 files changed, 67 insertions(+), 29 deletions(-) diff --git a/src/Disks/IDiskRemote.cpp b/src/Disks/IDiskRemote.cpp index b368e485f75..428de44259c 100644 --- a/src/Disks/IDiskRemote.cpp +++ b/src/Disks/IDiskRemote.cpp @@ -500,7 +500,7 @@ String IDiskRemote::getUniqueId(const String & path) const AsynchronousReaderPtr IDiskRemote::getThreadPoolReader() { - constexpr size_t pool_size = 16; + constexpr size_t pool_size = 50; constexpr size_t queue_size = 1000000; static AsynchronousReaderPtr reader = std::make_shared(pool_size, queue_size); return reader; diff --git a/src/Disks/ReadBufferFromRemoteFSGather.cpp b/src/Disks/ReadBufferFromRemoteFSGather.cpp index 4a886ff37a7..5ffb8b9f589 100644 --- a/src/Disks/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/ReadBufferFromRemoteFSGather.cpp @@ -29,7 +29,8 @@ namespace ErrorCodes #if USE_AWS_S3 SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const String & path) const { - return std::make_unique(client_ptr, bucket, fs::path(metadata.remote_fs_root_path) / path, max_single_read_retries, buf_size, threadpool_read); + return std::make_unique(client_ptr, bucket, + fs::path(metadata.remote_fs_root_path) / path, max_single_read_retries, settings, threadpool_read); } #endif diff --git a/src/Disks/ReadBufferFromRemoteFSGather.h b/src/Disks/ReadBufferFromRemoteFSGather.h index 7a858b0e126..efd3d7a2483 100644 --- a/src/Disks/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/ReadBufferFromRemoteFSGather.h @@ -67,13 +67,13 @@ public: const String & bucket_, IDiskRemote::Metadata metadata_, size_t max_single_read_retries_, - size_t buf_size_, + const ReadSettings & settings_, bool threadpool_read_ = false) : ReadBufferFromRemoteFSGather(metadata_) , client_ptr(std::move(client_ptr_)) , bucket(bucket_) , max_single_read_retries(max_single_read_retries_) - , buf_size(buf_size_) + , settings(settings_) , threadpool_read(threadpool_read_) { } @@ -84,7 +84,7 @@ private: std::shared_ptr client_ptr; String bucket; UInt64 max_single_read_retries; - size_t buf_size; + ReadSettings settings; bool threadpool_read; }; #endif diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 73269ef6695..13c9899fecc 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -132,12 +132,14 @@ DiskS3::DiskS3( String bucket_, String s3_root_path_, String metadata_path_, + ContextPtr context_, SettingsPtr settings_, GetDiskSettings settings_getter_) : IDiskRemote(name_, s3_root_path_, metadata_path_, "DiskS3", settings_->thread_pool_size) , bucket(std::move(bucket_)) , current_settings(std::move(settings_)) , settings_getter(settings_getter_) + , context(context_) { } @@ -204,7 +206,7 @@ std::unique_ptr DiskS3::readFile(const String & path, co auto s3_impl = std::make_unique( settings->client, bucket, metadata, - settings->s3_max_single_read_retries, read_settings.remote_fs_buffer_size, threadpool_read); + settings->s3_max_single_read_retries, read_settings, threadpool_read); if (threadpool_read) { @@ -362,7 +364,7 @@ int DiskS3::readSchemaVersion(const String & source_bucket, const String & sourc source_bucket, source_path + SCHEMA_VERSION_OBJECT, settings->s3_max_single_read_retries, - DBMS_DEFAULT_BUFFER_SIZE); + context->getReadSettings()); readIntText(version, buffer); @@ -1017,9 +1019,9 @@ void DiskS3::onFreeze(const String & path) revision_file_buf.finalize(); } -void DiskS3::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String &, const DisksMap &) +void DiskS3::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context_, const String &, const DisksMap &) { - auto new_settings = settings_getter(config, "storage_configuration.disks." + name, context); + auto new_settings = settings_getter(config, "storage_configuration.disks." + name, context_); current_settings.set(std::move(new_settings)); diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 3ef1db43742..000ae03468c 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -71,6 +71,7 @@ public: String bucket_, String s3_root_path_, String metadata_path_, + ContextPtr context_, SettingsPtr settings_, GetDiskSettings settings_getter_); @@ -177,6 +178,8 @@ private: static constexpr int RESTORABLE_SCHEMA_VERSION = 1; /// Directories with data. const std::vector data_roots {"data", "store"}; + + ContextPtr context; }; } diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index 0946c8875d3..5eabbfff5f8 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -186,6 +186,7 @@ void registerDiskS3(DiskFactory & factory) uri.bucket, uri.key, metadata_path, + context, getSettings(config, config_prefix, context), getSettings); diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index e21e4581c58..bf974440be2 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -32,13 +32,13 @@ namespace ErrorCodes ReadBufferFromS3::ReadBufferFromS3( std::shared_ptr client_ptr_, const String & bucket_, const String & key_, - UInt64 max_single_read_retries_, size_t buffer_size_, bool use_external_buffer_) + UInt64 max_single_read_retries_, const ReadSettings & settings_, bool use_external_buffer_) : SeekableReadBuffer(nullptr, 0) , client_ptr(std::move(client_ptr_)) , bucket(bucket_) , key(key_) , max_single_read_retries(max_single_read_retries_) - , buffer_size(buffer_size_) + , read_settings(settings_) , use_external_buffer(use_external_buffer_) { } @@ -158,19 +158,28 @@ off_t ReadBufferFromS3::getPosition() std::unique_ptr ReadBufferFromS3::initialize() { - LOG_TRACE(log, "Read S3 object. Bucket: {}, Key: {}, Offset: {}", bucket, key, offset); - Aws::S3::Model::GetObjectRequest req; req.SetBucket(bucket); req.SetKey(key); - req.SetRange(fmt::format("bytes={}-", offset)); + + auto right_offset = read_settings.remote_read_right_offset; + if (right_offset) + { + req.SetRange(fmt::format("bytes={}-{}", offset, right_offset)); + LOG_TRACE(log, "Read S3 object. Bucket: {}, Key: {}, Range: {}-{}", bucket, key, offset, right_offset); + } + else + { + req.SetRange(fmt::format("bytes={}-", offset)); + LOG_TRACE(log, "Read S3 object. Bucket: {}, Key: {}, Offset: {}", bucket, key, offset); + } Aws::S3::Model::GetObjectOutcome outcome = client_ptr->GetObject(req); if (outcome.IsSuccess()) { read_result = outcome.GetResultWithOwnership(); - return std::make_unique(read_result.GetBody(), buffer_size); + return std::make_unique(read_result.GetBody(), read_settings.remote_fs_buffer_size); } else throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 0c353d5916f..b27de8aa0b4 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -6,12 +6,14 @@ #if USE_AWS_S3 -# include +#include -# include -# include -# include -# include "SeekableReadBuffer.h" +#include +#include +#include +#include + +#include namespace Aws::S3 { @@ -30,7 +32,6 @@ private: String bucket; String key; UInt64 max_single_read_retries; - size_t buffer_size; off_t offset = 0; Aws::S3::Model::GetObjectResult read_result; std::unique_ptr impl; @@ -43,7 +44,7 @@ public: const String & bucket_, const String & key_, UInt64 max_single_read_retries_, - size_t buffer_size_, + const ReadSettings & settings_, bool use_external_buffer = false); bool nextImpl() override; @@ -53,6 +54,8 @@ public: private: std::unique_ptr initialize(); + + ReadSettings read_settings; bool use_external_buffer; }; diff --git a/src/IO/ReadBufferFromWebServer.cpp b/src/IO/ReadBufferFromWebServer.cpp index 31b8f7da1c7..bfbca078248 100644 --- a/src/IO/ReadBufferFromWebServer.cpp +++ b/src/IO/ReadBufferFromWebServer.cpp @@ -41,9 +41,17 @@ std::unique_ptr ReadBufferFromWebServer::initialize() ReadWriteBufferFromHTTP::HTTPHeaderEntries headers; - // read_settings.remote_read_max_bytes = 100000000; - headers.emplace_back(std::make_pair("Range", fmt::format("bytes={}-", offset))); - LOG_DEBUG(log, "Reading from offset: {}", offset); + auto right_offset = read_settings.remote_read_right_offset; + if (right_offset) + { + headers.emplace_back(std::make_pair("Range", fmt::format("bytes={}-{}", offset, right_offset))); + LOG_DEBUG(log, "Reading with range: {}-{}", offset, right_offset); + } + else + { + headers.emplace_back(std::make_pair("Range", fmt::format("bytes={}-", offset))); + LOG_DEBUG(log, "Reading from offset: {}", offset); + } const auto & settings = context->getSettingsRef(); const auto & config = context->getConfigRef(); @@ -85,7 +93,7 @@ void ReadBufferFromWebServer::initializeWithRetry() if (use_external_buffer) { /** - * See comment at line 120. + * See comment 30 lines lower. */ impl->set(internal_buffer.begin(), internal_buffer.size()); assert(working_buffer.begin() != nullptr); diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index edd5463bd7c..0548659e16e 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -77,6 +77,8 @@ struct ReadSettings size_t remote_fs_read_max_backoff_ms = 10000; size_t remote_fs_read_backoff_max_tries = 4; + size_t remote_read_right_offset = 0; /// Right offset for range reading. + bool http_retriable_read = true; size_t http_max_tries = 1; size_t http_retry_initial_backoff_ms = 100; diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index c61aebd040c..f28fcb60497 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -316,13 +316,13 @@ namespace detail successful_read = true; break; } - catch (const Poco::Exception &) + catch (const Poco::Exception & e) { if (i == settings.http_max_tries - 1 || (bytes_read && !settings.http_retriable_read)) throw; - tryLogCurrentException(__PRETTY_FUNCTION__); + LOG_ERROR(&Poco::Logger::get("ReadBufferFromHTTP"), e.what()); impl.reset(); sleepForMilliseconds(milliseconds_to_wait); diff --git a/src/Storages/MergeTree/MergeTreeReaderStream.cpp b/src/Storages/MergeTree/MergeTreeReaderStream.cpp index fc57b48e86d..04b1411d939 100644 --- a/src/Storages/MergeTree/MergeTreeReaderStream.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderStream.cpp @@ -31,6 +31,8 @@ MergeTreeReaderStream::MergeTreeReaderStream( /// Compute the size of the buffer. size_t max_mark_range_bytes = 0; size_t sum_mark_range_bytes = 0; + /// Rightmost bound to read. + size_t right_bound = 0; for (const auto & mark_range : all_mark_ranges) { @@ -53,6 +55,7 @@ MergeTreeReaderStream::MergeTreeReaderStream( } size_t mark_range_bytes; + size_t current_right_offset; /// If there are no marks after the end of range, just use file size if (right_mark >= marks_count @@ -60,14 +63,17 @@ MergeTreeReaderStream::MergeTreeReaderStream( && marks_loader.getMark(right_mark).offset_in_compressed_file == marks_loader.getMark(mark_range.end).offset_in_compressed_file)) { mark_range_bytes = file_size - (left_mark < marks_count ? marks_loader.getMark(left_mark).offset_in_compressed_file : 0); + current_right_offset = file_size; } else { mark_range_bytes = marks_loader.getMark(right_mark).offset_in_compressed_file - marks_loader.getMark(left_mark).offset_in_compressed_file; + current_right_offset = marks_loader.getMark(right_mark).offset_in_compressed_file; } max_mark_range_bytes = std::max(max_mark_range_bytes, mark_range_bytes); sum_mark_range_bytes += mark_range_bytes; + right_bound = std::max(right_bound, current_right_offset); } /// Avoid empty buffer. May happen while reading dictionary for DataTypeLowCardinality. @@ -76,6 +82,9 @@ MergeTreeReaderStream::MergeTreeReaderStream( if (max_mark_range_bytes != 0) read_settings = read_settings.adjustBufferSize(max_mark_range_bytes); + /// Set bound for reading from remote disk. + read_settings.remote_read_right_offset = right_bound; + /// Initialize the objects that shall be used to perform read operations. if (uncompressed_cache) { diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index ce1b049758e..d1f621c83de 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -230,7 +230,7 @@ bool StorageS3Source::initialize() file_path = fs::path(bucket) / current_key; read_buf = wrapReadBufferWithCompressionMethod( - std::make_unique(client, bucket, current_key, max_single_read_retries, DBMS_DEFAULT_BUFFER_SIZE), + std::make_unique(client, bucket, current_key, max_single_read_retries, getContext()->getReadSettings()), chooseCompressionMethod(current_key, compression_hint)); auto input_format = FormatFactory::instance().getInput(format, *read_buf, sample_block, getContext(), max_block_size, format_settings); QueryPipelineBuilder builder; From 4d9604e29882743638a5cd2b5f373e887634aa20 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 11 Oct 2021 04:06:31 +0000 Subject: [PATCH 131/919] fix --- src/Common/ErrorCodes.cpp | 2 +- src/Storages/FileLog/DirectoryWatcherBase.cpp | 20 +++++++++++------ src/Storages/FileLog/DirectoryWatcherBase.h | 9 +++++--- .../FileLog/FileLogDirectoryWatcher.cpp | 6 +++-- .../FileLog/FileLogDirectoryWatcher.h | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 22 +++++++++---------- 6 files changed, 36 insertions(+), 25 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index a8c3ca62e30..6f33636d064 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -589,7 +589,7 @@ M(619, POSTGRESQL_REPLICATION_INTERNAL_ERROR) \ M(620, QUERY_NOT_ALLOWED) \ M(624, NOT_REGULAR_FILE) \ - M(625, IO_ERROR) \ + M(625, IO_SETUP_ERROR) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index ed263767ad2..350e71aadfd 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -13,11 +13,15 @@ namespace ErrorCodes { extern const int FILE_DOESNT_EXIST; extern const int DIRECTORY_DOESNT_EXIST; - extern const int IO_ERROR; + extern const int IO_SETUP_ERROR; } -DirectoryWatcherBase::DirectoryWatcherBase(FileLogDirectoryWatcher & owner_, const std::string & path_, int event_mask_) - : owner(owner_), path(path_), event_mask(event_mask_) +static constexpr int event_size = sizeof(struct inotify_event); +static constexpr int buffer_size = 1024 * (NAME_MAX + event_size + 1); + +DirectoryWatcherBase::DirectoryWatcherBase( + FileLogDirectoryWatcher & owner_, const std::string & path_, ContextPtr context_, int event_mask_) + : WithContext(context_->getGlobalContext()), owner(owner_), path(path_), event_mask(event_mask_) { if (!std::filesystem::exists(path)) throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "The path {} does not exist.", path); @@ -27,7 +31,8 @@ DirectoryWatcherBase::DirectoryWatcherBase(FileLogDirectoryWatcher & owner_, con fd = inotify_init(); if (fd == -1) - throw Exception("cannot initialize inotify", ErrorCodes::IO_ERROR); + throw Exception("cannot initialize inotify", ErrorCodes::IO_SETUP_ERROR); + watch_task = getContext()->getMessageBrokerSchedulePool().createTask("directory_watch", [this] { watchFunc(); }); start(); } @@ -49,13 +54,13 @@ void DirectoryWatcherBase::watchFunc() int wd = inotify_add_watch(fd, path.c_str(), mask); if (wd == -1) { - owner.onError(Exception(ErrorCodes::IO_ERROR, "Watch directory {} failed.", path)); + owner.onError(Exception(ErrorCodes::IO_SETUP_ERROR, "Watch directory {} failed.", path)); } std::string buffer; - buffer.resize(4096); + buffer.resize(buffer_size); fd_set fds; - while (true) + while (!stopped) { FD_ZERO(&fds); FD_SET(fd, &fds); @@ -127,6 +132,7 @@ void DirectoryWatcherBase::start() void DirectoryWatcherBase::stop() { + stopped = true; if (watch_task) watch_task->deactivate(); } diff --git a/src/Storages/FileLog/DirectoryWatcherBase.h b/src/Storages/FileLog/DirectoryWatcherBase.h index 53fb3862a41..c9530453e2e 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.h +++ b/src/Storages/FileLog/DirectoryWatcherBase.h @@ -53,7 +53,7 @@ public: }; struct DirectoryEvent - { + { DirectoryEvent(const std::string & f, DirectoryEventType ev) : path(f), event(ev) { } /// The directory or file that has been changed. @@ -70,7 +70,8 @@ public: /// Creates a DirectoryWatcher for the directory given in path. /// To enable only specific events, an eventMask can be specified by /// OR-ing the desired event IDs (e.g., DW_ITEM_ADDED | DW_ITEM_MODIFIED). - explicit DirectoryWatcherBase(FileLogDirectoryWatcher & owner_, const std::string & path_, int event_mask_ = DW_FILTER_ENABLE_ALL); + explicit DirectoryWatcherBase( + FileLogDirectoryWatcher & owner_, const std::string & path_, ContextPtr context_, int event_mask_ = DW_FILTER_ENABLE_ALL); ~DirectoryWatcherBase(); @@ -80,7 +81,7 @@ public: /// Returns the directory being watched. const std::string & directory() const; - [[noreturn]] void watchFunc(); + void watchFunc(); protected: void start(); @@ -92,6 +93,8 @@ private: using TaskThread = BackgroundSchedulePool::TaskHolder; TaskThread watch_task; + std::atomic stopped{false}; + const std::string path; int event_mask; int fd; diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index b0fbf5d535b..93aa3d0cf9f 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -2,8 +2,10 @@ namespace DB { -FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_) - : path(path_), dw(std::make_unique(*this, path)), log(&Poco::Logger::get("DirectoryIterator (" + path + ")")) +FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_, ContextPtr context_) + : path(path_) + , dw(std::make_unique(*this, path, context_)) + , log(&Poco::Logger::get("DirectoryIterator (" + path + ")")) { } diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index c7216fa4872..b050d9a2c39 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -27,7 +27,7 @@ public: std::string error_msg = {}; }; - explicit FileLogDirectoryWatcher(const std::string & path_); + explicit FileLogDirectoryWatcher(const std::string & path_, ContextPtr context_); ~FileLogDirectoryWatcher() = default; Events getEventsAndReset(); diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 18515304c0c..00824c08690 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -86,7 +86,7 @@ StorageFileLog::StorageFileLog( #endif if (path_is_directory) - directory_watch = std::make_unique(path); + directory_watch = std::make_unique(path, context_); auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); task = std::make_shared(std::move(thread)); @@ -791,20 +791,20 @@ bool StorageFileLog::updateFileInfos() auto events = directory_watch->getEventsAndReset(); - for (const auto & [event_path, event_infos] : events) + for (const auto & [file_name, event_infos] : events) { - String file_name = std::filesystem::path(event_path).filename(); + String file_path = getFullDataPath(file_name); for (const auto & event_info : event_infos) { switch (event_info.type) { case DirectoryWatcherBase::DW_ITEM_ADDED: { - LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); + LOG_TRACE(log, "New event {} watched, file_name: {}", event_info.callback, file_name); /// Check if it is a regular file, and new file may be renamed or removed - if (std::filesystem::is_regular_file(event_path)) + if (std::filesystem::is_regular_file(file_path)) { - auto inode = getInode(event_path); + auto inode = getInode(file_path); file_infos.file_names.push_back(file_name); @@ -823,7 +823,7 @@ bool StorageFileLog::updateFileInfos() case DirectoryWatcherBase::DW_ITEM_MODIFIED: { - LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); + LOG_TRACE(log, "New event {} watched, file_name: {}", event_info.callback, file_name); /// When new file added and appended, it has two event: DW_ITEM_ADDED /// and DW_ITEM_MODIFIED, since the order of these two events in the /// sequence is uncentain, so we may can not find it in file_infos, just @@ -836,21 +836,21 @@ bool StorageFileLog::updateFileInfos() case DirectoryWatcherBase::DW_ITEM_REMOVED: case DirectoryWatcherBase::DW_ITEM_MOVED_FROM: { - LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); + LOG_TRACE(log, "New event {} watched, file_name: {}", event_info.callback, file_name); if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) it->second.status = FileStatus::REMOVED; break; } case DirectoryWatcherBase::DW_ITEM_MOVED_TO: { - LOG_TRACE(log, "New event {} watched, path: {}", event_info.callback, event_path); + LOG_TRACE(log, "New event {} watched, file_name: {}", event_info.callback, file_name); /// Similar to DW_ITEM_ADDED, but if it removed from an old file /// should obtain old meta file and rename meta file - if (std::filesystem::is_regular_file(event_path)) + if (std::filesystem::is_regular_file(file_path)) { file_infos.file_names.push_back(file_name); - auto inode = getInode(event_path); + auto inode = getInode(file_path); if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) it->second = FileContext{.inode = inode}; From e5857618d8c6901e7d3b8a5e7865241ce05e11c6 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 11 Oct 2021 04:12:04 +0000 Subject: [PATCH 132/919] update poco --- contrib/poco | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index 237b0184424..39fd359765a 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 237b018442435f732911e9fc6dd1d4d5328968a5 +Subproject commit 39fd359765a3a77b46d94ec3c5def3c7802a920f From d8bd1474bc2db7ded8c0c7ac560c0a51f36c6520 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 11 Oct 2021 04:18:21 +0000 Subject: [PATCH 133/919] fix --- src/Storages/FileLog/FileLogDirectoryWatcher.cpp | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 93aa3d0cf9f..bd4a7543366 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -5,7 +5,7 @@ namespace DB FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_, ContextPtr context_) : path(path_) , dw(std::make_unique(*this, path, context_)) - , log(&Poco::Logger::get("DirectoryIterator (" + path + ")")) + , log(&Poco::Logger::get("FileLogDirectoryWatcher(" + path + ")")) { } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 00824c08690..c8ffd44369e 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -86,7 +86,7 @@ StorageFileLog::StorageFileLog( #endif if (path_is_directory) - directory_watch = std::make_unique(path, context_); + directory_watch = std::make_unique(root_data_path, context_); auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); task = std::make_shared(std::move(thread)); From 31d98b0992c042bb43238e16f07ba9db3bfe2f3d Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Oct 2021 08:17:22 +0300 Subject: [PATCH 134/919] Remove redundant --- src/Disks/S3/DiskS3.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 13c9899fecc..c22bc32c84b 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -211,8 +211,7 @@ std::unique_ptr DiskS3::readFile(const String & path, co if (threadpool_read) { auto reader = getThreadPoolReader(); - auto buf = std::make_unique(reader, read_settings.priority, std::move(s3_impl)); - return std::make_unique(std::move(buf), settings->min_bytes_for_seek); + return std::make_unique(reader, read_settings.priority, std::move(s3_impl)); } else { From 6d93253a67f519eb289e7b6728da7ec2f81be4dc Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Oct 2021 11:34:08 +0000 Subject: [PATCH 135/919] Fix tests --- src/Storages/ExternalDataSourceConfiguration.cpp | 2 +- src/Storages/StorageMongoDB.cpp | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index c0a663bcb6f..8433885df18 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -111,7 +111,7 @@ std::optional getExternalDataSourceConfiguration(const } auto arg_value_ast = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context); - auto arg_value_literal = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context)->as(); + auto * arg_value_literal = evaluateConstantExpressionOrIdentifierAsLiteral(function_args[1], context)->as(); if (arg_value_literal) { auto arg_value = arg_value_literal->value; diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 02568b62a62..47856645398 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #include #include From a3d629a5b541ef2d0489b9b7e7e710ed3c7a0410 Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Mon, 11 Oct 2021 22:51:13 +0530 Subject: [PATCH 136/919] add x86 feature avx2/avx512 support for filter implementation --- cmake/cpu_features.cmake | 32 ++++++++++++ src/Columns/ColumnFixedString.cpp | 69 ++++++++++++++++++++++++- src/Columns/ColumnVector.cpp | 63 +++++++++++++++++++++- src/Columns/ColumnsCommon.cpp | 86 ++++++++++++++++++++++++++++++- 4 files changed, 246 insertions(+), 4 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index 46e42329958..d77ca0b32e3 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -18,6 +18,8 @@ option (ENABLE_PCLMULQDQ "Use pclmulqdq instructions on x86_64" 1) option (ENABLE_POPCNT "Use popcnt instructions on x86_64" 1) option (ENABLE_AVX "Use AVX instructions on x86_64" 0) option (ENABLE_AVX2 "Use AVX2 instructions on x86_64" 0) +option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 1) +option (ENABLE_BMI "Use BMI instructions on x86_64" 1) option (ARCH_NATIVE "Add -march=native compiler flag. This makes your binaries non-portable but more performant code may be generated. This option overrides ENABLE_* options for specific instruction set. Highly not recommended to use." 0) @@ -127,6 +129,36 @@ else () if (HAVE_AVX2 AND ENABLE_AVX2) set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () + + set (TEST_FLAG "-mavx512f -mavx512bw") + set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") + check_cxx_source_compiles(" + #include + int main() { + auto a = _mm512_setzero_epi32(); + (void)a; + auto b = _mm512_add_epi16(__m512i(), __m512i()); + (void)b; + return 0; + } + " HAVE_AVX512) + if (HAVE_AVX512 AND ENABLE_AVX512) + set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + endif () + + set (TEST_FLAG "-mbmi") + set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") + check_cxx_source_compiles(" + #include + int main() { + auto a = _blsr_u32(0); + (void)a; + return 0; + } + " HAVE_BMI) + if (HAVE_BMI AND ENABLE_BMI) + set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + endif () endif () cmake_pop_check_state () diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 94127fa8eb3..9daec1c1c64 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -230,8 +230,74 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result const UInt8 * filt_pos = filt.data(); const UInt8 * filt_end = filt_pos + col_size; const UInt8 * data_pos = chars.data(); +#if defined(__AVX512F__) && defined(__AVX512BW__) + static constexpr size_t SIMD_BYTES = 64; + const __m512i zero64 = _mm512_setzero_epi32(); + const UInt8 * filt_end_avx512 = filt_pos + col_size / SIMD_BYTES * SIMD_BYTES; + const size_t chars_per_simd_elements = SIMD_BYTES * n; -#ifdef __SSE2__ + while (filt_pos < filt_end_avx512) + { + uint64_t mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero64, _MM_CMPINT_GT); + + if (0xFFFFFFFFFFFFFFFF == mask) + { + res->chars.insert(data_pos, data_pos + chars_per_simd_elements); + } + else + { + size_t res_chars_size = res->chars.size(); + while (mask) + { + size_t index = __builtin_ctzll(mask); + res->chars.resize(res_chars_size + n); + memcpySmallAllowReadWriteOverflow15(&res->chars[res_chars_size], data_pos + index * n, n); + res_chars_size += n; + #ifdef __BMI__ + mask = _blsr_u64(mask); + #else + mask = mask & (mask-1); + #endif + } + } + data_pos += chars_per_simd_elements; + filt_pos += SIMD_BYTES; + } +#elif defined(__AVX2__) + static constexpr size_t SIMD_BYTES = 32; + const __m256i zero32 = _mm256_setzero_si256(); + const UInt8 * filt_end_avx2 = filt_pos + col_size / SIMD_BYTES * SIMD_BYTES; + const size_t chars_per_simd_elements = SIMD_BYTES * n; + + while (filt_pos < filt_end_avx2) + { + uint32_t mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero32)); + + if (0xFFFFFFFF == mask) + { + res->chars.insert(data_pos, data_pos + chars_per_simd_elements); + } + else + { + size_t res_chars_size = res->chars.size(); + while (mask) + { + size_t index = __builtin_ctz(mask); + res->chars.resize(res_chars_size + n); + memcpySmallAllowReadWriteOverflow15(&res->chars[res_chars_size], data_pos + index * n, n); + res_chars_size += n; + #ifdef __BMI__ + mask = _blsr_u32(mask); + #else + mask = mask & (mask-1); + #endif + } + } + data_pos += chars_per_simd_elements; + filt_pos += SIMD_BYTES; + } + +#elif defined(__SSE2__) /** A slightly more optimized version. * Based on the assumption that often pieces of consecutive values * completely pass or do not pass the filter. @@ -267,6 +333,7 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result data_pos += chars_per_simd_elements; filt_pos += SIMD_BYTES; } + #endif size_t res_chars_size = res->chars.size(); diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 3ee692a3ff4..000a7198446 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -311,7 +311,67 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s const UInt8 * filt_end = filt_pos + size; const T * data_pos = data.data(); -#ifdef __SSE2__ +#if defined(__AVX512F__) && defined(__AVX512BW__) + static constexpr size_t SIMD_BYTES = 64; + const __m512i zero64 = _mm512_setzero_epi32(); + const UInt8 * filt_end_avx512 = filt_pos + size / SIMD_BYTES * SIMD_BYTES; + + while (filt_pos < filt_end_avx512) + { + UInt64 mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero64, _MM_CMPINT_GT); + + if (0xFFFFFFFFFFFFFFFF == mask) + { + res_data.insert(data_pos, data_pos + SIMD_BYTES); + } + else + { + while (mask) + { + size_t index = __builtin_ctzll(mask); + res_data.push_back(data_pos[index]); + #ifdef __BMI__ + mask = _blsr_u64(mask); + #else + mask = mask & (mask-1); + #endif + } + } + + filt_pos += SIMD_BYTES; + data_pos += SIMD_BYTES; + } +#elif defined(__AVX2__) + static constexpr size_t SIMD_BYTES = 32; + const __m256i zero32 = _mm256_setzero_si256(); + const UInt8 * filt_end_avx2 = filt_pos + size / SIMD_BYTES * SIMD_BYTES; + + while (filt_pos < filt_end_avx2) + { + UInt32 mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero32)); + + if (0xFFFFFFFF == mask) + { + res_data.insert(data_pos, data_pos + SIMD_BYTES); + } + else + { + while (mask) + { + size_t index = __builtin_ctz(mask); + res_data.push_back(data_pos[index]); + #ifdef __BMI__ + mask = _blsr_u32(mask); + #else + mask = mask & (mask-1); + #endif + } + } + + filt_pos += SIMD_BYTES; + data_pos += SIMD_BYTES; + } +#elif defined(__SSE2__) /** A slightly more optimized version. * Based on the assumption that often pieces of consecutive values * completely pass or do not pass the filter. @@ -344,6 +404,7 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s filt_pos += SIMD_BYTES; data_pos += SIMD_BYTES; } + #endif while (filt_pos < filt_end) diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index a4d7de34382..36c292b4196 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -229,7 +229,89 @@ namespace memcpy(&res_elems[elems_size_old], &src_elems[arr_offset], arr_size * sizeof(T)); }; - #ifdef __SSE2__ + #if defined(__AVX512F__) && defined(__AVX512BW__) + const __m512i zero_vec = _mm512_setzero_epi32(); + static constexpr size_t SIMD_BYTES = 64; + const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; + + while (filt_pos < filt_end_aligned) + { + uint64_t mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero_vec, _MM_CMPINT_GT); + + if (mask == 0xffffffffffffffff) + { + /// SIMD_BYTES consecutive rows pass the filter + const auto first = offsets_pos == offsets_begin; + + const auto chunk_offset = first ? 0 : offsets_pos[-1]; + const auto chunk_size = offsets_pos[SIMD_BYTES - 1] - chunk_offset; + + result_offsets_builder.template insertChunk(offsets_pos, first, chunk_offset, chunk_size); + + /// copy elements for SIMD_BYTES arrays at once + const auto elems_size_old = res_elems.size(); + res_elems.resize(elems_size_old + chunk_size); + memcpy(&res_elems[elems_size_old], &src_elems[chunk_offset], chunk_size * sizeof(T)); + } + else + { + while (mask) + { + size_t index = __builtin_ctzll(mask); + copy_array(offsets_pos + index); + #ifdef __BMI__ + mask = _blsr_u64(mask); + #else + mask = mask & (mask-1); + #endif + } + } + + filt_pos += SIMD_BYTES; + offsets_pos += SIMD_BYTES; + } + #elif defined(__AVX2__) + const __m256i zero_vec = _mm256_setzero_si256(); + static constexpr size_t SIMD_BYTES = 32; + const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; + + while (filt_pos < filt_end_aligned) + { + uint32_t mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero_vec)); + + if (mask == 0xffffffff) + { + /// SIMD_BYTES consecutive rows pass the filter + const auto first = offsets_pos == offsets_begin; + + const auto chunk_offset = first ? 0 : offsets_pos[-1]; + const auto chunk_size = offsets_pos[SIMD_BYTES - 1] - chunk_offset; + + result_offsets_builder.template insertChunk(offsets_pos, first, chunk_offset, chunk_size); + + /// copy elements for SIMD_BYTES arrays at once + const auto elems_size_old = res_elems.size(); + res_elems.resize(elems_size_old + chunk_size); + memcpy(&res_elems[elems_size_old], &src_elems[chunk_offset], chunk_size * sizeof(T)); + } + else + { + while (mask) + { + size_t index = __builtin_ctz(mask); + copy_array(offsets_pos + index); + #ifdef __BMI__ + mask = _blsr_u32(mask); + #else + mask = mask & (mask-1); + #endif + } + } + + filt_pos += SIMD_BYTES; + offsets_pos += SIMD_BYTES; + } + #elif defined(__SSE2__) const __m128i zero_vec = _mm_setzero_si128(); static constexpr size_t SIMD_BYTES = 16; const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; @@ -268,7 +350,7 @@ namespace filt_pos += SIMD_BYTES; offsets_pos += SIMD_BYTES; - } + } #endif while (filt_pos < filt_end) From 0ef26244083333ef4ee8ede0049014334a573459 Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Tue, 12 Oct 2021 04:04:27 +0530 Subject: [PATCH 137/919] add specific flags for compiling filter operation source files --- cmake/cpu_features.cmake | 4 ++-- src/CMakeLists.txt | 7 +++++++ src/Columns/ColumnFixedString.cpp | 11 ++++++----- src/Columns/ColumnVector.cpp | 15 ++++++++------- 4 files changed, 23 insertions(+), 14 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index d77ca0b32e3..4ea9465be98 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -143,7 +143,7 @@ else () } " HAVE_AVX512) if (HAVE_AVX512 AND ENABLE_AVX512) - set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + set(X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} ${TEST_FLAG}") endif () set (TEST_FLAG "-mbmi") @@ -157,7 +157,7 @@ else () } " HAVE_BMI) if (HAVE_BMI AND ENABLE_BMI) - set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + set(X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} ${TEST_FLAG}") endif () endif () diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index cac5b70f489..45bb1a21d59 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -284,6 +284,13 @@ target_link_libraries (clickhouse_common_io dragonbox_to_chars ) +# Use X86 AVX2/AVX512 instructions to accelerate filter opertions +set_source_files_properties( + Columns/ColumnFixedString.cpp + Columns/ColumnsCommon.cpp + Columns/ColumnVector.cpp + PROPERTIES COMPILE_FLAGS "${X86_INTRINSICS_FLAGS}") + if(RE2_LIBRARY) target_link_libraries(clickhouse_common_io PUBLIC ${RE2_LIBRARY}) endif() diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 9daec1c1c64..4b31677d37e 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -230,6 +230,12 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result const UInt8 * filt_pos = filt.data(); const UInt8 * filt_end = filt_pos + col_size; const UInt8 * data_pos = chars.data(); + + /** A slightly more optimized version. + * Based on the assumption that often pieces of consecutive values + * completely pass or do not pass the filter. + * Therefore, we will optimistically check the parts of `SIMD_BYTES` values. + */ #if defined(__AVX512F__) && defined(__AVX512BW__) static constexpr size_t SIMD_BYTES = 64; const __m512i zero64 = _mm512_setzero_epi32(); @@ -298,11 +304,6 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result } #elif defined(__SSE2__) - /** A slightly more optimized version. - * Based on the assumption that often pieces of consecutive values - * completely pass or do not pass the filter. - * Therefore, we will optimistically check the parts of `SIMD_BYTES` values. - */ static constexpr size_t SIMD_BYTES = 16; const __m128i zero16 = _mm_setzero_si128(); diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 000a7198446..37f60e9f2b9 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -310,7 +310,12 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s const UInt8 * filt_pos = filt.data(); const UInt8 * filt_end = filt_pos + size; const T * data_pos = data.data(); - + + /** A slightly more optimized version. + * Based on the assumption that often pieces of consecutive values + * completely pass or do not pass the filter. + * Therefore, we will optimistically check the parts of `SIMD_BYTES` values. + */ #if defined(__AVX512F__) && defined(__AVX512BW__) static constexpr size_t SIMD_BYTES = 64; const __m512i zero64 = _mm512_setzero_epi32(); @@ -341,6 +346,7 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s filt_pos += SIMD_BYTES; data_pos += SIMD_BYTES; } + #elif defined(__AVX2__) static constexpr size_t SIMD_BYTES = 32; const __m256i zero32 = _mm256_setzero_si256(); @@ -371,13 +377,8 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s filt_pos += SIMD_BYTES; data_pos += SIMD_BYTES; } -#elif defined(__SSE2__) - /** A slightly more optimized version. - * Based on the assumption that often pieces of consecutive values - * completely pass or do not pass the filter. - * Therefore, we will optimistically check the parts of `SIMD_BYTES` values. - */ +#elif defined(__SSE2__) static constexpr size_t SIMD_BYTES = 16; const __m128i zero16 = _mm_setzero_si128(); const UInt8 * filt_end_sse = filt_pos + size / SIMD_BYTES * SIMD_BYTES; From be28e94471f4342e7b73b01b15b56b7b98392f45 Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Tue, 12 Oct 2021 06:30:56 +0530 Subject: [PATCH 138/919] add compile flag -force-vector-width=16 due to llvm autovectorization bug with avx512 --- cmake/cpu_features.cmake | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index 4ea9465be98..e77f330fbc0 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -129,8 +129,8 @@ else () if (HAVE_AVX2 AND ENABLE_AVX2) set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () - - set (TEST_FLAG "-mavx512f -mavx512bw") +#we have to add -force-vector-width=16 due to llvm autovectorization bug with avx512 + set (TEST_FLAG "-mavx512f -mavx512bw -force-vector-width=16") set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") check_cxx_source_compiles(" #include From 3be83a26d00dcfa4712627fc57a08169919a34d6 Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Tue, 12 Oct 2021 08:24:47 +0530 Subject: [PATCH 139/919] fixed Typos and Styles issues --- src/CMakeLists.txt | 2 +- src/Columns/ColumnFixedString.cpp | 8 ++++---- src/Columns/ColumnVector.cpp | 7 +++---- src/Columns/ColumnsCommon.cpp | 18 +++++++++--------- 4 files changed, 17 insertions(+), 18 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 45bb1a21d59..d5977b31159 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -284,7 +284,7 @@ target_link_libraries (clickhouse_common_io dragonbox_to_chars ) -# Use X86 AVX2/AVX512 instructions to accelerate filter opertions +# Use X86 AVX2/AVX512 instructions to accelerate filter operations set_source_files_properties( Columns/ColumnFixedString.cpp Columns/ColumnsCommon.cpp diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 4b31677d37e..65f56495445 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -238,13 +238,13 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result */ #if defined(__AVX512F__) && defined(__AVX512BW__) static constexpr size_t SIMD_BYTES = 64; - const __m512i zero64 = _mm512_setzero_epi32(); + const __m512i zero64 = _mm512_setzero_epi32(); const UInt8 * filt_end_avx512 = filt_pos + col_size / SIMD_BYTES * SIMD_BYTES; const size_t chars_per_simd_elements = SIMD_BYTES * n; while (filt_pos < filt_end_avx512) - { - uint64_t mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero64, _MM_CMPINT_GT); + { + uint64_t mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero64, _MM_CMPINT_GT); if (0xFFFFFFFFFFFFFFFF == mask) { @@ -263,7 +263,7 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result mask = _blsr_u64(mask); #else mask = mask & (mask-1); - #endif + #endif } } data_pos += chars_per_simd_elements; diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 37f60e9f2b9..78922841e37 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -310,7 +310,6 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s const UInt8 * filt_pos = filt.data(); const UInt8 * filt_end = filt_pos + size; const T * data_pos = data.data(); - /** A slightly more optimized version. * Based on the assumption that often pieces of consecutive values * completely pass or do not pass the filter. @@ -318,11 +317,11 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s */ #if defined(__AVX512F__) && defined(__AVX512BW__) static constexpr size_t SIMD_BYTES = 64; - const __m512i zero64 = _mm512_setzero_epi32(); + const __m512i zero64 = _mm512_setzero_epi32(); const UInt8 * filt_end_avx512 = filt_pos + size / SIMD_BYTES * SIMD_BYTES; while (filt_pos < filt_end_avx512) - { + { UInt64 mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero64, _MM_CMPINT_GT); if (0xFFFFFFFFFFFFFFFF == mask) @@ -339,7 +338,7 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s mask = _blsr_u64(mask); #else mask = mask & (mask-1); - #endif + #endif } } diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index 36c292b4196..d8e0cf74669 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -235,8 +235,8 @@ namespace const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; while (filt_pos < filt_end_aligned) - { - uint64_t mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero_vec, _MM_CMPINT_GT); + { + uint64_t mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero_vec, _MM_CMPINT_GT); if (mask == 0xffffffffffffffff) { @@ -258,18 +258,18 @@ namespace while (mask) { size_t index = __builtin_ctzll(mask); - copy_array(offsets_pos + index); + copy_array(offsets_pos + index); #ifdef __BMI__ mask = _blsr_u64(mask); #else mask = mask & (mask-1); - #endif + #endif } } filt_pos += SIMD_BYTES; offsets_pos += SIMD_BYTES; - } + } #elif defined(__AVX2__) const __m256i zero_vec = _mm256_setzero_si256(); static constexpr size_t SIMD_BYTES = 32; @@ -277,7 +277,7 @@ namespace while (filt_pos < filt_end_aligned) { - uint32_t mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero_vec)); + uint32_t mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero_vec)); if (mask == 0xffffffff) { @@ -299,18 +299,18 @@ namespace while (mask) { size_t index = __builtin_ctz(mask); - copy_array(offsets_pos + index); + copy_array(offsets_pos + index); #ifdef __BMI__ mask = _blsr_u32(mask); #else mask = mask & (mask-1); - #endif + #endif } } filt_pos += SIMD_BYTES; offsets_pos += SIMD_BYTES; - } + } #elif defined(__SSE2__) const __m128i zero_vec = _mm_setzero_si128(); static constexpr size_t SIMD_BYTES = 16; From 8f349907fb2fad802d9ceeb9cab7147d71fb8deb Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Tue, 12 Oct 2021 09:21:54 +0530 Subject: [PATCH 140/919] Fixed trailing whitespaces issue --- src/Columns/ColumnFixedString.cpp | 8 ++++---- src/Columns/ColumnVector.cpp | 6 +++--- src/Columns/ColumnsCommon.cpp | 4 ++-- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 65f56495445..a84db5701a7 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -235,7 +235,7 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result * Based on the assumption that often pieces of consecutive values * completely pass or do not pass the filter. * Therefore, we will optimistically check the parts of `SIMD_BYTES` values. - */ + */ #if defined(__AVX512F__) && defined(__AVX512BW__) static constexpr size_t SIMD_BYTES = 64; const __m512i zero64 = _mm512_setzero_epi32(); @@ -271,13 +271,13 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result } #elif defined(__AVX2__) static constexpr size_t SIMD_BYTES = 32; - const __m256i zero32 = _mm256_setzero_si256(); + const __m256i zero32 = _mm256_setzero_si256(); const UInt8 * filt_end_avx2 = filt_pos + col_size / SIMD_BYTES * SIMD_BYTES; const size_t chars_per_simd_elements = SIMD_BYTES * n; while (filt_pos < filt_end_avx2) { - uint32_t mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero32)); + uint32_t mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero32)); if (0xFFFFFFFF == mask) { @@ -296,7 +296,7 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result mask = _blsr_u32(mask); #else mask = mask & (mask-1); - #endif + #endif } } data_pos += chars_per_simd_elements; diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 78922841e37..6bc596008d8 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -348,12 +348,12 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s #elif defined(__AVX2__) static constexpr size_t SIMD_BYTES = 32; - const __m256i zero32 = _mm256_setzero_si256(); + const __m256i zero32 = _mm256_setzero_si256(); const UInt8 * filt_end_avx2 = filt_pos + size / SIMD_BYTES * SIMD_BYTES; while (filt_pos < filt_end_avx2) { - UInt32 mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero32)); + UInt32 mask = _mm256_movemask_epi8(_mm256_cmpgt_epi8(_mm256_loadu_si256(reinterpret_cast(filt_pos)), zero32)); if (0xFFFFFFFF == mask) { @@ -369,7 +369,7 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s mask = _blsr_u32(mask); #else mask = mask & (mask-1); - #endif + #endif } } diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index d8e0cf74669..845bac79ae0 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -237,7 +237,7 @@ namespace while (filt_pos < filt_end_aligned) { uint64_t mask = _mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(filt_pos)), zero_vec, _MM_CMPINT_GT); - + if (mask == 0xffffffffffffffff) { /// SIMD_BYTES consecutive rows pass the filter @@ -350,7 +350,7 @@ namespace filt_pos += SIMD_BYTES; offsets_pos += SIMD_BYTES; - } + } #endif while (filt_pos < filt_end) From ce645371f5a40862d03b23c973af717255b7620b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Oct 2021 14:46:27 +0300 Subject: [PATCH 141/919] Update container name. --- .../test_backward_compatibility/test_aggregate_fixed_key.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py index 8819be527fd..d895ee0eb78 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py @@ -3,9 +3,9 @@ import pytest from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', with_zookeeper=True, image='yandex/clickhouse-server', tag='21.3', with_installed_binary=True) -node2 = cluster.add_instance('node2', with_zookeeper=True, image='yandex/clickhouse-server') -node3 = cluster.add_instance('node3', with_zookeeper=True, image='yandex/clickhouse-server') +node1 = cluster.add_instance('node1', with_zookeeper=True, image='clickhouse/server', tag='21.3', with_installed_binary=True) +node2 = cluster.add_instance('node2', with_zookeeper=True, image='clickhouse/server') +node3 = cluster.add_instance('node3', with_zookeeper=True, image='clickhouse/server') @pytest.fixture(scope="module") From a1ab3b6606397c5748d7d478c753b96224a9c328 Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Tue, 12 Oct 2021 22:44:37 +0530 Subject: [PATCH 142/919] Fixed code style issue --- src/Columns/ColumnsCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index 845bac79ae0..804d00d4d7c 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -270,7 +270,7 @@ namespace filt_pos += SIMD_BYTES; offsets_pos += SIMD_BYTES; } - #elif defined(__AVX2__) + #elif defined(__AVX2__) const __m256i zero_vec = _mm256_setzero_si256(); static constexpr size_t SIMD_BYTES = 32; const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; From 002c8d3a802a6db935e2ffd33ede7dbf9f6b0107 Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Wed, 13 Oct 2021 03:38:18 +0530 Subject: [PATCH 143/919] add disable vectorize flag due to llvm bug --- cmake/cpu_features.cmake | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index e77f330fbc0..f0750c69294 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -129,8 +129,8 @@ else () if (HAVE_AVX2 AND ENABLE_AVX2) set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () -#we have to add -force-vector-width=16 due to llvm autovectorization bug with avx512 - set (TEST_FLAG "-mavx512f -mavx512bw -force-vector-width=16") +#Disable vectorize due to llvm autovectorization bug with avx512 + set (TEST_FLAG "-mavx512f -mavx512bw -fno-slp-vectorize -fno-vectorize") set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") check_cxx_source_compiles(" #include From f08dd0e8d8c475c7949a23d8b20499509e0c298c Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Wed, 13 Oct 2021 05:13:34 +0530 Subject: [PATCH 144/919] modify flags definitions --- cmake/cpu_features.cmake | 37 +++++++++++++++++++++++++------ src/Columns/ColumnFixedString.cpp | 2 +- src/Columns/ColumnVector.cpp | 2 +- src/Columns/ColumnsCommon.cpp | 2 +- 4 files changed, 33 insertions(+), 10 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index f0750c69294..44a804e64af 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -18,8 +18,10 @@ option (ENABLE_PCLMULQDQ "Use pclmulqdq instructions on x86_64" 1) option (ENABLE_POPCNT "Use popcnt instructions on x86_64" 1) option (ENABLE_AVX "Use AVX instructions on x86_64" 0) option (ENABLE_AVX2 "Use AVX2 instructions on x86_64" 0) -option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 1) -option (ENABLE_BMI "Use BMI instructions on x86_64" 1) +option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 0) +option (ENABLE_BMI "Use BMI instructions on x86_64" 0) +option (ENABLE_AVX2_FOR_SPEC_OP "Use avx2 instructions for specific operations on x86_64" 1) +option (ENABLE_AVX512_FOR_SPEC_OP "Use avx512 instructions for specific operations on x86_64" 0) option (ARCH_NATIVE "Add -march=native compiler flag. This makes your binaries non-portable but more performant code may be generated. This option overrides ENABLE_* options for specific instruction set. Highly not recommended to use." 0) @@ -129,8 +131,8 @@ else () if (HAVE_AVX2 AND ENABLE_AVX2) set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () -#Disable vectorize due to llvm autovectorization bug with avx512 - set (TEST_FLAG "-mavx512f -mavx512bw -fno-slp-vectorize -fno-vectorize") + + set (TEST_FLAG "-mavx512f -mavx512bw") set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") check_cxx_source_compiles(" #include @@ -143,7 +145,7 @@ else () } " HAVE_AVX512) if (HAVE_AVX512 AND ENABLE_AVX512) - set(X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} ${TEST_FLAG}") + set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () set (TEST_FLAG "-mbmi") @@ -157,8 +159,29 @@ else () } " HAVE_BMI) if (HAVE_BMI AND ENABLE_BMI) - set(X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} ${TEST_FLAG}") - endif () + set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") + endif () + + if (ENABLE_AVX2_FOR_SPEC_OP) + set (X86_INTRINSICS_FLAGS "") + if (HAVE_BMI) + set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mbmi") + endif () + if (HAVE_AVX AND HAVE_AVX2) + set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mavx -mavx2") + endif () + endif () + + if (ENABLE_AVX512_FOR_SPEC_OP) + set (X86_INTRINSICS_FLAGS "") + if (HAVE_BMI) + set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mbmi") + endif () + if (HAVE_AVX512) + set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mavx512f -mavx512bw") + endif () + endif () + endif () cmake_pop_check_state () diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index a84db5701a7..729d82da41d 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -269,7 +269,7 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result data_pos += chars_per_simd_elements; filt_pos += SIMD_BYTES; } -#elif defined(__AVX2__) +#elif defined(__AVX__) && defined(__AVX2__) static constexpr size_t SIMD_BYTES = 32; const __m256i zero32 = _mm256_setzero_si256(); const UInt8 * filt_end_avx2 = filt_pos + col_size / SIMD_BYTES * SIMD_BYTES; diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 6bc596008d8..4793b6bd9d5 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -346,7 +346,7 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s data_pos += SIMD_BYTES; } -#elif defined(__AVX2__) +#elif defined(__AVX__) && defined(__AVX2__) static constexpr size_t SIMD_BYTES = 32; const __m256i zero32 = _mm256_setzero_si256(); const UInt8 * filt_end_avx2 = filt_pos + size / SIMD_BYTES * SIMD_BYTES; diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index 804d00d4d7c..ea5ca4e45b4 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -270,7 +270,7 @@ namespace filt_pos += SIMD_BYTES; offsets_pos += SIMD_BYTES; } - #elif defined(__AVX2__) + #elif defined(__AVX__) && defined(__AVX2__) const __m256i zero_vec = _mm256_setzero_si256(); static constexpr size_t SIMD_BYTES = 32; const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES; From 80760c0a0c66711757fc087679112df27a57a9c3 Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Wed, 13 Oct 2021 05:59:20 +0530 Subject: [PATCH 145/919] revise compile definitions --- cmake/cpu_features.cmake | 28 +++------------------------- src/CMakeLists.txt | 7 ------- 2 files changed, 3 insertions(+), 32 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index 44a804e64af..c5a6eaf9c8f 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -16,12 +16,10 @@ option (ENABLE_SSE41 "Use SSE4.1 instructions on x86_64" 1) option (ENABLE_SSE42 "Use SSE4.2 instructions on x86_64" 1) option (ENABLE_PCLMULQDQ "Use pclmulqdq instructions on x86_64" 1) option (ENABLE_POPCNT "Use popcnt instructions on x86_64" 1) -option (ENABLE_AVX "Use AVX instructions on x86_64" 0) -option (ENABLE_AVX2 "Use AVX2 instructions on x86_64" 0) +option (ENABLE_AVX "Use AVX instructions on x86_64" 1) +option (ENABLE_AVX2 "Use AVX2 instructions on x86_64" 1) option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 0) -option (ENABLE_BMI "Use BMI instructions on x86_64" 0) -option (ENABLE_AVX2_FOR_SPEC_OP "Use avx2 instructions for specific operations on x86_64" 1) -option (ENABLE_AVX512_FOR_SPEC_OP "Use avx512 instructions for specific operations on x86_64" 0) +option (ENABLE_BMI "Use BMI instructions on x86_64" 1) option (ARCH_NATIVE "Add -march=native compiler flag. This makes your binaries non-portable but more performant code may be generated. This option overrides ENABLE_* options for specific instruction set. Highly not recommended to use." 0) @@ -162,26 +160,6 @@ else () set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () - if (ENABLE_AVX2_FOR_SPEC_OP) - set (X86_INTRINSICS_FLAGS "") - if (HAVE_BMI) - set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mbmi") - endif () - if (HAVE_AVX AND HAVE_AVX2) - set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mavx -mavx2") - endif () - endif () - - if (ENABLE_AVX512_FOR_SPEC_OP) - set (X86_INTRINSICS_FLAGS "") - if (HAVE_BMI) - set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mbmi") - endif () - if (HAVE_AVX512) - set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mavx512f -mavx512bw") - endif () - endif () - endif () cmake_pop_check_state () diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index d5977b31159..cac5b70f489 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -284,13 +284,6 @@ target_link_libraries (clickhouse_common_io dragonbox_to_chars ) -# Use X86 AVX2/AVX512 instructions to accelerate filter operations -set_source_files_properties( - Columns/ColumnFixedString.cpp - Columns/ColumnsCommon.cpp - Columns/ColumnVector.cpp - PROPERTIES COMPILE_FLAGS "${X86_INTRINSICS_FLAGS}") - if(RE2_LIBRARY) target_link_libraries(clickhouse_common_io PUBLIC ${RE2_LIBRARY}) endif() From 4b080489b5a15d89bd6b8b84aeae0f23acc5ed06 Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Wed, 13 Oct 2021 07:34:21 +0530 Subject: [PATCH 146/919] revise flags definitions --- cmake/cpu_features.cmake | 28 +++++++++++++++++++++++++--- src/CMakeLists.txt | 7 +++++++ 2 files changed, 32 insertions(+), 3 deletions(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index c5a6eaf9c8f..41a3dc26d5a 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -16,10 +16,12 @@ option (ENABLE_SSE41 "Use SSE4.1 instructions on x86_64" 1) option (ENABLE_SSE42 "Use SSE4.2 instructions on x86_64" 1) option (ENABLE_PCLMULQDQ "Use pclmulqdq instructions on x86_64" 1) option (ENABLE_POPCNT "Use popcnt instructions on x86_64" 1) -option (ENABLE_AVX "Use AVX instructions on x86_64" 1) -option (ENABLE_AVX2 "Use AVX2 instructions on x86_64" 1) +option (ENABLE_AVX "Use AVX instructions on x86_64" 0) +option (ENABLE_AVX2 "Use AVX2 instructions on x86_64" 0) option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 0) -option (ENABLE_BMI "Use BMI instructions on x86_64" 1) +option (ENABLE_BMI "Use BMI instructions on x86_64" 0) +option (ENABLE_AVX2_FOR_SPEC_OP "Use avx2 instructions for specific operations on x86_64" 0) +option (ENABLE_AVX512_FOR_SPEC_OP "Use avx512 instructions for specific operations on x86_64" 0) option (ARCH_NATIVE "Add -march=native compiler flag. This makes your binaries non-portable but more performant code may be generated. This option overrides ENABLE_* options for specific instruction set. Highly not recommended to use." 0) @@ -160,6 +162,26 @@ else () set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () +#Limit avx2/avx512 flag for specific source build + if (ENABLE_AVX2_FOR_SPEC_OP) + set (X86_INTRINSICS_FLAGS "") + if (HAVE_BMI) + set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mbmi") + endif () + if (HAVE_AVX AND HAVE_AVX2) + set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mavx -mavx2") + endif () + endif () + + if (ENABLE_AVX512_FOR_SPEC_OP) + set (X86_INTRINSICS_FLAGS "") + if (HAVE_BMI) + set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mbmi") + endif () + if (HAVE_AVX512) + set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mavx512f -mavx512bw") + endif () + endif () endif () cmake_pop_check_state () diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index cac5b70f489..d5977b31159 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -284,6 +284,13 @@ target_link_libraries (clickhouse_common_io dragonbox_to_chars ) +# Use X86 AVX2/AVX512 instructions to accelerate filter operations +set_source_files_properties( + Columns/ColumnFixedString.cpp + Columns/ColumnsCommon.cpp + Columns/ColumnVector.cpp + PROPERTIES COMPILE_FLAGS "${X86_INTRINSICS_FLAGS}") + if(RE2_LIBRARY) target_link_libraries(clickhouse_common_io PUBLIC ${RE2_LIBRARY}) endif() From 0409b775bf2943295ed31c5ad59dc6bd36fbb435 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 13 Oct 2021 02:11:15 +0000 Subject: [PATCH 147/919] fix --- src/Storages/FileLog/DirectoryWatcherBase.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index 00c37fe9a75..4f2a775c265 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -65,9 +65,10 @@ void DirectoryWatcherBase::watchFunc() #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wreserved-identifier" FD_ZERO(&fds); - FD_SET(fd, &fds); #pragma clang diagnostic pop + FD_SET(fd, &fds); + struct timeval tv; tv.tv_sec = 0; tv.tv_usec = 200000; From 74e1900034e3beec14fdcf67c9f31f79f271ae2d Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 13 Oct 2021 05:23:00 +0000 Subject: [PATCH 148/919] fix --- src/Storages/FileLog/DirectoryWatcherBase.cpp | 14 +++++++++---- .../FileLog/ReadBufferFromFileLog.cpp | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 20 +++++++++---------- 3 files changed, 21 insertions(+), 15 deletions(-) diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index 4f2a775c265..076475b8635 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -24,14 +24,14 @@ DirectoryWatcherBase::DirectoryWatcherBase( : WithContext(context_->getGlobalContext()), owner(owner_), path(path_), event_mask(event_mask_) { if (!std::filesystem::exists(path)) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "The path {} does not exist.", path); + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "The path {} does not exist", path); if (!std::filesystem::is_directory(path)) - throw Exception(ErrorCodes::DIRECTORY_DOESNT_EXIST, "The path {} does not a directory.", path); + throw Exception(ErrorCodes::DIRECTORY_DOESNT_EXIST, "The path {} does not a directory", path); fd = inotify_init(); if (fd == -1) - throw Exception("cannot initialize inotify", ErrorCodes::IO_SETUP_ERROR); + throw Exception("Cannot initialize inotify", ErrorCodes::IO_SETUP_ERROR); watch_task = getContext()->getMessageBrokerSchedulePool().createTask("directory_watch", [this] { watchFunc(); }); start(); @@ -54,7 +54,7 @@ void DirectoryWatcherBase::watchFunc() int wd = inotify_add_watch(fd, path.c_str(), mask); if (wd == -1) { - owner.onError(Exception(ErrorCodes::IO_SETUP_ERROR, "Watch directory {} failed.", path)); + owner.onError(Exception(ErrorCodes::IO_SETUP_ERROR, "Watch directory {} failed", path)); } std::string buffer; @@ -62,11 +62,17 @@ void DirectoryWatcherBase::watchFunc() fd_set fds; while (!stopped) { +#if defined(__clang__) #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wreserved-identifier" FD_ZERO(&fds); #pragma clang diagnostic pop +#else + FD_ZERO(&fds); +#endif + + FD_SET(fd, &fds); struct timeval tv; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index cf52b2a2ccc..745d8c46a63 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -107,7 +107,7 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ auto & file_meta = StorageFileLog::findInMap(file_infos.meta_by_inode, file_ctx.inode); if (!file_ctx.reader) - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Ifstream for file {} does not initialized.", file_meta.file_name); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Ifstream for file {} does not initialized", file_meta.file_name); auto & reader = file_ctx.reader.value(); Record record; diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index c8ffd44369e..516be811666 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -270,7 +270,7 @@ void StorageFileLog::deserialize() { throw Exception( ErrorCodes::NOT_REGULAR_FILE, - "The file {} under {} is not a regular file when deserializing meta files.", + "The file {} under {} is not a regular file when deserializing meta files", dir_entry.path().c_str(), root_meta_path); } @@ -281,12 +281,12 @@ void StorageFileLog::deserialize() if (!tryReadIntText(inode, in)) { - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", dir_entry.path().c_str()); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed", dir_entry.path().c_str()); } assertChar('\n', in); if (!tryReadIntText(last_written_pos, in)) { - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", dir_entry.path().c_str()); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed", dir_entry.path().c_str()); } meta.file_name = dir_entry.path().filename(); @@ -322,7 +322,7 @@ Pipe StorageFileLog::read( { throw Exception( ErrorCodes::CANNOT_READ_ALL_DATA, - "Can not read from table {}, because it has been depended by other tables.", + "Can not read from table {}, because it has been depended by other tables", table_id.getTableName()); } @@ -433,7 +433,7 @@ void StorageFileLog::assertStreamGood(const std::ifstream & reader) { if (!reader.good()) { - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Stream is in bad state."); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Stream is in bad state"); } } @@ -457,7 +457,7 @@ void StorageFileLog::openFilesAndSetPos() auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); if (meta.last_writen_position > static_cast(file_end)) { - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "File {} has been broken.", file); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "File {} has been broken", file); } /// update file end at the monment, used in ReadBuffer and serialize meta.last_open_end = file_end; @@ -516,16 +516,16 @@ void StorageFileLog::checkOffsetIsValid(const String & full_name, UInt64 offset) if (!tryReadIntText(_, in)) { - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", full_name); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed", full_name); } assertChar('\n', in); if (!tryReadIntText(last_written_pos, in)) { - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed.", full_name); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Read meta file {} failed", full_name); } if (last_written_pos > offset) throw Exception( - ErrorCodes::LOGICAL_ERROR, "Last stored last_written_pos in meta file {} is bigger than current last_written_pos.", full_name); + ErrorCodes::LOGICAL_ERROR, "Last stored last_written_pos in meta file {} is bigger than current last_written_pos", full_name); } size_t StorageFileLog::getMaxBlockSize() const @@ -633,7 +633,7 @@ bool StorageFileLog::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); + throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist", ErrorCodes::LOGICAL_ERROR); auto metadata_snapshot = getInMemoryMetadataPtr(); auto max_streams_number = std::min(filelog_settings->max_threads.value, file_infos.file_names.size()); From 868dbc4bad1a90dc428bb9f7a6e81a5d34a27259 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 13 Oct 2021 09:10:41 +0000 Subject: [PATCH 149/919] use poll replace select fix fix --- CMakeLists.txt | 3 +-- cmake/find/filelog.cmake | 13 ++++++++++ src/Storages/FileLog/DirectoryWatcherBase.cpp | 25 ++++--------------- 3 files changed, 19 insertions(+), 22 deletions(-) create mode 100644 cmake/find/filelog.cmake diff --git a/CMakeLists.txt b/CMakeLists.txt index f6cbe89eafc..7e2ace06397 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -570,6 +570,7 @@ include (cmake/find/yaml-cpp.cmake) include (cmake/find/s2geometry.cmake) include (cmake/find/nlp.cmake) include (cmake/find/bzip2.cmake) +include (cmake/find/filelog.cmake) if(NOT USE_INTERNAL_PARQUET_LIBRARY) set (ENABLE_ORC OFF CACHE INTERNAL "") @@ -583,8 +584,6 @@ include (cmake/find/sentry.cmake) include (cmake/find/stats.cmake) include (cmake/find/datasketches.cmake) -set (USE_FILELOG ON) - set (USE_INTERNAL_CITYHASH_LIBRARY ON CACHE INTERNAL "") find_contrib_lib(cityhash) diff --git a/cmake/find/filelog.cmake b/cmake/find/filelog.cmake new file mode 100644 index 00000000000..6c38ba33fbf --- /dev/null +++ b/cmake/find/filelog.cmake @@ -0,0 +1,13 @@ +option (ENABLE_FILELOG "Enable FILELOG" ON) + +if (NOT ENABLE_FILELOG) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use StorageFileLog with ENABLE_FILELOG=OFF") + return() +endif() + +# StorageFileLog only support Linux platform +if(${CMAKE_SYSTEM_NAME} MATCHES "Linux") + set (USE_FILELOG 1) + message (STATUS "Using StorageFileLog = 1") +endif(${CMAKE_SYSTEM_NAME} MATCHES "Linux") + diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index 076475b8635..4f6e429e1fd 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include namespace DB { @@ -59,27 +59,12 @@ void DirectoryWatcherBase::watchFunc() std::string buffer; buffer.resize(buffer_size); - fd_set fds; + pollfd pfd; + pfd.fd = fd; + pfd.events = POLLIN; while (!stopped) { -#if defined(__clang__) -#pragma clang diagnostic push -#pragma clang diagnostic ignored "-Wreserved-identifier" - FD_ZERO(&fds); -#pragma clang diagnostic pop - -#else - FD_ZERO(&fds); -#endif - - - FD_SET(fd, &fds); - - struct timeval tv; - tv.tv_sec = 0; - tv.tv_usec = 200000; - - if (select(fd + 1, &fds, nullptr, nullptr, &tv) == 1) + if (poll(&pfd, 1, 5000) > 0 && pfd.revents & POLLIN) { int n = read(fd, buffer.data(), buffer.size()); int i = 0; From c82fd3491ded3999a16ae26d5c6c4ec874cd9493 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 13 Oct 2021 11:06:27 +0000 Subject: [PATCH 150/919] fix fix --- src/Storages/FileLog/FileLogDirectoryWatcher.cpp | 2 +- src/Storages/FileLog/FileLogDirectoryWatcher.h | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index bd4a7543366..6d3230616f4 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -4,8 +4,8 @@ namespace DB { FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_, ContextPtr context_) : path(path_) - , dw(std::make_unique(*this, path, context_)) , log(&Poco::Logger::get("FileLogDirectoryWatcher(" + path + ")")) + , dw(std::make_unique(*this, path, context_)) { } diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index b050d9a2c39..4733dd25e0e 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -51,14 +51,16 @@ private: /// The data race is because dw create a separate thread to monitor file events /// and put into events, then if we destruct events first, the monitor thread still /// running, it may access events during events destruction, leads to data race. + /// And we should put other members before dw as well, because all of them can be + /// accessed in thread created by dw. Events events; - std::unique_ptr dw; - Poco::Logger * log; std::mutex mutex; Error error; + + std::unique_ptr dw; }; } From 168020e0db0a5217b90191b293d5212b024361b3 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 13 Oct 2021 14:02:32 +0000 Subject: [PATCH 151/919] fix --- src/Storages/FileLog/FileLogDirectoryWatcher.cpp | 12 ++++++------ src/Storages/FileLog/FileLogDirectoryWatcher.h | 13 +++++++------ 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 6d3230616f4..4dadbed3f09 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -30,7 +30,7 @@ const std::string & FileLogDirectoryWatcher::getPath() const return path; } -void FileLogDirectoryWatcher::onItemAdded(const DirectoryWatcherBase::DirectoryEvent & ev) +void FileLogDirectoryWatcher::onItemAdded(DirectoryWatcherBase::DirectoryEvent ev) { std::lock_guard lock(mutex); @@ -48,7 +48,7 @@ void FileLogDirectoryWatcher::onItemAdded(const DirectoryWatcherBase::DirectoryE } -void FileLogDirectoryWatcher::onItemRemoved(const DirectoryWatcherBase::DirectoryEvent & ev) +void FileLogDirectoryWatcher::onItemRemoved(DirectoryWatcherBase::DirectoryEvent ev) { std::lock_guard lock(mutex); @@ -71,7 +71,7 @@ void FileLogDirectoryWatcher::onItemRemoved(const DirectoryWatcherBase::Director /// So, if we record all of these events, it will use a lot of memory, and then we /// need to handle it one by one in StorageFileLog::updateFileInfos, this is unnecessary /// because it is equal to just record and handle one MODIY event -void FileLogDirectoryWatcher::onItemModified(const DirectoryWatcherBase::DirectoryEvent & ev) +void FileLogDirectoryWatcher::onItemModified(DirectoryWatcherBase::DirectoryEvent ev) { std::lock_guard lock(mutex); @@ -91,7 +91,7 @@ void FileLogDirectoryWatcher::onItemModified(const DirectoryWatcherBase::Directo } } -void FileLogDirectoryWatcher::onItemMovedFrom(const DirectoryWatcherBase::DirectoryEvent & ev) +void FileLogDirectoryWatcher::onItemMovedFrom(DirectoryWatcherBase::DirectoryEvent ev) { std::lock_guard lock(mutex); @@ -108,7 +108,7 @@ void FileLogDirectoryWatcher::onItemMovedFrom(const DirectoryWatcherBase::Direct } } -void FileLogDirectoryWatcher::onItemMovedTo(const DirectoryWatcherBase::DirectoryEvent & ev) +void FileLogDirectoryWatcher::onItemMovedTo(DirectoryWatcherBase::DirectoryEvent ev) { std::lock_guard lock(mutex); @@ -125,7 +125,7 @@ void FileLogDirectoryWatcher::onItemMovedTo(const DirectoryWatcherBase::Director } } -void FileLogDirectoryWatcher::onError(const Exception & e) +void FileLogDirectoryWatcher::onError(Exception e) { std::lock_guard lock(mutex); LOG_ERROR(log, "Error happened during watching directory {}: {}", path, error.error_msg); diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index 4733dd25e0e..4259cc8722e 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -36,12 +36,13 @@ public: const std::string & getPath() const; - void onItemAdded(const DirectoryWatcherBase::DirectoryEvent & ev); - void onItemRemoved(const DirectoryWatcherBase::DirectoryEvent & ev); - void onItemModified(const DirectoryWatcherBase::DirectoryEvent & ev); - void onItemMovedFrom(const DirectoryWatcherBase::DirectoryEvent & ev); - void onItemMovedTo(const DirectoryWatcherBase::DirectoryEvent & ev); - void onError(const Exception &); + /// Here must pass by value, otherwise will lead to stack-use-of-scope + void onItemAdded(DirectoryWatcherBase::DirectoryEvent ev); + void onItemRemoved(DirectoryWatcherBase::DirectoryEvent ev); + void onItemModified(DirectoryWatcherBase::DirectoryEvent ev); + void onItemMovedFrom(DirectoryWatcherBase::DirectoryEvent ev); + void onItemMovedTo(DirectoryWatcherBase::DirectoryEvent ev); + void onError(Exception); private: const std::string path; From 3d0e595579a641f30b46b3577a66059d87a15e0e Mon Sep 17 00:00:00 2001 From: jasperzhu Date: Wed, 13 Oct 2021 22:49:21 +0530 Subject: [PATCH 152/919] revise compile flag --- cmake/cpu_features.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/cpu_features.cmake b/cmake/cpu_features.cmake index 41a3dc26d5a..330ab10f1bf 100644 --- a/cmake/cpu_features.cmake +++ b/cmake/cpu_features.cmake @@ -163,8 +163,8 @@ else () endif () #Limit avx2/avx512 flag for specific source build + set (X86_INTRINSICS_FLAGS "") if (ENABLE_AVX2_FOR_SPEC_OP) - set (X86_INTRINSICS_FLAGS "") if (HAVE_BMI) set (X86_INTRINSICS_FLAGS "${X86_INTRINSICS_FLAGS} -mbmi") endif () From 9cbf4b842295a09196ae096425925f98912cc5a2 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Wed, 13 Oct 2021 22:20:24 +0300 Subject: [PATCH 153/919] distributed_push_down_limit: Update and add RU --- docs/en/operations/settings/settings.md | 10 ++++++---- docs/ru/operations/settings/settings.md | 16 ++++++++++++++++ 2 files changed, 22 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index aa70eb4f721..65940172f0b 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1786,14 +1786,16 @@ FORMAT PrettyCompactMonoBlock Default value: 0 -## distributed_push_down_limit (#distributed-push-down-limit} +## distributed_push_down_limit {#distributed-push-down-limit} -LIMIT will be applied on each shard separatelly. Usually you don't need to use it, since this will be done automatically if it is possible, i.e. for simple query SELECT FROM LIMIT. +Enables or disables [LIMIT](#limit) applying on each shard separatelly. Usually you don't need to use it, since this will be done automatically if it is possible, i.e. for simple query SELECT FROM LIMIT. Possible values: -- 0 - Disabled -- 1 - Enabled +- 0 - Disabled. +- 1 - Enabled. + +Default value: `1`. !!! note "Note" That with this setting the result of the query may be inaccurate. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index bccbbf69e39..67b23808f61 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1705,6 +1705,22 @@ ClickHouse генерирует исключение Значение по умолчанию: 0. +## distributed_push_down_limit {#distributed-push-down-limit} + +Включает или отключает [LIMIT](#limit), применяемый к каждому шарду по отдельности. + +Обычно, её включение не требуется, так как это будет сделано автоматически, если это возможно. + +Возможные значения: + +- 0 — Выключена. +- 1 — Включена. + +Значение по умолчанию: `1`. + +!!! note "Примечание" + При использованаии этой настройки, результат запроса может быть неточным. + ## optimize_skip_unused_shards {#optimize-skip-unused-shards} Включает или отключает пропуск неиспользуемых шардов для запросов [SELECT](../../sql-reference/statements/select/index.md) , в которых условие ключа шардирования задано в секции `WHERE/PREWHERE`. Предполагается, что данные распределены с помощью ключа шардирования, в противном случае настройка ничего не делает. From 3116767834dd59cd6a8ed444966911bef060f12e Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 14 Oct 2021 02:09:43 +0000 Subject: [PATCH 154/919] fix --- src/Storages/FileLog/DirectoryWatcherBase.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/FileLog/DirectoryWatcherBase.h b/src/Storages/FileLog/DirectoryWatcherBase.h index c9530453e2e..db89746358b 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.h +++ b/src/Storages/FileLog/DirectoryWatcherBase.h @@ -57,7 +57,7 @@ public: DirectoryEvent(const std::string & f, DirectoryEventType ev) : path(f), event(ev) { } /// The directory or file that has been changed. - const std::string & path; + const std::string path; /// The kind of event. DirectoryEventType event; }; From 324dfd4f814e246b544e8d79fe1b79c9cf152446 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 14 Oct 2021 13:32:49 +0300 Subject: [PATCH 155/919] Refactor and improve TSV, CSV and JSONCompactEachRow formats, fix some bugs in formats --- src/Core/Block.cpp | 11 + src/Core/Block.h | 1 + src/Core/Settings.h | 4 +- src/DataTypes/DataTypeLowCardinality.h | 1 + src/DataTypes/IDataType.h | 3 + .../Serializations/ISerialization.cpp | 15 + src/DataTypes/Serializations/ISerialization.h | 8 + .../SerializationFixedString.cpp | 2 +- .../Serializations/SerializationNullable.cpp | 48 +- .../Serializations/SerializationNullable.h | 7 + .../Serializations/SerializationString.cpp | 2 +- src/Formats/FormatFactory.cpp | 5 +- src/Formats/FormatSettings.h | 2 + src/Formats/JSONEachRowUtils.cpp | 55 +- src/Formats/JSONEachRowUtils.h | 8 +- src/Formats/registerFormats.cpp | 2 + src/Processors/Formats/IInputFormat.h | 13 +- src/Processors/Formats/IRowOutputFormat.cpp | 7 + src/Processors/Formats/IRowOutputFormat.h | 3 + .../Formats/Impl/CSVRowInputFormat.cpp | 411 ++++---------- .../Formats/Impl/CSVRowInputFormat.h | 36 +- .../Formats/Impl/CSVRowOutputFormat.cpp | 50 +- .../Formats/Impl/CSVRowOutputFormat.h | 6 +- .../Impl/JSONAsStringRowInputFormat.cpp | 2 +- .../Impl/JSONCompactEachRowRowInputFormat.cpp | 386 ++++++-------- .../Impl/JSONCompactEachRowRowInputFormat.h | 48 +- .../JSONCompactEachRowRowOutputFormat.cpp | 105 ++-- .../Impl/JSONCompactEachRowRowOutputFormat.h | 9 +- .../Impl/JSONEachRowRowInputFormat.cpp | 48 +- .../Formats/Impl/RegexpRowInputFormat.cpp | 6 +- .../Formats/Impl/TSKVRowInputFormat.cpp | 2 +- .../Formats/Impl/TSKVRowOutputFormat.cpp | 2 +- .../Impl/TabSeparatedRawRowInputFormat.h | 58 -- .../Impl/TabSeparatedRawRowOutputFormat.h | 35 -- .../Impl/TabSeparatedRowInputFormat.cpp | 501 ++++++------------ .../Formats/Impl/TabSeparatedRowInputFormat.h | 41 +- .../Impl/TabSeparatedRowOutputFormat.cpp | 98 ++-- .../Impl/TabSeparatedRowOutputFormat.h | 8 +- .../Impl/TemplateBlockOutputFormat.cpp | 2 +- .../Formats/Impl/TemplateRowInputFormat.cpp | 16 +- .../Formats/Impl/ValuesBlockInputFormat.cpp | 4 +- .../RowInputFormatWithNamesAndTypes.cpp | 265 +++++++++ .../Formats/RowInputFormatWithNamesAndTypes.h | 73 +++ tests/queries/0_stateless/00300_csv.reference | 4 + tests/queries/0_stateless/00300_csv.sql | 3 +- tests/queries/0_stateless/00301_csv.sh | 4 +- .../00938_template_input_format.reference | 8 + .../00938_template_input_format.sh | 24 + .../01034_JSONCompactEachRow.reference | 21 + .../0_stateless/01034_JSONCompactEachRow.sql | 15 +- .../01195_formats_diagnostic_info.reference | 13 +- .../01195_formats_diagnostic_info.sh | 16 + ...output_format_tsv_csv_with_names.reference | 14 + .../01375_output_format_tsv_csv_with_names.sh | 9 + ..._tsv_csv_with_names_write_prefix.reference | 25 + ...ge_file_tsv_csv_with_names_write_prefix.sh | 12 +- ...48_json_compact_strings_each_row.reference | 21 + .../01448_json_compact_strings_each_row.sql | 15 +- ...097_json_strings_deserialization.reference | 4 + .../02097_json_strings_deserialization.sh | 14 + .../02098_with_types_use_header.reference | 16 + .../02098_with_types_use_header.sh | 33 ++ .../02099_tsv_raw_format.reference | 113 ++++ .../0_stateless/02099_tsv_raw_format.sh | 59 +++ ...ardinality_nullable_null_default.reference | 14 + ...0_low_cardinality_nullable_null_default.sh | 21 + ...ty_as_default_and_omitted_fields.reference | 16 + ...101_empty_as_default_and_omitted_fields.sh | 39 ++ ...llel_formatting_json_and_friends.reference | 28 +- ...59_parallel_formatting_json_and_friends.sh | 6 +- ...lel_parsing_with_names_and_types.reference | 20 + ...7_parallel_parsing_with_names_and_types.sh | 32 ++ 72 files changed, 1743 insertions(+), 1285 deletions(-) delete mode 100644 src/Processors/Formats/Impl/TabSeparatedRawRowInputFormat.h delete mode 100644 src/Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h create mode 100644 src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp create mode 100644 src/Processors/Formats/RowInputFormatWithNamesAndTypes.h create mode 100644 tests/queries/0_stateless/02097_json_strings_deserialization.reference create mode 100755 tests/queries/0_stateless/02097_json_strings_deserialization.sh create mode 100644 tests/queries/0_stateless/02098_with_types_use_header.reference create mode 100755 tests/queries/0_stateless/02098_with_types_use_header.sh create mode 100644 tests/queries/0_stateless/02099_tsv_raw_format.reference create mode 100755 tests/queries/0_stateless/02099_tsv_raw_format.sh create mode 100644 tests/queries/0_stateless/02100_low_cardinality_nullable_null_default.reference create mode 100755 tests/queries/0_stateless/02100_low_cardinality_nullable_null_default.sh create mode 100644 tests/queries/0_stateless/02101_empty_as_default_and_omitted_fields.reference create mode 100755 tests/queries/0_stateless/02101_empty_as_default_and_omitted_fields.sh create mode 100644 tests/queries/1_stateful/00167_parallel_parsing_with_names_and_types.reference create mode 100755 tests/queries/1_stateful/00167_parallel_parsing_with_names_and_types.sh diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index a59ac60155e..43c3fa9a54a 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -582,6 +582,17 @@ DataTypes Block::getDataTypes() const return res; } +Names Block::getDataTypeNames() const +{ + Names res; + res.reserve(columns()); + + for (const auto & elem : data) + res.push_back(elem.type->getName()); + + return res; +} + bool blocksHaveEqualStructure(const Block & lhs, const Block & rhs) { diff --git a/src/Core/Block.h b/src/Core/Block.h index a7e3cee194b..973b0028219 100644 --- a/src/Core/Block.h +++ b/src/Core/Block.h @@ -90,6 +90,7 @@ public: NamesAndTypesList getNamesAndTypesList() const; Names getNames() const; DataTypes getDataTypes() const; + Names getDataTypeNames() const; /// Returns number of rows from first column in block, not equal to nullptr. If no columns, returns 0. size_t rows() const; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a5767955045..9d1f8163dfe 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -557,8 +557,10 @@ class IColumn; M(Bool, input_format_csv_arrays_as_nested_csv, false, R"(When reading Array from CSV, expect that its elements were serialized in nested CSV and then put into string. Example: "[""Hello"", ""world"", ""42"""" TV""]". Braces around array can be omitted.)", 0) \ M(Bool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow, CSVWithNames, TSVWithNames and TSKV formats).", 0) \ M(Bool, input_format_with_names_use_header, true, "For TSVWithNames and CSVWithNames input formats this controls whether format parser is to assume that column data appear in the input exactly as they are specified in the header.", 0) \ + M(Bool, input_format_with_types_use_header, true, "For TSVWithNamesTypes and CSVWithNamesAndTypes input formats this controls whether format parser should check if data types from the input match data types from the header.", 0) \ M(Bool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).", 0) \ - M(Bool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, CSV and TSV formats).", IMPORTANT) \ + M(Bool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, JSONCompactEachRow, CSV and TSV formats).", IMPORTANT) \ + M(Bool, input_format_csv_empty_as_default, true, "Treat empty fields in CSV input as default values.", 0) \ M(Bool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \ M(Bool, input_format_tsv_enum_as_number, false, "Treat inserted enum values in TSV formats as enum indices \\N", 0) \ M(Bool, input_format_null_as_default, true, "For text input formats initialize null fields with default values if data type of this field is not nullable", 0) \ diff --git a/src/DataTypes/DataTypeLowCardinality.h b/src/DataTypes/DataTypeLowCardinality.h index 1266174c6d6..7f4286046d9 100644 --- a/src/DataTypes/DataTypeLowCardinality.h +++ b/src/DataTypes/DataTypeLowCardinality.h @@ -51,6 +51,7 @@ public: bool isNullable() const override { return false; } bool onlyNull() const override { return false; } bool lowCardinality() const override { return true; } + bool isLowCardinalityNullable() const override { return dictionary_type->isNullable(); } static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type); static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type, MutableColumnPtr && keys); diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 360bf9f16e0..6b156336f99 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -270,6 +270,9 @@ public: virtual bool lowCardinality() const { return false; } + /// Checks if this type is LowCardinality(Nullable(...)) + virtual bool isLowCardinalityNullable() const { return false; } + /// Strings, Numbers, Date, DateTime, Nullable virtual bool canBeInsideLowCardinality() const { return false; } diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index 7077c5bfa14..e7ee8f56ecb 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -195,4 +196,18 @@ bool ISerialization::isSpecialCompressionAllowed(const SubstreamPath & path) return true; } +void ISerialization::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + String field; + /// Read until \t or \n. + readString(field, istr); + ReadBufferFromString buf(field); + deserializeWholeText(column, buf, settings); +} + +void ISerialization::serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + serializeText(column, row_num, ostr, settings); +} + } diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index f1d82a2000a..4fda939aeca 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -246,6 +246,14 @@ public: serializeText(column, row_num, ostr, settings); } + /** Text deserialization without escaping and quoting. Reads all data until first \n or \t + * into a temporary string and then call deserializeWholeText. It was implemented this way + * because this function is rarely used and because proper implementation requires a lot of + * additional code in data types serialization and ReadHelpers. + */ + virtual void deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const; + virtual void serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const; + static String getFileNameForStream(const NameAndTypePair & column, const SubstreamPath & path); static String getFileNameForStream(const String & name_in_storage, const SubstreamPath & path); static String getSubcolumnNameForStream(const SubstreamPath & path); diff --git a/src/DataTypes/Serializations/SerializationFixedString.cpp b/src/DataTypes/Serializations/SerializationFixedString.cpp index 5c63631e2a3..972313a564f 100644 --- a/src/DataTypes/Serializations/SerializationFixedString.cpp +++ b/src/DataTypes/Serializations/SerializationFixedString.cpp @@ -163,7 +163,7 @@ void SerializationFixedString::deserializeTextQuoted(IColumn & column, ReadBuffe void SerializationFixedString::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { - read(*this, column, [&istr](ColumnFixedString::Chars & data) { readStringInto(data, istr); }); + read(*this, column, [&istr](ColumnFixedString::Chars & data) { readStringUntilEOFInto(data, istr); }); } diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index b607d5871d6..865e4250bdc 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -218,8 +219,36 @@ void SerializationNullable::deserializeTextEscaped(IColumn & column, ReadBuffer deserializeTextEscapedImpl(column, istr, settings, nested); } +void SerializationNullable::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + deserializeTextRawImpl(column, istr, settings, nested); +} + +void SerializationNullable::serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + const ColumnNullable & col = assert_cast(column); + + if (col.isNullAt(row_num)) + writeString(settings.tsv.null_representation, ostr); + else + nested->serializeTextRaw(col.getNestedColumn(), row_num, ostr, settings); +} + +template +ReturnType SerializationNullable::deserializeTextRawImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested) +{ + return deserializeTextEscapedAndRawImpl(column, istr, settings, nested); +} + template ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, + const SerializationPtr & nested) +{ + return deserializeTextEscapedAndRawImpl(column, istr, settings, nested); +} + +template +ReturnType SerializationNullable::deserializeTextEscapedAndRawImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested) { /// Little tricky, because we cannot discriminate null from first character. @@ -229,7 +258,13 @@ ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, R /// This is not null, surely. return safeDeserialize(column, *nested, [] { return false; }, - [&nested, &istr, &settings] (IColumn & nested_column) { nested->deserializeTextEscaped(nested_column, istr, settings); }); + [&nested, &istr, &settings] (IColumn & nested_column) + { + if constexpr (escaped) + nested->deserializeTextEscaped(nested_column, istr, settings); + else + nested->deserializeTextRaw(nested_column, istr, settings); + }); } else { @@ -255,7 +290,10 @@ ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, R { /// We could step back to consume backslash again. --istr.position(); - nested->deserializeTextEscaped(nested_column, istr, settings); + if constexpr (escaped) + nested->deserializeTextEscaped(nested_column, istr, settings); + else + nested->deserializeTextRaw(nested_column, istr, settings); } else { @@ -263,7 +301,10 @@ ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, R ReadBufferFromMemory prefix("\\", 1); ConcatReadBuffer prepended_istr(prefix, istr); - nested->deserializeTextEscaped(nested_column, prepended_istr, settings); + if constexpr (escaped) + nested->deserializeTextEscaped(nested_column, prepended_istr, settings); + else + nested->deserializeTextRaw(nested_column, prepended_istr, settings); /// Synchronise cursor position in original buffer. @@ -469,5 +510,6 @@ template bool SerializationNullable::deserializeTextEscapedImpl(IColumn & template bool SerializationNullable::deserializeTextQuotedImpl(IColumn & column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested); template bool SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested); template bool SerializationNullable::deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested); +template bool SerializationNullable::deserializeTextRawImpl(IColumn & column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested); } diff --git a/src/DataTypes/Serializations/SerializationNullable.h b/src/DataTypes/Serializations/SerializationNullable.h index b0b96c021d3..acc3456e1fd 100644 --- a/src/DataTypes/Serializations/SerializationNullable.h +++ b/src/DataTypes/Serializations/SerializationNullable.h @@ -68,6 +68,9 @@ public: void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + /// If ReturnType is bool, check for NULL and deserialize value into non-nullable column (and return true) or insert default value of nested type (and return false) /// If ReturnType is void, deserialize Nullable(T) template @@ -80,6 +83,10 @@ public: static ReturnType deserializeTextCSVImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested); template static ReturnType deserializeTextJSONImpl(IColumn & column, ReadBuffer & istr, const FormatSettings &, const SerializationPtr & nested); + template + static ReturnType deserializeTextRawImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested); + template + static ReturnType deserializeTextEscapedAndRawImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested); }; } diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index c3c24ed6749..89f7fe4ad9d 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -245,7 +245,7 @@ static inline void read(IColumn & column, Reader && reader) void SerializationString::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { - read(column, [&](ColumnString::Chars & data) { readStringInto(data, istr); }); + read(column, [&](ColumnString::Chars & data) { readStringUntilEOFInto(data, istr); }); } diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index d3ff5cbf8a7..4b9c2fbf78d 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -15,7 +15,6 @@ #include #include -#include namespace DB { @@ -57,7 +56,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes; format_settings.csv.crlf_end_of_line = settings.output_format_csv_crlf_end_of_line; format_settings.csv.delimiter = settings.format_csv_delimiter; - format_settings.csv.empty_as_default = settings.input_format_defaults_for_omitted_fields; + format_settings.csv.empty_as_default = settings.input_format_csv_empty_as_default; format_settings.csv.input_format_enum_as_number = settings.input_format_csv_enum_as_number; format_settings.csv.null_representation = settings.output_format_csv_null_representation; format_settings.csv.unquoted_null_literal_as_null = settings.input_format_csv_unquoted_null_literal_as_null; @@ -108,10 +107,12 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.values.deduce_templates_of_expressions = settings.input_format_values_deduce_templates_of_expressions; format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions; format_settings.with_names_use_header = settings.input_format_with_names_use_header; + format_settings.with_types_use_header = settings.input_format_with_types_use_header; format_settings.write_statistics = settings.output_format_write_statistics; format_settings.arrow.low_cardinality_as_dictionary = settings.output_format_arrow_low_cardinality_as_dictionary; format_settings.arrow.import_nested = settings.input_format_arrow_import_nested; format_settings.orc.import_nested = settings.input_format_orc_import_nested; + format_settings.defaults_for_omitted_fields = settings.input_format_defaults_for_omitted_fields; /// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context if (format_settings.schema.is_server) diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 8c894c77e82..0aa34a1aa36 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -25,10 +25,12 @@ struct FormatSettings bool skip_unknown_fields = false; bool with_names_use_header = false; + bool with_types_use_header = false; bool write_statistics = true; bool import_nested_json = false; bool null_as_default = true; bool decimal_trailing_zeros = false; + bool defaults_for_omitted_fields = true; enum class DateTimeInputFormat { diff --git a/src/Formats/JSONEachRowUtils.cpp b/src/Formats/JSONEachRowUtils.cpp index b918825df79..b1eca1c6932 100644 --- a/src/Formats/JSONEachRowUtils.cpp +++ b/src/Formats/JSONEachRowUtils.cpp @@ -1,4 +1,7 @@ #include +#include +#include + #include namespace DB @@ -9,7 +12,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -std::pair fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) +template +static std::pair fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) { skipWhitespaceIfAny(in); @@ -49,19 +53,19 @@ std::pair fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, D } else { - pos = find_first_symbols<'{', '}', '\\', '"'>(pos, in.buffer().end()); + pos = find_first_symbols(pos, in.buffer().end()); if (pos > in.buffer().end()) throw Exception("Position in buffer is out of bounds. There must be a bug.", ErrorCodes::LOGICAL_ERROR); else if (pos == in.buffer().end()) continue; - else if (*pos == '{') + else if (*pos == opening_bracket) { ++balance; ++pos; } - else if (*pos == '}') + else if (*pos == closing_bracket) { --balance; ++pos; @@ -87,6 +91,16 @@ std::pair fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, D return {loadAtPosition(in, memory, pos), number_of_rows}; } +std::pair fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) +{ + return fileSegmentationEngineJSONEachRowImpl<'{', '}'>(in, memory, min_chunk_size); +} + +std::pair fileSegmentationEngineJSONCompactEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) +{ + return fileSegmentationEngineJSONEachRowImpl<'[', ']'>(in, memory, min_chunk_size); +} + bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf) { /// For JSONEachRow we can safely skip whitespace characters @@ -94,4 +108,37 @@ bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf) return buf.eof() || *buf.position() == '['; } +bool readFieldImpl(ReadBuffer & in, IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, const String & column_name, const FormatSettings & format_settings, bool yield_strings) +{ + try + { + bool as_nullable = format_settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable(); + + if (yield_strings) + { + String str; + readJSONString(str, in); + + ReadBufferFromString buf(str); + + if (as_nullable) + return SerializationNullable::deserializeWholeTextImpl(column, buf, format_settings, serialization); + + serialization->deserializeWholeText(column, buf, format_settings); + return true; + } + + if (as_nullable) + return SerializationNullable::deserializeTextJSONImpl(column, in, format_settings, serialization); + + serialization->deserializeTextJSON(column, in, format_settings); + return true; + } + catch (Exception & e) + { + e.addMessage("(while reading the value of key " + column_name + ")"); + throw; + } +} + } diff --git a/src/Formats/JSONEachRowUtils.h b/src/Formats/JSONEachRowUtils.h index 79dd6c6c192..7954da7a6c4 100644 --- a/src/Formats/JSONEachRowUtils.h +++ b/src/Formats/JSONEachRowUtils.h @@ -1,10 +1,16 @@ #pragma once +#include +#include + namespace DB { -std::pair fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size); +std::pair fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size); +std::pair fileSegmentationEngineJSONCompactEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size); bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf); +bool readFieldImpl(ReadBuffer & in, IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, const String & column_name, const FormatSettings & format_settings, bool yield_strings); + } diff --git a/src/Formats/registerFormats.cpp b/src/Formats/registerFormats.cpp index 3e4c0366e8a..6faddd3c63f 100644 --- a/src/Formats/registerFormats.cpp +++ b/src/Formats/registerFormats.cpp @@ -15,6 +15,7 @@ void registerFileSegmentationEngineCSV(FormatFactory & factory); void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory); void registerFileSegmentationEngineRegexp(FormatFactory & factory); void registerFileSegmentationEngineJSONAsString(FormatFactory & factory); +void registerFileSegmentationEngineJSONCompactEachRow(FormatFactory & factory); /// Formats for both input/output. @@ -88,6 +89,7 @@ void registerFormats() registerFileSegmentationEngineJSONEachRow(factory); registerFileSegmentationEngineRegexp(factory); registerFileSegmentationEngineJSONAsString(factory); + registerFileSegmentationEngineJSONCompactEachRow(factory); registerInputFormatNative(factory); registerOutputFormatNative(factory); diff --git a/src/Processors/Formats/IInputFormat.h b/src/Processors/Formats/IInputFormat.h index f133161c3ec..ff58a614966 100644 --- a/src/Processors/Formats/IInputFormat.h +++ b/src/Processors/Formats/IInputFormat.h @@ -16,16 +16,11 @@ struct ColumnMapping using OptionalIndexes = std::vector>; OptionalIndexes column_indexes_for_input_fields; - /// Tracks which columns we have read in a single read() call. - /// For columns that are never read, it is initialized to false when we - /// read the file header, and never changed afterwards. - /// For other columns, it is updated on each read() call. - std::vector read_columns; + /// The list of column indexes that are not presented in input data. + std::vector not_presented_columns; - - /// Whether we have any columns that are not read from file at all, - /// and must be always initialized with defaults. - bool have_always_default_columns{false}; + /// The list of column names in input data. Needed for better exception messages. + std::vector names_of_columns; }; using ColumnMappingPtr = std::shared_ptr; diff --git a/src/Processors/Formats/IRowOutputFormat.cpp b/src/Processors/Formats/IRowOutputFormat.cpp index 6b7a9a46eaa..ad111bdc66a 100644 --- a/src/Processors/Formats/IRowOutputFormat.cpp +++ b/src/Processors/Formats/IRowOutputFormat.cpp @@ -113,4 +113,11 @@ void IRowOutputFormat::writeTotals(const DB::Columns & columns, size_t row_num) write(columns, row_num); } +void registerOutputFormatWithNamesAndTypes(const String & base_format_name, RegisterOutputFormatWithNamesAndTypes register_func) +{ + register_func(base_format_name, false, false); + register_func(base_format_name + "WithNames", true, false); + register_func(base_format_name + "WithNamesAndTypes", true, true); +} + } diff --git a/src/Processors/Formats/IRowOutputFormat.h b/src/Processors/Formats/IRowOutputFormat.h index c35d93b6133..50c70a527bf 100644 --- a/src/Processors/Formats/IRowOutputFormat.h +++ b/src/Processors/Formats/IRowOutputFormat.h @@ -87,4 +87,7 @@ private: }; +using RegisterOutputFormatWithNamesAndTypes = std::function; +void registerOutputFormatWithNamesAndTypes(const String & base_format_name, RegisterOutputFormatWithNamesAndTypes register_func); + } diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 8ccc04faf35..824c33858d6 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -19,63 +19,24 @@ namespace ErrorCodes } -CSVRowInputFormat::CSVRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_, - bool with_names_, const FormatSettings & format_settings_) - : RowInputFormatWithDiagnosticInfo(header_, in_, params_) - , with_names(with_names_) - , format_settings(format_settings_) +CSVRowInputFormat::CSVRowInputFormat( + const Block & header_, + ReadBuffer & in_, + const Params & params_, + bool with_names_, + bool with_types_, + const FormatSettings & format_settings_) + : RowInputFormatWithNamesAndTypes(header_, in_, params_, with_names_, with_types_, format_settings_) { - const String bad_delimiters = " \t\"'.UL"; if (bad_delimiters.find(format_settings.csv.delimiter) != String::npos) - throw Exception(String("CSV format may not work correctly with delimiter '") + format_settings.csv.delimiter + - "'. Try use CustomSeparated format instead.", ErrorCodes::BAD_ARGUMENTS); - - const auto & sample = getPort().getHeader(); - size_t num_columns = sample.columns(); - - data_types.resize(num_columns); - column_indexes_by_names.reserve(num_columns); - - for (size_t i = 0; i < num_columns; ++i) - { - const auto & column_info = sample.getByPosition(i); - - data_types[i] = column_info.type; - column_indexes_by_names.emplace(column_info.name, i); - } -} - - -/// Map an input file column to a table column, based on its name. -void CSVRowInputFormat::addInputColumn(const String & column_name) -{ - const auto column_it = column_indexes_by_names.find(column_name); - if (column_it == column_indexes_by_names.end()) - { - if (format_settings.skip_unknown_fields) - { - column_mapping->column_indexes_for_input_fields.push_back(std::nullopt); - return; - } - throw Exception( - "Unknown field found in CSV header: '" + column_name + "' " + - "at position " + std::to_string(column_mapping->column_indexes_for_input_fields.size()) + - "\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed", - ErrorCodes::INCORRECT_DATA - ); - } - - const auto column_index = column_it->second; - - if (column_mapping->read_columns[column_index]) - throw Exception("Duplicate field found while parsing CSV header: " + column_name, ErrorCodes::INCORRECT_DATA); - - column_mapping->read_columns[column_index] = true; - column_mapping->column_indexes_for_input_fields.emplace_back(column_index); + String("CSV format may not work correctly with delimiter '") + format_settings.csv.delimiter + + "'. Try use CustomSeparated format instead.", + ErrorCodes::BAD_ARGUMENTS); } + static void skipEndOfLine(ReadBuffer & in) { /// \n (Unix) or \r\n (DOS/Windows) or \n\r (Mac OS Classic) @@ -99,29 +60,6 @@ static void skipEndOfLine(ReadBuffer & in) throw Exception("Expected end of line", ErrorCodes::INCORRECT_DATA); } - -static void skipDelimiter(ReadBuffer & in, const char delimiter, bool is_last_column) -{ - if (is_last_column) - { - if (in.eof()) - return; - - /// we support the extra delimiter at the end of the line - if (*in.position() == delimiter) - { - ++in.position(); - if (in.eof()) - return; - } - - skipEndOfLine(in); - } - else - assertChar(delimiter, in); -} - - /// Skip `whitespace` symbols allowed in CSV. static inline void skipWhitespacesAndTabs(ReadBuffer & in) { @@ -131,255 +69,138 @@ static inline void skipWhitespacesAndTabs(ReadBuffer & in) ++in.position(); } - -static void skipRow(ReadBuffer & in, const FormatSettings::CSV & settings, size_t num_columns) +void CSVRowInputFormat::skipFieldDelimiter() { - String tmp; - for (size_t i = 0; i < num_columns; ++i) - { - skipWhitespacesAndTabs(in); - readCSVString(tmp, in, settings); - skipWhitespacesAndTabs(in); - - skipDelimiter(in, settings.delimiter, i + 1 == num_columns); - } + skipWhitespacesAndTabs(*in); + assertChar(format_settings.csv.delimiter, *in); } -void CSVRowInputFormat::setupAllColumnsByTableSchema() +String CSVRowInputFormat::readFieldIntoString() { - const auto & header = getPort().getHeader(); - column_mapping->read_columns.assign(header.columns(), true); - column_mapping->column_indexes_for_input_fields.resize(header.columns()); - - for (size_t i = 0; i < column_mapping->column_indexes_for_input_fields.size(); ++i) - column_mapping->column_indexes_for_input_fields[i] = i; + skipWhitespacesAndTabs(*in); + String field; + readCSVString(field, *in, format_settings.csv); + return field; } - -void CSVRowInputFormat::readPrefix() +void CSVRowInputFormat::skipField() { - /// In this format, we assume, that if first string field contain BOM as value, it will be written in quotes, - /// so BOM at beginning of stream cannot be confused with BOM in first string value, and it is safe to skip it. - skipBOMIfExists(*in); - - size_t num_columns = data_types.size(); - const auto & header = getPort().getHeader(); - - /// This is a bit of abstraction leakage, but we have almost the same code in other places. - /// Thus, we check if this InputFormat is working with the "real" beginning of the data in case of parallel parsing. - if (with_names && getCurrentUnitNumber() == 0) - { - /// This CSV file has a header row with column names. Depending on the - /// settings, use it or skip it. - if (format_settings.with_names_use_header) - { - /// Look at the file header to see which columns we have there. - /// The missing columns are filled with defaults. - column_mapping->read_columns.assign(header.columns(), false); - do - { - String column_name; - skipWhitespacesAndTabs(*in); - readCSVString(column_name, *in, format_settings.csv); - skipWhitespacesAndTabs(*in); - - addInputColumn(column_name); - } - while (checkChar(format_settings.csv.delimiter, *in)); - - skipDelimiter(*in, format_settings.csv.delimiter, true); - - for (auto read_column : column_mapping->read_columns) - { - if (!read_column) - { - column_mapping->have_always_default_columns = true; - break; - } - } - - return; - } - else - { - skipRow(*in, format_settings.csv, num_columns); - setupAllColumnsByTableSchema(); - } - } - else if (!column_mapping->is_set) - setupAllColumnsByTableSchema(); + readFieldIntoString(); } - -bool CSVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext) +void CSVRowInputFormat::skipRowEndDelimiter() { + skipWhitespacesAndTabs(*in); + if (in->eof()) - return false; + return; - updateDiagnosticInfo(); + /// we support the extra delimiter at the end of the line + if (*in->position() == format_settings.csv.delimiter) + ++in->position(); - /// Track whether we have to fill any columns in this row with default - /// values. If not, we return an empty column mask to the caller, so that - /// it doesn't have to check it. - bool have_default_columns = column_mapping->have_always_default_columns; + skipWhitespacesAndTabs(*in); + if (in->eof()) + return; - ext.read_columns.assign(column_mapping->read_columns.size(), true); - const auto delimiter = format_settings.csv.delimiter; - for (size_t file_column = 0; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column) - { - const auto & table_column = column_mapping->column_indexes_for_input_fields[file_column]; - const bool is_last_file_column = file_column + 1 == column_mapping->column_indexes_for_input_fields.size(); - - if (table_column) - { - skipWhitespacesAndTabs(*in); - ext.read_columns[*table_column] = readField(*columns[*table_column], data_types[*table_column], - serializations[*table_column], is_last_file_column); - - if (!ext.read_columns[*table_column]) - have_default_columns = true; - skipWhitespacesAndTabs(*in); - } - else - { - /// We never read this column from the file, just skip it. - String tmp; - readCSVString(tmp, *in, format_settings.csv); - } - - skipDelimiter(*in, delimiter, is_last_file_column); - } - - if (have_default_columns) - { - for (size_t i = 0; i < column_mapping->read_columns.size(); i++) - { - if (!column_mapping->read_columns[i]) - { - /// The column value for this row is going to be overwritten - /// with default by the caller, but the general assumption is - /// that the column size increases for each row, so we have - /// to insert something. Since we do not care about the exact - /// value, we do not have to use the default value specified by - /// the data type, and can just use IColumn::insertDefault(). - columns[i]->insertDefault(); - ext.read_columns[i] = false; - } - } - } - - return true; + skipEndOfLine(*in); } -bool CSVRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) +void CSVRowInputFormat::skipRow() +{ + do + { + skipField(); + skipWhitespacesAndTabs(*in); + } + while (checkChar(format_settings.csv.delimiter, *in)); + + skipRowEndDelimiter(); +} + +std::vector CSVRowInputFormat::readHeaderRow() +{ + std::vector fields; + do + { + fields.push_back(readFieldIntoString()); + skipWhitespacesAndTabs(*in); + } + while (checkChar(format_settings.csv.delimiter, *in)); + + skipRowEndDelimiter(); + return fields; +} + +bool CSVRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) { const char delimiter = format_settings.csv.delimiter; - for (size_t file_column = 0; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column) + try { - if (file_column == 0 && in->eof()) - { - out << "\n"; - return false; - } - skipWhitespacesAndTabs(*in); - if (column_mapping->column_indexes_for_input_fields[file_column].has_value()) + assertChar(delimiter, *in); + } + catch (const DB::Exception &) + { + if (*in->position() == '\n' || *in->position() == '\r') { - const auto & header = getPort().getHeader(); - size_t col_idx = column_mapping->column_indexes_for_input_fields[file_column].value(); - if (!deserializeFieldAndPrintDiagnosticInfo(header.getByPosition(col_idx).name, data_types[col_idx], *columns[col_idx], - out, file_column)) - return false; + out << "ERROR: Line feed found where delimiter (" << delimiter << ") is expected." + " It's like your file has less columns than expected.\n" + "And if your file has the right number of columns, maybe it has unescaped quotes in values.\n"; } else { - static const String skipped_column_str = ""; - static const DataTypePtr skipped_column_type = std::make_shared(); - static const MutableColumnPtr skipped_column = skipped_column_type->createColumn(); - if (!deserializeFieldAndPrintDiagnosticInfo(skipped_column_str, skipped_column_type, *skipped_column, out, file_column)) - return false; - } - skipWhitespacesAndTabs(*in); - - /// Delimiters - if (file_column + 1 == column_mapping->column_indexes_for_input_fields.size()) - { - if (in->eof()) - return false; - - /// we support the extra delimiter at the end of the line - if (*in->position() == delimiter) - { - ++in->position(); - if (in->eof()) - break; - } - - if (!in->eof() && *in->position() != '\n' && *in->position() != '\r') - { - out << "ERROR: There is no line feed. "; - verbosePrintString(in->position(), in->position() + 1, out); - out << " found instead.\n" - " It's like your file has more columns than expected.\n" - "And if your file has the right number of columns, maybe it has an unquoted string value with a comma.\n"; - - return false; - } - - skipEndOfLine(*in); - } - else - { - try - { - assertChar(delimiter, *in); - } - catch (const DB::Exception &) - { - if (*in->position() == '\n' || *in->position() == '\r') - { - out << "ERROR: Line feed found where delimiter (" << delimiter << ") is expected." - " It's like your file has less columns than expected.\n" - "And if your file has the right number of columns, maybe it has unescaped quotes in values.\n"; - } - else - { - out << "ERROR: There is no delimiter (" << delimiter << "). "; - verbosePrintString(in->position(), in->position() + 1, out); - out << " found instead.\n"; - } - return false; - } + out << "ERROR: There is no delimiter (" << delimiter << "). "; + verbosePrintString(in->position(), in->position() + 1, out); + out << " found instead.\n"; } + return false; } return true; } +bool CSVRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out) +{ + skipWhitespacesAndTabs(*in); + + if (in->eof()) + return true; + + /// we support the extra delimiter at the end of the line + if (*in->position() == format_settings.csv.delimiter) + { + ++in->position(); + skipWhitespacesAndTabs(*in); + if (in->eof()) + return true; + } + + if (!in->eof() && *in->position() != '\n' && *in->position() != '\r') + { + out << "ERROR: There is no line feed. "; + verbosePrintString(in->position(), in->position() + 1, out); + out << " found instead.\n" + " It's like your file has more columns than expected.\n" + "And if your file has the right number of columns, maybe it has an unquoted string value with a comma.\n"; + + return false; + } + + skipEndOfLine(*in); + return true; +} void CSVRowInputFormat::syncAfterError() { skipToNextLineOrEOF(*in); } -void CSVRowInputFormat::tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) +bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & /*column_name*/) { - const auto & index = column_mapping->column_indexes_for_input_fields[file_column]; - if (index) - { - const bool is_last_file_column = file_column + 1 == column_mapping->column_indexes_for_input_fields.size(); - readField(column, type, serializations[*index], is_last_file_column); - } - else - { - String tmp; - readCSVString(tmp, *in, format_settings.csv); - } -} + skipWhitespacesAndTabs(*in); -bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column) -{ const bool at_delimiter = !in->eof() && *in->position() == format_settings.csv.delimiter; const bool at_last_column_line_end = is_last_file_column && (in->eof() || *in->position() == '\n' || *in->position() == '\r'); @@ -398,7 +219,7 @@ bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, co column.insertDefault(); return false; } - else if (format_settings.null_as_default && !type->isNullable()) + else if (format_settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable()) { /// If value is null but type is not nullable then use default value instead. return SerializationNullable::deserializeTextCSVImpl(column, *in, format_settings, serialization); @@ -411,28 +232,23 @@ bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, co } } -void CSVRowInputFormat::resetParser() -{ - RowInputFormatWithDiagnosticInfo::resetParser(); - column_mapping->column_indexes_for_input_fields.clear(); - column_mapping->read_columns.clear(); - column_mapping->have_always_default_columns = false; -} void registerInputFormatCSV(FormatFactory & factory) { - for (bool with_names : {false, true}) + auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - factory.registerInputFormat(with_names ? "CSVWithNames" : "CSV", [=]( + factory.registerInputFormat(format_name, [with_names, with_types]( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, const FormatSettings & settings) { - return std::make_shared(sample, buf, params, with_names, settings); + return std::make_shared(sample, buf, std::move(params), with_names, with_types, settings); }); - } + }; + + registerInputFormatWithNamesAndTypes("CSV", register_func); } static std::pair fileSegmentationEngineCSVImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) @@ -501,8 +317,7 @@ static std::pair fileSegmentationEngineCSVImpl(ReadBuffer & in, DB void registerFileSegmentationEngineCSV(FormatFactory & factory) { - factory.registerFileSegmentationEngine("CSV", &fileSegmentationEngineCSVImpl); - factory.registerFileSegmentationEngine("CSVWithNames", &fileSegmentationEngineCSVImpl); + registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "CSV", &fileSegmentationEngineCSVImpl); } } diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index b6075745b39..2e036fa2318 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -4,7 +4,7 @@ #include #include -#include +#include #include @@ -14,41 +14,39 @@ namespace DB /** A stream for inputting data in csv format. * Does not conform with https://tools.ietf.org/html/rfc4180 because it skips spaces and tabs between values. */ -class CSVRowInputFormat : public RowInputFormatWithDiagnosticInfo +class CSVRowInputFormat : public RowInputFormatWithNamesAndTypes { public: /** with_names - in the first line the header with column names + * with_types - on the next line header with type names */ CSVRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_, - bool with_names_, const FormatSettings & format_settings_); + bool with_names_, bool with_types_, const FormatSettings & format_settings_); String getName() const override { return "CSVRowInputFormat"; } - bool readRow(MutableColumns & columns, RowReadExtension & ext) override; - void readPrefix() override; bool allowSyncAfterError() const override { return true; } void syncAfterError() override; - void resetParser() override; private: - /// There fields are computed in constructor. - bool with_names; - const FormatSettings format_settings; - DataTypes data_types; - using IndexesMap = std::unordered_map; - IndexesMap column_indexes_by_names; - - void addInputColumn(const String & column_name); - - void setupAllColumnsByTableSchema(); - bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override; - void tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) override; + bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override; + bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) override; bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override { return *pos != '\n' && *pos != '\r' && *pos != format_settings.csv.delimiter && *pos != ' ' && *pos != '\t'; } - bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column); + bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override; + + void skipField(const String & /*column_name*/) override { skipField(); } + void skipField(); + + void skipRow() override; + void skipFieldDelimiter() override; + void skipRowEndDelimiter() override; + + Names readHeaderRow() override; + String readFieldIntoString(); }; } diff --git a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp index 14d0e519c0c..9fba7ba3627 100644 --- a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp @@ -8,8 +8,8 @@ namespace DB { -CSVRowOutputFormat::CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_, params_), with_names(with_names_), format_settings(format_settings_) +CSVRowOutputFormat::CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, bool with_types_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_) + : IRowOutputFormat(header_, out_, params_), with_names(with_names_), with_types(with_types_), format_settings(format_settings_) { const auto & sample = getPort(PortKind::Main).getHeader(); size_t columns = sample.columns(); @@ -18,25 +18,27 @@ CSVRowOutputFormat::CSVRowOutputFormat(WriteBuffer & out_, const Block & header_ data_types[i] = sample.safeGetByPosition(i).type; } +void CSVRowOutputFormat::writeLine(const std::vector & values) +{ + for (size_t i = 0; i < values.size(); ++i) + { + writeCSVString(values[i], out); + if (i + 1 == values.size()) + writeRowEndDelimiter(); + else + writeFieldDelimiter(); + } +} void CSVRowOutputFormat::doWritePrefix() { const auto & sample = getPort(PortKind::Main).getHeader(); - size_t columns = sample.columns(); if (with_names) - { - for (size_t i = 0; i < columns; ++i) - { - writeCSVString(sample.safeGetByPosition(i).name, out); + writeLine(sample.getNames()); - char delimiter = format_settings.csv.delimiter; - if (i + 1 == columns) - delimiter = '\n'; - - writeChar(delimiter, out); - } - } + if (with_types) + writeLine(sample.getDataTypeNames()); } @@ -72,18 +74,20 @@ void CSVRowOutputFormat::writeBeforeExtremes() void registerOutputFormatCSV(FormatFactory & factory) { - for (bool with_names : {false, true}) + auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - factory.registerOutputFormat(with_names ? "CSVWithNames" : "CSV", [=]( - WriteBuffer & buf, - const Block & sample, - const RowOutputFormatParams & params, - const FormatSettings & format_settings) + factory.registerOutputFormat(format_name, [=]( + WriteBuffer & buf, + const Block & sample, + const RowOutputFormatParams & params, + const FormatSettings & format_settings) { - return std::make_shared(buf, sample, with_names, params, format_settings); + return std::make_shared(buf, sample, with_names, with_types, params, format_settings); }); - factory.markOutputFormatSupportsParallelFormatting(with_names ? "CSVWithNames" : "CSV"); - } + factory.markOutputFormatSupportsParallelFormatting(format_name); + }; + + registerOutputFormatWithNamesAndTypes("CSV", register_func); } } diff --git a/src/Processors/Formats/Impl/CSVRowOutputFormat.h b/src/Processors/Formats/Impl/CSVRowOutputFormat.h index 780a6c4d3ce..7f5d90203ea 100644 --- a/src/Processors/Formats/Impl/CSVRowOutputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowOutputFormat.h @@ -20,7 +20,7 @@ public: /** with_names - output in the first line a header with column names * with_types - output in the next line header with the names of the types */ - CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_); + CSVRowOutputFormat(WriteBuffer & out_, const Block & header_, bool with_names_, bool with_types, const RowOutputFormatParams & params_, const FormatSettings & format_settings_); String getName() const override { return "CSVRowOutputFormat"; } @@ -38,9 +38,11 @@ public: return String("text/csv; charset=UTF-8; header=") + (with_names ? "present" : "absent"); } -protected: +private: + void writeLine(const std::vector & values); bool with_names; + bool with_types; const FormatSettings format_settings; DataTypes data_types; }; diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp index 5b32bf94c4d..a5e0ac6862c 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp @@ -185,7 +185,7 @@ void registerInputFormatJSONAsString(FormatFactory & factory) void registerFileSegmentationEngineJSONAsString(FormatFactory & factory) { - factory.registerFileSegmentationEngine("JSONAsString", &fileSegmentationEngineJSONEachRowImpl); + factory.registerFileSegmentationEngine("JSONAsString", &fileSegmentationEngineJSONEachRow); } void registerNonTrivialPrefixAndSuffixCheckerJSONAsString(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index 962e9d6e5ac..c551597ca5f 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -1,8 +1,11 @@ +#include + #include #include - -#include +#include #include +#include +#include #include #include @@ -16,179 +19,36 @@ namespace ErrorCodes } -JSONCompactEachRowRowInputFormat::JSONCompactEachRowRowInputFormat(ReadBuffer & in_, - const Block & header_, - Params params_, - const FormatSettings & format_settings_, - bool with_names_, - bool yield_strings_) - : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), with_names(with_names_), yield_strings(yield_strings_) +JSONCompactEachRowRowInputFormat::JSONCompactEachRowRowInputFormat( + const Block & header_, + ReadBuffer & in_, + Params params_, + bool with_names_, + bool with_types_, + bool yield_strings_, + const FormatSettings & format_settings_) + : RowInputFormatWithNamesAndTypes(header_, in_, std::move(params_), with_names_, with_types_, format_settings_) + , yield_strings(yield_strings_) { - const auto & sample = getPort().getHeader(); - size_t num_columns = sample.columns(); - - data_types.resize(num_columns); - column_indexes_by_names.reserve(num_columns); - - for (size_t i = 0; i < num_columns; ++i) - { - const auto & column_info = sample.getByPosition(i); - - data_types[i] = column_info.type; - column_indexes_by_names.emplace(column_info.name, i); - } } -void JSONCompactEachRowRowInputFormat::resetParser() +void JSONCompactEachRowRowInputFormat::skipRowStartDelimiter() { - IRowInputFormat::resetParser(); - column_indexes_for_input_fields.clear(); - not_seen_columns.clear(); -} - -void JSONCompactEachRowRowInputFormat::readPrefix() -{ - /// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it. - skipBOMIfExists(*in); - - if (with_names) - { - size_t num_columns = getPort().getHeader().columns(); - read_columns.assign(num_columns, false); - - assertChar('[', *in); - do - { - skipWhitespaceIfAny(*in); - String column_name; - readJSONString(column_name, *in); - addInputColumn(column_name); - skipWhitespaceIfAny(*in); - } - while (checkChar(',', *in)); - assertChar(']', *in); - skipEndOfLine(); - - /// Type checking - assertChar('[', *in); - for (size_t i = 0; i < column_indexes_for_input_fields.size(); ++i) - { - skipWhitespaceIfAny(*in); - String data_type; - readJSONString(data_type, *in); - - if (column_indexes_for_input_fields[i] && - data_types[*column_indexes_for_input_fields[i]]->getName() != data_type) - { - throw Exception( - "Type of '" + getPort().getHeader().getByPosition(*column_indexes_for_input_fields[i]).name - + "' must be " + data_types[*column_indexes_for_input_fields[i]]->getName() + - ", not " + data_type, - ErrorCodes::INCORRECT_DATA - ); - } - - if (i != column_indexes_for_input_fields.size() - 1) - assertChar(',', *in); - skipWhitespaceIfAny(*in); - } - assertChar(']', *in); - } - else - { - size_t num_columns = getPort().getHeader().columns(); - read_columns.assign(num_columns, true); - column_indexes_for_input_fields.resize(num_columns); - - for (size_t i = 0; i < num_columns; ++i) - { - column_indexes_for_input_fields[i] = i; - } - } - - for (size_t i = 0; i < read_columns.size(); ++i) - { - if (!read_columns[i]) - { - not_seen_columns.emplace_back(i); - } - } -} - -void JSONCompactEachRowRowInputFormat::addInputColumn(const String & column_name) -{ - names_of_columns.emplace_back(column_name); - - const auto column_it = column_indexes_by_names.find(column_name); - if (column_it == column_indexes_by_names.end()) - { - if (format_settings.skip_unknown_fields) - { - column_indexes_for_input_fields.push_back(std::nullopt); - return; - } - - throw Exception( - "Unknown field found in JSONCompactEachRow header: '" + column_name + "' " + - "at position " + std::to_string(column_indexes_for_input_fields.size()) + - "\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed", - ErrorCodes::INCORRECT_DATA - ); - } - - const auto column_index = column_it->second; - - if (read_columns[column_index]) - throw Exception("Duplicate field found while parsing JSONCompactEachRow header: " + column_name, ErrorCodes::INCORRECT_DATA); - - read_columns[column_index] = true; - column_indexes_for_input_fields.emplace_back(column_index); -} - -bool JSONCompactEachRowRowInputFormat::readRow(DB::MutableColumns &columns, DB::RowReadExtension &ext) -{ - skipEndOfLine(); - - if (in->eof()) - return false; - - size_t num_columns = columns.size(); - - read_columns.assign(num_columns, false); - + skipWhitespaceIfAny(*in); assertChar('[', *in); - for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column) - { - const auto & table_column = column_indexes_for_input_fields[file_column]; - if (table_column) - { - readField(*table_column, columns); - } - else - { - skipJSONField(*in, StringRef(names_of_columns[file_column])); - } +} - skipWhitespaceIfAny(*in); - if (in->eof()) - throw ParsingException("Unexpected end of stream while parsing JSONCompactEachRow format", ErrorCodes::CANNOT_READ_ALL_DATA); - if (file_column + 1 != column_indexes_for_input_fields.size()) - { - assertChar(',', *in); - skipWhitespaceIfAny(*in); - } - } +void JSONCompactEachRowRowInputFormat::skipFieldDelimiter() +{ + skipWhitespaceIfAny(*in); + assertChar(',', *in); +} + +void JSONCompactEachRowRowInputFormat::skipRowEndDelimiter() +{ + skipWhitespaceIfAny(*in); assertChar(']', *in); - for (const auto & name : not_seen_columns) - columns[name]->insertDefault(); - - ext.read_columns = read_columns; - return true; -} - -void JSONCompactEachRowRowInputFormat::skipEndOfLine() -{ skipWhitespaceIfAny(*in); if (!in->eof() && (*in->position() == ',' || *in->position() == ';')) ++in->position(); @@ -196,39 +56,55 @@ void JSONCompactEachRowRowInputFormat::skipEndOfLine() skipWhitespaceIfAny(*in); } -void JSONCompactEachRowRowInputFormat::readField(size_t index, MutableColumns & columns) +String JSONCompactEachRowRowInputFormat::readFieldIntoString() { - try + skipWhitespaceIfAny(*in); + String field; + readJSONString(field, *in); + return field; +} + +void JSONCompactEachRowRowInputFormat::skipField(const String & column_name) +{ + skipWhitespaceIfAny(*in); + skipJSONField(*in, column_name); +} + +void JSONCompactEachRowRowInputFormat::skipRow() +{ + skipRowStartDelimiter(); + size_t i = 0; + do { - read_columns[index] = true; - const auto & type = data_types[index]; - const auto & serialization = serializations[index]; - - if (yield_strings) - { - String str; - readJSONString(str, *in); - - ReadBufferFromString buf(str); - - if (format_settings.null_as_default && !type->isNullable()) - read_columns[index] = SerializationNullable::deserializeWholeTextImpl(*columns[index], buf, format_settings, serialization); - else - serialization->deserializeWholeText(*columns[index], buf, format_settings); - } - else - { - if (format_settings.null_as_default && !type->isNullable()) - read_columns[index] = SerializationNullable::deserializeTextJSONImpl(*columns[index], *in, format_settings, serialization); - else - serialization->deserializeTextJSON(*columns[index], *in, format_settings); - } + if (i >= column_mapping->names_of_columns.size()) + throw Exception(ErrorCodes::INCORRECT_DATA, "The number of columns in a row differs from the number of column names"); + skipField(column_mapping->names_of_columns[i++]); + skipWhitespaceIfAny(*in); } - catch (Exception & e) + while (checkChar(',', *in)); + + skipRowEndDelimiter(); +} + +std::vector JSONCompactEachRowRowInputFormat::readHeaderRow() +{ + skipRowStartDelimiter(); + std::vector fields; + do { - e.addMessage("(while reading the value of key " + getPort().getHeader().getByPosition(index).name + ")"); - throw; + fields.push_back(readFieldIntoString()); + skipWhitespaceIfAny(*in); } + while (checkChar(',', *in)); + + skipRowEndDelimiter(); + return fields; +} + +bool JSONCompactEachRowRowInputFormat::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool /*is_last_file_column*/, const String & column_name) +{ + skipWhitespaceIfAny(*in); + return readFieldImpl(*in, column, type, serialization, column_name, format_settings, yield_strings); } void JSONCompactEachRowRowInputFormat::syncAfterError() @@ -236,43 +112,99 @@ void JSONCompactEachRowRowInputFormat::syncAfterError() skipToUnescapedNextLineOrEOF(*in); } +bool JSONCompactEachRowRowInputFormat::parseRowStartWithDiagnosticInfo(WriteBuffer & out) +{ + skipWhitespaceIfAny(*in); + if (!checkChar('[', *in)) + { + out << "ERROR: There is no '[' before the row.\n"; + return false; + } + + return true; +} + +bool JSONCompactEachRowRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) +{ + try + { + skipWhitespaceIfAny(*in); + assertChar(',', *in); + } + catch (const DB::Exception &) + { + if (*in->position() == ']') + { + out << "ERROR: Closing parenthesis (']') found where comma is expected." + " It's like your file has less columns than expected.\n" + "And if your file has the right number of columns, maybe it has unescaped quotes in values.\n"; + } + else + { + out << "ERROR: There is no comma. "; + verbosePrintString(in->position(), in->position() + 1, out); + out << " found instead.\n"; + } + return false; + } + + return true; +} + +bool JSONCompactEachRowRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out) +{ + skipWhitespaceIfAny(*in); + + if (in->eof()) + { + out << "ERROR: Unexpected end of file. ']' expected at the end of row."; + return false; + } + + if (!checkChar(']', *in)) + { + out << "ERROR: There is no closing parenthesis (']') at the end of the row. "; + verbosePrintString(in->position(), in->position() + 1, out); + out << " found instead.\n"; + return false; + } + + skipWhitespaceIfAny(*in); + + if (in->eof()) + return true; + + if ((*in->position() == ',' || *in->position() == ';')) + ++in->position(); + + skipWhitespaceIfAny(*in); + return true; +} + void registerInputFormatJSONCompactEachRow(FormatFactory & factory) { - factory.registerInputFormat("JSONCompactEachRow", []( - ReadBuffer & buf, - const Block & sample, - IRowInputFormat::Params params, - const FormatSettings & settings) + for (bool yield_strings : {true, false}) { - return std::make_shared(buf, sample, std::move(params), settings, false, false); - }); + auto register_func = [&](const String & format_name, bool with_names, bool with_types) + { + factory.registerInputFormat(format_name, [with_names, with_types, yield_strings]( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(sample, buf, std::move(params), with_names, with_types, yield_strings, settings); + }); + }; - factory.registerInputFormat("JSONCompactEachRowWithNamesAndTypes", []( - ReadBuffer & buf, - const Block & sample, - IRowInputFormat::Params params, - const FormatSettings & settings) - { - return std::make_shared(buf, sample, std::move(params), settings, true, false); - }); + registerInputFormatWithNamesAndTypes(yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", register_func); + } +} - factory.registerInputFormat("JSONCompactStringsEachRow", []( - ReadBuffer & buf, - const Block & sample, - IRowInputFormat::Params params, - const FormatSettings & settings) - { - return std::make_shared(buf, sample, std::move(params), settings, false, true); - }); - - factory.registerInputFormat("JSONCompactStringsEachRowWithNamesAndTypes", []( - ReadBuffer & buf, - const Block & sample, - IRowInputFormat::Params params, - const FormatSettings & settings) - { - return std::make_shared(buf, sample, std::move(params), settings, true, true); - }); +void registerFileSegmentationEngineJSONCompactEachRow(FormatFactory & factory) +{ + registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "JSONCompactEachRow", &fileSegmentationEngineJSONCompactEachRow); + registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "JSONCompactStringsEachRow", &fileSegmentationEngineJSONCompactEachRow); } } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h index 4077eb6e008..fe8fc2acda3 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include @@ -17,49 +17,43 @@ class ReadBuffer; * - JSONCompactStringsEachRowWithNamesAndTypes * */ -class JSONCompactEachRowRowInputFormat : public IRowInputFormat +class JSONCompactEachRowRowInputFormat : public RowInputFormatWithNamesAndTypes { public: JSONCompactEachRowRowInputFormat( - ReadBuffer & in_, const Block & header_, + ReadBuffer & in_, Params params_, - const FormatSettings & format_settings_, bool with_names_, - bool yield_strings_); + bool with_types_, + bool yield_strings_, + const FormatSettings & format_settings_); String getName() const override { return "JSONCompactEachRowRowInputFormat"; } - - void readPrefix() override; - bool readRow(MutableColumns & columns, RowReadExtension & ext) override; bool allowSyncAfterError() const override { return true; } void syncAfterError() override; - void resetParser() override; private: - void addInputColumn(const String & column_name); - void skipEndOfLine(); - void readField(size_t index, MutableColumns & columns); + bool parseRowStartWithDiagnosticInfo(WriteBuffer & out) override; + bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override; + bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) override; + bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override + { + return *pos != ',' && *pos != ']' && *pos != ' ' && *pos != '\t'; + } - const FormatSettings format_settings; + bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override; - using IndexesMap = std::unordered_map; - IndexesMap column_indexes_by_names; + void skipField(const String & column_name) override; + void skipRow() override; + void skipRowStartDelimiter() override; + void skipFieldDelimiter() override; + void skipRowEndDelimiter() override; - using OptionalIndexes = std::vector>; - OptionalIndexes column_indexes_for_input_fields; + Names readHeaderRow() override; + String readFieldIntoString(); - DataTypes data_types; - std::vector read_columns; - std::vector not_seen_columns; - - /// This is for the correct exceptions in skipping unknown fields. - std::vector names_of_columns; - - /// For *WithNamesAndTypes formats. - bool with_names; - /// For JSONCompactString* formats. bool yield_strings; }; diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp index 1ce4277023d..c7df76e3b83 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp @@ -13,12 +13,10 @@ JSONCompactEachRowRowOutputFormat::JSONCompactEachRowRowOutputFormat(WriteBuffer const RowOutputFormatParams & params_, const FormatSettings & settings_, bool with_names_, + bool with_types_, bool yield_strings_) - : IRowOutputFormat(header_, out_, params_), settings(settings_), with_names(with_names_), yield_strings(yield_strings_) + : IRowOutputFormat(header_, out_, params_), settings(settings_), with_names(with_names_), with_types(with_types_), yield_strings(yield_strings_) { - const auto & sample = getPort(PortKind::Main).getHeader(); - NamesAndTypesList columns(sample.getNamesAndTypesList()); - fields.assign(columns.begin(), columns.end()); } @@ -57,39 +55,40 @@ void JSONCompactEachRowRowOutputFormat::writeTotals(const Columns & columns, siz { writeChar('\n', out); size_t num_columns = columns.size(); - writeChar('[', out); + writeRowStartDelimiter(); for (size_t i = 0; i < num_columns; ++i) { if (i != 0) - JSONCompactEachRowRowOutputFormat::writeFieldDelimiter(); + writeFieldDelimiter(); - JSONCompactEachRowRowOutputFormat::writeField(*columns[i], *serializations[i], row_num); + writeField(*columns[i], *serializations[i], row_num); } - writeCString("]\n", out); + writeRowEndDelimiter(); +} + +void JSONCompactEachRowRowOutputFormat::writeLine(const std::vector & values) +{ + writeRowStartDelimiter(); + for (size_t i = 0; i < values.size(); ++i) + { + writeChar('\"', out); + writeString(values[i], out); + writeChar('\"', out); + if (i != values.size() - 1) + writeFieldDelimiter(); + } + writeRowEndDelimiter(); } void JSONCompactEachRowRowOutputFormat::doWritePrefix() { + const auto & header = getPort(PortKind::Main).getHeader(); + if (with_names) - { - writeChar('[', out); - for (size_t i = 0; i < fields.size(); ++i) - { - writeChar('\"', out); - writeString(fields[i].name, out); - writeChar('\"', out); - if (i != fields.size() - 1) - writeCString(", ", out); - } - writeCString("]\n[", out); - for (size_t i = 0; i < fields.size(); ++i) - { - writeJSONString(fields[i].type->getName(), out, settings); - if (i != fields.size() - 1) - writeCString(", ", out); - } - writeCString("]\n", out); - } + writeLine(header.getNames()); + + if (with_types) + writeLine(header.getDataTypeNames()); } void JSONCompactEachRowRowOutputFormat::consumeTotals(DB::Chunk chunk) @@ -100,45 +99,23 @@ void JSONCompactEachRowRowOutputFormat::consumeTotals(DB::Chunk chunk) void registerOutputFormatJSONCompactEachRow(FormatFactory & factory) { - factory.registerOutputFormat("JSONCompactEachRow", []( - WriteBuffer & buf, - const Block & sample, - const RowOutputFormatParams & params, - const FormatSettings & format_settings) + for (bool yield_strings : {false, true}) { - return std::make_shared(buf, sample, params, format_settings, false, false); - }); - factory.markOutputFormatSupportsParallelFormatting("JSONCompactEachRow"); + auto register_func = [&](const String & format_name, bool with_names, bool with_types) + { + factory.registerOutputFormat(format_name, [=]( + WriteBuffer & buf, + const Block & sample, + const RowOutputFormatParams & params, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, params, format_settings, with_names, with_types, yield_strings); + }); + factory.markOutputFormatSupportsParallelFormatting(format_name); + }; - factory.registerOutputFormat("JSONCompactEachRowWithNamesAndTypes", []( - WriteBuffer &buf, - const Block &sample, - const RowOutputFormatParams & params, - const FormatSettings &format_settings) - { - return std::make_shared(buf, sample, params, format_settings, true, false); - }); - factory.markOutputFormatSupportsParallelFormatting("JSONCompactEachRowWithNamesAndTypes"); - - factory.registerOutputFormat("JSONCompactStringsEachRow", []( - WriteBuffer & buf, - const Block & sample, - const RowOutputFormatParams & params, - const FormatSettings & format_settings) - { - return std::make_shared(buf, sample, params, format_settings, false, true); - }); - factory.markOutputFormatSupportsParallelFormatting("JSONCompactStringsEachRow"); - - factory.registerOutputFormat("JSONCompactStringsEachRowWithNamesAndTypes", []( - WriteBuffer &buf, - const Block &sample, - const RowOutputFormatParams & params, - const FormatSettings &format_settings) - { - return std::make_shared(buf, sample, params, format_settings, true, true); - }); - factory.markOutputFormatSupportsParallelFormatting("JSONCompactStringsEachRowWithNamesAndTypes"); + registerOutputFormatWithNamesAndTypes(yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", register_func); + } } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h index 792eb906f4b..aa12ba7e809 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h @@ -21,15 +21,14 @@ public: const RowOutputFormatParams & params_, const FormatSettings & settings_, bool with_names_, + bool with_types_, bool yield_strings_); String getName() const override { return "JSONCompactEachRowRowOutputFormat"; } void doWritePrefix() override; - void writeBeforeTotals() override {} void writeTotals(const Columns & columns, size_t row_num) override; - void writeAfterTotals() override {} void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; void writeFieldDelimiter() override; @@ -42,11 +41,11 @@ protected: void consumeExtremes(Chunk) override {} private: + void writeLine(const std::vector & values); + FormatSettings settings; - - NamesAndTypes fields; - bool with_names; + bool with_types; bool yield_strings; }; } diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index d04ba2a49e4..28481313974 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -136,37 +136,10 @@ void JSONEachRowRowInputFormat::readField(size_t index, MutableColumns & columns if (seen_columns[index]) throw Exception("Duplicate field found while parsing JSONEachRow format: " + columnName(index), ErrorCodes::INCORRECT_DATA); - try - { - seen_columns[index] = read_columns[index] = true; - const auto & type = getPort().getHeader().getByPosition(index).type; - const auto & serialization = serializations[index]; - - if (yield_strings) - { - String str; - readJSONString(str, *in); - - ReadBufferFromString buf(str); - - if (format_settings.null_as_default && !type->isNullable()) - read_columns[index] = SerializationNullable::deserializeWholeTextImpl(*columns[index], buf, format_settings, serialization); - else - serialization->deserializeWholeText(*columns[index], buf, format_settings); - } - else - { - if (format_settings.null_as_default && !type->isNullable()) - read_columns[index] = SerializationNullable::deserializeTextJSONImpl(*columns[index], *in, format_settings, serialization); - else - serialization->deserializeTextJSON(*columns[index], *in, format_settings); - } - } - catch (Exception & e) - { - e.addMessage("(while reading the value of key " + columnName(index) + ")"); - throw; - } + seen_columns[index] = true; + const auto & type = getPort().getHeader().getByPosition(index).type; + const auto & serialization = serializations[index]; + read_columns[index] = readFieldImpl(*in, *columns[index], type, serialization, columnName(index), format_settings, yield_strings); } inline bool JSONEachRowRowInputFormat::advanceToNextKey(size_t key_index) @@ -282,8 +255,13 @@ bool JSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtensi if (!seen_columns[i]) header.getByPosition(i).type->insertDefaultInto(*columns[i]); - /// return info about defaults set - ext.read_columns = read_columns; + /// Return info about defaults set. + /// If defaults_for_omitted_fields is set to 0, we should just leave already inserted defaults. + if (format_settings.defaults_for_omitted_fields) + ext.read_columns = read_columns; + else + ext.read_columns.assign(read_columns.size(), true); + return true; } @@ -355,8 +333,8 @@ void registerInputFormatJSONEachRow(FormatFactory & factory) void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory) { - factory.registerFileSegmentationEngine("JSONEachRow", &fileSegmentationEngineJSONEachRowImpl); - factory.registerFileSegmentationEngine("JSONStringsEachRow", &fileSegmentationEngineJSONEachRowImpl); + factory.registerFileSegmentationEngine("JSONEachRow", &fileSegmentationEngineJSONEachRow); + factory.registerFileSegmentationEngine("JSONStringsEachRow", &fileSegmentationEngineJSONEachRow); } void registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index 8cb0fce609e..62c0eaa457e 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -60,7 +60,7 @@ RegexpRowInputFormat::ColumnFormat RegexpRowInputFormat::stringToFormat(const St bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns) { const auto & type = getPort().getHeader().getByPosition(index).type; - bool parse_as_nullable = format_settings.null_as_default && !type->isNullable(); + bool parse_as_nullable = format_settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable(); bool read = true; ReadBuffer field_buf(const_cast(matched_fields[index].data()), matched_fields[index].size(), 0); try @@ -94,9 +94,9 @@ bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns) break; case ColumnFormat::Raw: if (parse_as_nullable) - read = SerializationNullable::deserializeWholeTextImpl(*columns[index], field_buf, format_settings, serialization); + read = SerializationNullable::deserializeTextRawImpl(*columns[index], field_buf, format_settings, serialization); else - serialization->deserializeWholeText(*columns[index], field_buf, format_settings); + serialization->deserializeTextRaw(*columns[index], field_buf, format_settings); break; default: break; diff --git a/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp b/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp index 331d6e435d1..606c67aa0d1 100644 --- a/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp @@ -143,7 +143,7 @@ bool TSKVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ex seen_columns[index] = read_columns[index] = true; const auto & type = getPort().getHeader().getByPosition(index).type; const auto & serialization = serializations[index]; - if (format_settings.null_as_default && !type->isNullable()) + if (format_settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable()) read_columns[index] = SerializationNullable::deserializeTextEscapedImpl(*columns[index], *in, format_settings, serialization); else serialization->deserializeTextEscaped(*columns[index], *in, format_settings); diff --git a/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp b/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp index 6161303d23a..14dec8420a8 100644 --- a/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TSKVRowOutputFormat.cpp @@ -8,7 +8,7 @@ namespace DB { TSKVRowOutputFormat::TSKVRowOutputFormat(WriteBuffer & out_, const Block & header, const RowOutputFormatParams & params_, const FormatSettings & format_settings_) - : TabSeparatedRowOutputFormat(out_, header, false, false, params_, format_settings_) + : TabSeparatedRowOutputFormat(out_, header, false, false, false, params_, format_settings_) { const auto & sample = getPort(PortKind::Main).getHeader(); NamesAndTypesList columns(sample.getNamesAndTypesList()); diff --git a/src/Processors/Formats/Impl/TabSeparatedRawRowInputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRawRowInputFormat.h deleted file mode 100644 index 3e12388bede..00000000000 --- a/src/Processors/Formats/Impl/TabSeparatedRawRowInputFormat.h +++ /dev/null @@ -1,58 +0,0 @@ -#pragma once - -#include -#include -#include -#include - - -namespace DB -{ - -/** A stream to input data in tsv format, but without escaping individual values. - * It only supports columns without '\n' or '\t' - */ -class TabSeparatedRawRowInputFormat : public TabSeparatedRowInputFormat -{ -public: - /** with_names - the first line is the header with the names of the columns - * with_types - on the next line header with type names - */ - TabSeparatedRawRowInputFormat( - const Block & header_, - ReadBuffer & in_, - const Params & params_, - bool with_names_, - bool with_types_, - const FormatSettings & format_settings_) - : TabSeparatedRowInputFormat(header_, in_, params_, with_names_, with_types_, format_settings_) - { - } - - String getName() const override { return "TabSeparatedRawRowInputFormat"; } - - bool readField(IColumn & column, const DataTypePtr &, const SerializationPtr & serialization, bool) override - { - String tmp; - - while (!in->eof()) - { - char * pos = find_first_symbols<'\n', '\t'>(in->position(), in->buffer().end()); - - tmp.append(in->position(), pos - in->position()); - in->position() = pos; - - if (pos == in->buffer().end()) - in->next(); - else - break; - } - - ReadBufferFromString cell(tmp); - serialization->deserializeWholeText(column, cell, format_settings); - - return true; - } -}; - -} diff --git a/src/Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h deleted file mode 100644 index dc9312e53bc..00000000000 --- a/src/Processors/Formats/Impl/TabSeparatedRawRowOutputFormat.h +++ /dev/null @@ -1,35 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -/** A stream for outputting data in tsv format, but without escaping individual values. - * (That is, the output is irreversible.) - */ -class TabSeparatedRawRowOutputFormat : public TabSeparatedRowOutputFormat -{ -public: - TabSeparatedRawRowOutputFormat( - WriteBuffer & out_, - const Block & header_, - bool with_names_, - bool with_types_, - const RowOutputFormatParams & params_, - const FormatSettings & format_settings_) - : TabSeparatedRowOutputFormat(out_, header_, with_names_, with_types_, params_, format_settings_) - { - } - - String getName() const override { return "TabSeparatedRawRowOutputFormat"; } - - void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override - { - serialization.serializeText(column, row_num, out, format_settings); - } -}; - -} diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 1ff52c9f695..ec6dfef4f0c 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -1,9 +1,8 @@ #include -#include +#include #include #include -#include #include #include #include @@ -19,19 +18,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } - -static void skipTSVRow(ReadBuffer & in, const size_t num_columns) -{ - NullOutput null_sink; - - for (size_t i = 0; i < num_columns; ++i) - { - readEscapedStringInto(null_sink, in); - assertChar(i == num_columns - 1 ? '\n' : '\t', in); - } -} - - /** Check for a common error case - usage of Windows line feed. */ static void checkForCarriageReturn(ReadBuffer & in) @@ -45,187 +31,74 @@ static void checkForCarriageReturn(ReadBuffer & in) } -TabSeparatedRowInputFormat::TabSeparatedRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_, - bool with_names_, bool with_types_, const FormatSettings & format_settings_) - : RowInputFormatWithDiagnosticInfo(header_, in_, params_), with_names(with_names_), with_types(with_types_), format_settings(format_settings_) +TabSeparatedRowInputFormat::TabSeparatedRowInputFormat( + const Block & header_, + ReadBuffer & in_, + const Params & params_, + bool with_names_, + bool with_types_, + bool is_raw_, + const FormatSettings & format_settings_) + : RowInputFormatWithNamesAndTypes(header_, in_, params_, with_names_, with_types_, format_settings_), is_raw(is_raw_) { - const auto & sample = getPort().getHeader(); - size_t num_columns = sample.columns(); - - data_types.resize(num_columns); - column_indexes_by_names.reserve(num_columns); - - for (size_t i = 0; i < num_columns; ++i) - { - const auto & column_info = sample.getByPosition(i); - - data_types[i] = column_info.type; - column_indexes_by_names.emplace(column_info.name, i); - } - - column_mapping->column_indexes_for_input_fields.reserve(num_columns); - column_mapping->read_columns.assign(num_columns, false); } - -void TabSeparatedRowInputFormat::setupAllColumnsByTableSchema() +void TabSeparatedRowInputFormat::skipFieldDelimiter() { - const auto & header = getPort().getHeader(); - column_mapping->read_columns.assign(header.columns(), true); - column_mapping->column_indexes_for_input_fields.resize(header.columns()); - - for (size_t i = 0; i < column_mapping->column_indexes_for_input_fields.size(); ++i) - column_mapping->column_indexes_for_input_fields[i] = i; + assertChar('\t', *in); } - -void TabSeparatedRowInputFormat::addInputColumn(const String & column_name) -{ - const auto column_it = column_indexes_by_names.find(column_name); - if (column_it == column_indexes_by_names.end()) - { - if (format_settings.skip_unknown_fields) - { - column_mapping->column_indexes_for_input_fields.push_back(std::nullopt); - return; - } - - throw Exception( - "Unknown field found in TSV header: '" + column_name + "' " + - "at position " + std::to_string(column_mapping->column_indexes_for_input_fields.size()) + - "\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed", - ErrorCodes::INCORRECT_DATA - ); - } - - const auto column_index = column_it->second; - - if (column_mapping->read_columns[column_index]) - throw Exception("Duplicate field found while parsing TSV header: " + column_name, ErrorCodes::INCORRECT_DATA); - - column_mapping->read_columns[column_index] = true; - column_mapping->column_indexes_for_input_fields.emplace_back(column_index); -} - - -void TabSeparatedRowInputFormat::fillUnreadColumnsWithDefaults(MutableColumns & columns, RowReadExtension & row_read_extension) -{ - /// It is safe to memorize this on the first run - the format guarantees this does not change - if (unlikely(row_num == 1)) - { - columns_to_fill_with_default_values.clear(); - for (size_t index = 0; index < column_mapping->read_columns.size(); ++index) - if (column_mapping->read_columns[index] == 0) - columns_to_fill_with_default_values.push_back(index); - } - - for (const auto column_index : columns_to_fill_with_default_values) - { - data_types[column_index]->insertDefaultInto(*columns[column_index]); - row_read_extension.read_columns[column_index] = false; - } -} - - -void TabSeparatedRowInputFormat::readPrefix() -{ - if (with_names || with_types || data_types.at(0)->textCanContainOnlyValidUTF8()) - { - /// In this format, we assume that column name or type cannot contain BOM, - /// so, if format has header, - /// then BOM at beginning of stream cannot be confused with name or type of field, and it is safe to skip it. - skipBOMIfExists(*in); - } - - /// This is a bit of abstraction leakage, but we have almost the same code in other places. - /// Thus, we check if this InputFormat is working with the "real" beginning of the data in case of parallel parsing. - if (with_names && getCurrentUnitNumber() == 0) - { - if (format_settings.with_names_use_header) - { - String column_name; - for (;;) - { - readEscapedString(column_name, *in); - if (!checkChar('\t', *in)) - { - /// Check last column for \r before adding it, otherwise an error will be: - /// "Unknown field found in TSV header" - checkForCarriageReturn(*in); - addInputColumn(column_name); - break; - } - else - addInputColumn(column_name); - } - - - if (!in->eof()) - { - assertChar('\n', *in); - } - } - else - { - setupAllColumnsByTableSchema(); - skipTSVRow(*in, column_mapping->column_indexes_for_input_fields.size()); - } - } - else if (!column_mapping->is_set) - setupAllColumnsByTableSchema(); - - if (with_types) - { - skipTSVRow(*in, column_mapping->column_indexes_for_input_fields.size()); - } -} - - -bool TabSeparatedRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext) +void TabSeparatedRowInputFormat::skipRowEndDelimiter() { if (in->eof()) - return false; + return; - updateDiagnosticInfo(); + if (unlikely(row_num <= 1)) + checkForCarriageReturn(*in); - ext.read_columns.assign(column_mapping->read_columns.size(), true); - for (size_t file_column = 0; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column) + assertChar('\n', *in); +} + +String TabSeparatedRowInputFormat::readFieldIntoString() +{ + String field; + readEscapedString(field, *in); + return field; +} + +void TabSeparatedRowInputFormat::skipField() +{ + NullOutput null_sink; + readEscapedStringInto(null_sink, *in); +} + +void TabSeparatedRowInputFormat::skipRow() +{ + do { - const auto & column_index = column_mapping->column_indexes_for_input_fields[file_column]; - const bool is_last_file_column = file_column + 1 == column_mapping->column_indexes_for_input_fields.size(); - if (column_index) - { - const auto & type = data_types[*column_index]; - ext.read_columns[*column_index] = readField(*columns[*column_index], type, serializations[*column_index], is_last_file_column); - } - else - { - NullOutput null_sink; - readEscapedStringInto(null_sink, *in); - } - - /// skip separators - if (file_column + 1 < column_mapping->column_indexes_for_input_fields.size()) - { - assertChar('\t', *in); - } - else if (!in->eof()) - { - if (unlikely(row_num == 1)) - checkForCarriageReturn(*in); - - assertChar('\n', *in); - } + skipField(); } + while (checkChar('\t', *in)); - fillUnreadColumnsWithDefaults(columns, ext); + skipRowEndDelimiter(); +} - return true; +std::vector TabSeparatedRowInputFormat::readHeaderRow() +{ + std::vector fields; + do + { + fields.push_back(readFieldIntoString()); + } + while (checkChar('\t', *in)); + + skipRowEndDelimiter(); + return fields; } bool TabSeparatedRowInputFormat::readField(IColumn & column, const DataTypePtr & type, - const SerializationPtr & serialization, bool is_last_file_column) + const SerializationPtr & serialization, bool is_last_file_column, const String & /*column_name*/) { const bool at_delimiter = !is_last_file_column && !in->eof() && *in->position() == '\t'; const bool at_last_column_line_end = is_last_file_column && (in->eof() || *in->position() == '\n'); @@ -235,137 +108,112 @@ bool TabSeparatedRowInputFormat::readField(IColumn & column, const DataTypePtr & column.insertDefault(); return false; } - else if (format_settings.null_as_default && !type->isNullable()) + + bool as_nullable = format_settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable(); + + if (is_raw) + { + if (as_nullable) + return SerializationNullable::deserializeTextRawImpl(column, *in, format_settings, serialization); + + serialization->deserializeTextRaw(column, *in, format_settings); + return true; + } + + + + if (as_nullable) return SerializationNullable::deserializeTextEscapedImpl(column, *in, format_settings, serialization); serialization->deserializeTextEscaped(column, *in, format_settings); return true; } -bool TabSeparatedRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) +bool TabSeparatedRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) { - for (size_t file_column = 0; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column) + try { - if (file_column == 0 && in->eof()) + assertChar('\t', *in); + } + catch (const DB::Exception &) + { + if (*in->position() == '\n') { - out << "\n"; - return false; + out << "ERROR: Line feed found where tab is expected." + " It's like your file has less columns than expected.\n" + "And if your file has the right number of columns, " + "maybe it has an unescaped backslash in value before tab, which causes the tab to be escaped.\n"; } - - if (column_mapping->column_indexes_for_input_fields[file_column].has_value()) + else if (*in->position() == '\r') { - const auto & header = getPort().getHeader(); - size_t col_idx = column_mapping->column_indexes_for_input_fields[file_column].value(); - if (!deserializeFieldAndPrintDiagnosticInfo(header.getByPosition(col_idx).name, data_types[col_idx], *columns[col_idx], - out, file_column)) - return false; + out << "ERROR: Carriage return found where tab is expected.\n"; } else { - static const String skipped_column_str = ""; - static const DataTypePtr skipped_column_type = std::make_shared(); - static const MutableColumnPtr skipped_column = skipped_column_type->createColumn(); - if (!deserializeFieldAndPrintDiagnosticInfo(skipped_column_str, skipped_column_type, *skipped_column, out, file_column)) - return false; - } - - /// Delimiters - if (file_column + 1 == column_mapping->column_indexes_for_input_fields.size()) - { - if (!in->eof()) - { - try - { - assertChar('\n', *in); - } - catch (const DB::Exception &) - { - if (*in->position() == '\t') - { - out << "ERROR: Tab found where line feed is expected." - " It's like your file has more columns than expected.\n" - "And if your file has the right number of columns, maybe it has an unescaped tab in a value.\n"; - } - else if (*in->position() == '\r') - { - out << "ERROR: Carriage return found where line feed is expected." - " It's like your file has DOS/Windows style line separators, that is illegal in TabSeparated format.\n"; - } - else - { - out << "ERROR: There is no line feed. "; - verbosePrintString(in->position(), in->position() + 1, out); - out << " found instead.\n"; - } - return false; - } - } - } - else - { - try - { - assertChar('\t', *in); - } - catch (const DB::Exception &) - { - if (*in->position() == '\n') - { - out << "ERROR: Line feed found where tab is expected." - " It's like your file has less columns than expected.\n" - "And if your file has the right number of columns, " - "maybe it has an unescaped backslash in value before tab, which causes the tab to be escaped.\n"; - } - else if (*in->position() == '\r') - { - out << "ERROR: Carriage return found where tab is expected.\n"; - } - else - { - out << "ERROR: There is no tab. "; - verbosePrintString(in->position(), in->position() + 1, out); - out << " found instead.\n"; - } - return false; - } + out << "ERROR: There is no tab. "; + verbosePrintString(in->position(), in->position() + 1, out); + out << " found instead.\n"; } + return false; } return true; } -void TabSeparatedRowInputFormat::tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) +bool TabSeparatedRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out) { - const auto & index = column_mapping->column_indexes_for_input_fields[file_column]; - if (index) - { - bool can_be_parsed_as_null = removeLowCardinality(type)->isNullable(); + if (in->eof()) + return true; - // check null value for type is not nullable. don't cross buffer bound for simplicity, so maybe missing some case - if (!can_be_parsed_as_null && !in->eof()) + try + { + assertChar('\n', *in); + } + catch (const DB::Exception &) + { + if (*in->position() == '\t') { - if (*in->position() == '\\' && in->available() >= 2) + out << "ERROR: Tab found where line feed is expected." + " It's like your file has more columns than expected.\n" + "And if your file has the right number of columns, maybe it has an unescaped tab in a value.\n"; + } + else if (*in->position() == '\r') + { + out << "ERROR: Carriage return found where line feed is expected." + " It's like your file has DOS/Windows style line separators, that is illegal in TabSeparated format.\n"; + } + else + { + out << "ERROR: There is no line feed. "; + verbosePrintString(in->position(), in->position() + 1, out); + out << " found instead.\n"; + } + return false; + } + + return true; +} + +void TabSeparatedRowInputFormat::checkNullValueForNonNullable(DataTypePtr type) +{ + bool can_be_parsed_as_null = removeLowCardinality(type)->isNullable() || format_settings.null_as_default; + + // check null value for type is not nullable. don't cross buffer bound for simplicity, so maybe missing some case + if (!can_be_parsed_as_null && !in->eof()) + { + if (*in->position() == '\\' && in->available() >= 2) + { + ++in->position(); + if (*in->position() == 'N') { ++in->position(); - if (*in->position() == 'N') - { - ++in->position(); - throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected NULL value of not Nullable type {}", type->getName()); - } - else - { - --in->position(); - } + throw Exception(ErrorCodes::INCORRECT_DATA, "Unexpected NULL value of not Nullable type {}", type->getName()); + } + else + { + --in->position(); } } - - const bool is_last_file_column = file_column + 1 == column_mapping->column_indexes_for_input_fields.size(); - readField(column, type, serializations[*index], is_last_file_column); - } - else - { - NullOutput null_sink; - readEscapedStringInto(null_sink, *in); } } @@ -374,66 +222,28 @@ void TabSeparatedRowInputFormat::syncAfterError() skipToUnescapedNextLineOrEOF(*in); } -void TabSeparatedRowInputFormat::resetParser() -{ - RowInputFormatWithDiagnosticInfo::resetParser(); - const auto & sample = getPort().getHeader(); - column_mapping->read_columns.assign(sample.columns(), false); - column_mapping->column_indexes_for_input_fields.clear(); - columns_to_fill_with_default_values.clear(); -} - void registerInputFormatTabSeparated(FormatFactory & factory) { - for (const auto * name : {"TabSeparated", "TSV"}) + for (bool is_raw : {false, true}) { - factory.registerInputFormat(name, []( - ReadBuffer & buf, - const Block & sample, - IRowInputFormat::Params params, - const FormatSettings & settings) + auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - return std::make_shared(sample, buf, params, false, false, settings); - }); - } + factory.registerInputFormat(format_name, [with_names, with_types, is_raw]( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(sample, buf, std::move(params), with_names, with_types, is_raw, settings); + }); + }; - for (const auto * name : {"TabSeparatedRaw", "TSVRaw"}) - { - factory.registerInputFormat(name, []( - ReadBuffer & buf, - const Block & sample, - IRowInputFormat::Params params, - const FormatSettings & settings) - { - return std::make_shared(sample, buf, params, false, false, settings); - }); - } - - for (const auto * name : {"TabSeparatedWithNames", "TSVWithNames"}) - { - factory.registerInputFormat(name, []( - ReadBuffer & buf, - const Block & sample, - IRowInputFormat::Params params, - const FormatSettings & settings) - { - return std::make_shared(sample, buf, params, true, false, settings); - }); - } - - for (const auto * name : {"TabSeparatedWithNamesAndTypes", "TSVWithNamesAndTypes"}) - { - factory.registerInputFormat(name, []( - ReadBuffer & buf, - const Block & sample, - IRowInputFormat::Params params, - const FormatSettings & settings) - { - return std::make_shared(sample, buf, params, true, true, settings); - }); + registerInputFormatWithNamesAndTypes(is_raw ? "TabSeparatedRaw" : "TabSeparated", register_func); + registerInputFormatWithNamesAndTypes(is_raw ? "TSVRaw" : "TSV", register_func); } } +template static std::pair fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) { bool need_more_data = true; @@ -442,13 +252,18 @@ static std::pair fileSegmentationEngineTabSeparatedImpl(ReadBuffer while (loadAtPosition(in, memory, pos) && need_more_data) { - pos = find_first_symbols<'\\', '\r', '\n'>(pos, in.buffer().end()); + if constexpr (is_raw) + pos = find_first_symbols<'\r', '\n'>(pos, in.buffer().end()); + else + pos = find_first_symbols<'\\', '\r', '\n'>(pos, in.buffer().end()); if (pos > in.buffer().end()) - throw Exception("Position in buffer is out of bounds. There must be a bug.", ErrorCodes::LOGICAL_ERROR); - else if (pos == in.buffer().end()) + throw Exception("Position in buffer is out of bounds. There must be a bug.", ErrorCodes::LOGICAL_ERROR); + + if (pos == in.buffer().end()) continue; - else if (*pos == '\\') + + if (!is_raw && *pos == '\\') { ++pos; if (loadAtPosition(in, memory, pos)) @@ -472,11 +287,13 @@ static std::pair fileSegmentationEngineTabSeparatedImpl(ReadBuffer void registerFileSegmentationEngineTabSeparated(FormatFactory & factory) { + registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "TSV", &fileSegmentationEngineTabSeparatedImpl); + registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "TabSeparated", &fileSegmentationEngineTabSeparatedImpl); + registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "TSVRaw", &fileSegmentationEngineTabSeparatedImpl); + registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "TabSeparatedRaw", &fileSegmentationEngineTabSeparatedImpl); + // We can use the same segmentation engine for TSKV. - for (const auto & name : {"TabSeparated", "TSV", "TSKV", "TabSeparatedWithNames", "TSVWithNames"}) - { - factory.registerFileSegmentationEngine(name, &fileSegmentationEngineTabSeparatedImpl); - } + factory.registerFileSegmentationEngine("TSKV", &fileSegmentationEngineTabSeparatedImpl); } } diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h index 8127b5ceba7..31e6e12400a 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB @@ -10,46 +10,39 @@ namespace DB /** A stream to input data in tsv format. */ -class TabSeparatedRowInputFormat : public RowInputFormatWithDiagnosticInfo +class TabSeparatedRowInputFormat : public RowInputFormatWithNamesAndTypes { public: /** with_names - the first line is the header with the names of the columns * with_types - on the next line header with type names */ TabSeparatedRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_, - bool with_names_, bool with_types_, const FormatSettings & format_settings_); + bool with_names_, bool with_types_, bool is_raw, const FormatSettings & format_settings_); String getName() const override { return "TabSeparatedRowInputFormat"; } - bool readRow(MutableColumns & columns, RowReadExtension &) override; - void readPrefix() override; bool allowSyncAfterError() const override { return true; } void syncAfterError() override; - void resetParser() override; - -protected: - bool with_names; - bool with_types; - const FormatSettings format_settings; - - virtual bool readField(IColumn & column, const DataTypePtr & type, - const SerializationPtr & serialization, bool is_last_file_column); - private: - DataTypes data_types; + bool is_raw; - using IndexesMap = std::unordered_map; - IndexesMap column_indexes_by_names; + bool readField(IColumn & column, const DataTypePtr & type, + const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override; - std::vector columns_to_fill_with_default_values; + void skipField(const String & /*column_name*/) override { skipField(); } + void skipField(); + void skipRow() override; + void skipFieldDelimiter() override; + void skipRowEndDelimiter() override; - void addInputColumn(const String & column_name); - void setupAllColumnsByTableSchema(); - void fillUnreadColumnsWithDefaults(MutableColumns & columns, RowReadExtension & row_read_extension); + Names readHeaderRow() override; + String readFieldIntoString(); - bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override; - void tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) override; + void checkNullValueForNonNullable(DataTypePtr type) override; + + bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override; + bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) override; bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override { return *pos != '\n' && *pos != '\t'; } }; diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index 71d5bdba355..9a4f079867e 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -1,5 +1,4 @@ #include -#include #include #include @@ -11,41 +10,43 @@ TabSeparatedRowOutputFormat::TabSeparatedRowOutputFormat( const Block & header_, bool with_names_, bool with_types_, + bool is_raw_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_) - : IRowOutputFormat(header_, out_, params_), with_names(with_names_), with_types(with_types_), format_settings(format_settings_) + : IRowOutputFormat(header_, out_, params_), with_names(with_names_), with_types(with_types_), is_raw(is_raw_), format_settings(format_settings_) { } +void TabSeparatedRowOutputFormat::writeLine(const std::vector & values) +{ + for (size_t i = 0; i < values.size(); ++i) + { + writeEscapedString(values[i], out); + if (i + 1 == values.size()) + writeRowEndDelimiter(); + else + writeFieldDelimiter(); + } +} void TabSeparatedRowOutputFormat::doWritePrefix() { const auto & header = getPort(PortKind::Main).getHeader(); - size_t columns = header.columns(); if (with_names) - { - for (size_t i = 0; i < columns; ++i) - { - writeEscapedString(header.safeGetByPosition(i).name, out); - writeChar(i == columns - 1 ? '\n' : '\t', out); - } - } + writeLine(header.getNames()); if (with_types) - { - for (size_t i = 0; i < columns; ++i) - { - writeEscapedString(header.safeGetByPosition(i).type->getName(), out); - writeChar(i == columns - 1 ? '\n' : '\t', out); - } - } + writeLine(header.getDataTypeNames()); } void TabSeparatedRowOutputFormat::writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) { - serialization.serializeTextEscaped(column, row_num, out, format_settings); + if (is_raw) + serialization.serializeTextRaw(column, row_num, out, format_settings); + else + serialization.serializeTextEscaped(column, row_num, out, format_settings); } @@ -75,56 +76,23 @@ void TabSeparatedRowOutputFormat::writeBeforeExtremes() void registerOutputFormatTabSeparated(FormatFactory & factory) { - for (const auto * name : {"TabSeparated", "TSV"}) + for (bool is_raw : {false, true}) { - factory.registerOutputFormat(name, []( - WriteBuffer & buf, - const Block & sample, - const RowOutputFormatParams & params, - const FormatSettings & settings) + auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - return std::make_shared(buf, sample, false, false, params, settings); - }); - factory.markOutputFormatSupportsParallelFormatting(name); - } + factory.registerOutputFormat(format_name, [=]( + WriteBuffer & buf, + const Block & sample, + const RowOutputFormatParams & params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, with_names, with_types, is_raw, params, settings); + }); + factory.markOutputFormatSupportsParallelFormatting(format_name); + }; - for (const auto * name : {"TabSeparatedRaw", "TSVRaw"}) - { - factory.registerOutputFormat(name, []( - WriteBuffer & buf, - const Block & sample, - const RowOutputFormatParams & params, - const FormatSettings & settings) - { - return std::make_shared(buf, sample, false, false, params, settings); - }); - factory.markOutputFormatSupportsParallelFormatting(name); - } - - for (const auto * name : {"TabSeparatedWithNames", "TSVWithNames"}) - { - factory.registerOutputFormat(name, []( - WriteBuffer & buf, - const Block & sample, - const RowOutputFormatParams & params, - const FormatSettings & settings) - { - return std::make_shared(buf, sample, true, false, params, settings); - }); - factory.markOutputFormatSupportsParallelFormatting(name); - } - - for (const auto * name : {"TabSeparatedWithNamesAndTypes", "TSVWithNamesAndTypes"}) - { - factory.registerOutputFormat(name, []( - WriteBuffer & buf, - const Block & sample, - const RowOutputFormatParams & params, - const FormatSettings & settings) - { - return std::make_shared(buf, sample, true, true, params, settings); - }); - factory.markOutputFormatSupportsParallelFormatting(name); + registerOutputFormatWithNamesAndTypes(is_raw ? "TSVRaw" : "TSV", register_func); + registerOutputFormatWithNamesAndTypes(is_raw ? "TabSeparatedRaw" : "TabSeparated", register_func); } } diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h index e3190be70e8..7dcc6529f1c 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h @@ -23,6 +23,7 @@ public: const Block & header_, bool with_names_, bool with_types_, + bool is_raw_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_); @@ -39,10 +40,13 @@ public: /// https://www.iana.org/assignments/media-types/text/tab-separated-values String getContentType() const override { return "text/tab-separated-values; charset=UTF-8"; } -protected: - +private: + void writeLine(const std::vector & values); bool with_names; bool with_types; + bool is_raw; + +protected: const FormatSettings format_settings; }; diff --git a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp index ed98ab372b6..db5db4701a9 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp @@ -130,7 +130,7 @@ void TemplateBlockOutputFormat::serializeField(const IColumn & column, const ISe serialization.serializeTextXML(column, row_num, out, settings); break; case ColumnFormat::Raw: - serialization.serializeText(column, row_num, out, settings); + serialization.serializeTextRaw(column, row_num, out, settings); break; default: __builtin_unreachable(); diff --git a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp index 021b2532b39..c096b62e967 100644 --- a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp @@ -45,8 +45,8 @@ TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, ReadBuffer } else { - if (format.formats[i] == ColumnFormat::Xml || format.formats[i] == ColumnFormat::Raw) - format.throwInvalidFormat("XML and Raw deserialization is not supported", i); + if (format.formats[i] == ColumnFormat::Xml) + format.throwInvalidFormat("XML deserialization is not supported", i); } } @@ -54,8 +54,8 @@ TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, ReadBuffer std::vector column_in_format(header_.columns(), false); for (size_t i = 0; i < row_format.columnsCount(); ++i) { - if (row_format.formats[i] == ColumnFormat::Xml || row_format.formats[i] == ColumnFormat::Raw) - row_format.throwInvalidFormat("XML and Raw deserialization is not supported", i); + if (row_format.formats[i] == ColumnFormat::Xml) + row_format.throwInvalidFormat("XML deserialization is not supported", i); if (row_format.format_idx_to_column_idx[i]) { @@ -194,7 +194,7 @@ bool TemplateRowInputFormat::deserializeField(const DataTypePtr & type, { ColumnFormat col_format = row_format.formats[file_column]; bool read = true; - bool parse_as_nullable = settings.null_as_default && !type->isNullable(); + bool parse_as_nullable = settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable(); try { switch (col_format) @@ -226,6 +226,12 @@ bool TemplateRowInputFormat::deserializeField(const DataTypePtr & type, else serialization->deserializeTextJSON(column, buf, settings); break; + case ColumnFormat::Raw: + if (parse_as_nullable) + read = SerializationNullable::deserializeTextRawImpl(column, buf, settings, serialization); + else + serialization->deserializeTextRaw(column, buf, settings); + break; default: __builtin_unreachable(); } diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 0f6a21055d0..4eb447b82c3 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -168,7 +168,7 @@ bool ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx) bool read = true; const auto & type = types[column_idx]; const auto & serialization = serializations[column_idx]; - if (format_settings.null_as_default && !type->isNullable()) + if (format_settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable()) read = SerializationNullable::deserializeTextQuotedImpl(column, buf, format_settings, serialization); else serialization->deserializeTextQuoted(column, buf, format_settings); @@ -409,7 +409,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx Field value = convertFieldToType(expression_value, type, value_raw.second.get()); /// Check that we are indeed allowed to insert a NULL. - if (value.isNull() && !type.isNullable()) + if (value.isNull() && !type.isNullable() && !type.isLowCardinalityNullable()) { if (format_settings.null_as_default) { diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp new file mode 100644 index 00000000000..614ec27c0d5 --- /dev/null +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -0,0 +1,265 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; +} + +RowInputFormatWithNamesAndTypes::RowInputFormatWithNamesAndTypes( + const Block & header_, + ReadBuffer & in_, + const Params & params_, + bool with_names_, + bool with_types_, + const FormatSettings & format_settings_) + : RowInputFormatWithDiagnosticInfo(header_, in_, params_), format_settings(format_settings_), with_names(with_names_), with_types(with_types_) +{ + const auto & sample = getPort().getHeader(); + size_t num_columns = sample.columns(); + + data_types.resize(num_columns); + column_indexes_by_names.reserve(num_columns); + + for (size_t i = 0; i < num_columns; ++i) + { + const auto & column_info = sample.getByPosition(i); + + data_types[i] = column_info.type; + column_indexes_by_names.emplace(column_info.name, i); + } +} + +void RowInputFormatWithNamesAndTypes::setupAllColumnsByTableSchema() +{ + const auto & header = getPort().getHeader(); + column_mapping->column_indexes_for_input_fields.resize(header.columns()); + column_mapping->names_of_columns = header.getNames(); + + for (size_t i = 0; i < column_mapping->column_indexes_for_input_fields.size(); ++i) + column_mapping->column_indexes_for_input_fields[i] = i; +} + +void RowInputFormatWithNamesAndTypes::addInputColumn(const String & column_name, std::vector & read_columns) +{ + column_mapping->names_of_columns.push_back(column_name); + + const auto column_it = column_indexes_by_names.find(column_name); + if (column_it == column_indexes_by_names.end()) + { + if (format_settings.skip_unknown_fields) + { + column_mapping->column_indexes_for_input_fields.push_back(std::nullopt); + return; + } + + throw Exception( + ErrorCodes::INCORRECT_DATA, + "Unknown field found in {} header: '{}' at position {}\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed", + getName(), column_name, column_mapping->column_indexes_for_input_fields.size()); + } + + const auto column_index = column_it->second; + + if (read_columns[column_index]) + throw Exception("Duplicate field found while parsing TSV header: " + column_name, ErrorCodes::INCORRECT_DATA); + + read_columns[column_index] = true; + column_mapping->column_indexes_for_input_fields.emplace_back(column_index); +} + +void RowInputFormatWithNamesAndTypes::readPrefix() +{ + if (with_names || with_types || data_types.at(0)->textCanContainOnlyValidUTF8()) + { + /// We assume that column name or type cannot contain BOM, so, if format has header, + /// then BOM at beginning of stream cannot be confused with name or type of field, and it is safe to skip it. + skipBOMIfExists(*in); + } + + /// This is a bit of abstraction leakage, but we need it in parallel parsing: + /// we check if this InputFormat is working with the "real" beginning of the data. + if (with_names && getCurrentUnitNumber() == 0) + { + if (format_settings.with_names_use_header) + { + std::vector read_columns(data_types.size(), false); + auto column_names = readHeaderRow(); + for (const auto & name : column_names) + addInputColumn(name, read_columns); + + for (size_t i = 0; i != read_columns.size(); ++i) + { + if (!read_columns[i]) + column_mapping->not_presented_columns.push_back(i); + } + } + else + { + setupAllColumnsByTableSchema(); + skipRow(); + } + } + else if (!column_mapping->is_set) + setupAllColumnsByTableSchema(); + + if (with_types && getCurrentUnitNumber() == 0) + { + if (format_settings.with_types_use_header) + { + auto types = readHeaderRow(); + if (types.size() != column_mapping->column_indexes_for_input_fields.size()) + throw Exception( + ErrorCodes::INCORRECT_DATA, + "The number of data types differs from the number of column names in input data"); + + /// Check that types from input matches types from header. + for (size_t i = 0; i < types.size(); ++i) + { + if (column_mapping->column_indexes_for_input_fields[i] && + data_types[*column_mapping->column_indexes_for_input_fields[i]]->getName() != types[i]) + { + throw Exception( + ErrorCodes::INCORRECT_DATA, + "Type of '{}' must be {}, not {}", + getPort().getHeader().getByPosition(*column_mapping->column_indexes_for_input_fields[i]).name, + data_types[*column_mapping->column_indexes_for_input_fields[i]]->getName(), types[i]); + } + } + } + else + skipRow(); + } +} + +void RowInputFormatWithNamesAndTypes::insertDefaultsForNotSeenColumns(MutableColumns & columns, RowReadExtension & ext) +{ + for (auto index : column_mapping->not_presented_columns) + { + columns[index]->insertDefault(); + ext.read_columns[index] = false; + } +} + +bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadExtension & ext) +{ + if (in->eof()) + return false; + + updateDiagnosticInfo(); + skipRowStartDelimiter(); + + ext.read_columns.resize(data_types.size()); + for (size_t file_column = 0; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column) + { + const auto & column_index = column_mapping->column_indexes_for_input_fields[file_column]; + const bool is_last_file_column = file_column + 1 == column_mapping->column_indexes_for_input_fields.size(); + if (column_index) + ext.read_columns[*column_index] = readField( + *columns[*column_index], + data_types[*column_index], + serializations[*column_index], + is_last_file_column, + column_mapping->names_of_columns[file_column]); + else + skipField(column_mapping->names_of_columns[file_column]); + + if (!is_last_file_column) + skipFieldDelimiter(); + } + + skipRowEndDelimiter(); + + insertDefaultsForNotSeenColumns(columns, ext); + + /// If defaults_for_omitted_fields is set to 0, we should leave already inserted defaults. + if (!format_settings.defaults_for_omitted_fields) + ext.read_columns.assign(ext.read_columns.size(), true); + + return true; +} + +void RowInputFormatWithNamesAndTypes::resetParser() +{ + RowInputFormatWithDiagnosticInfo::resetParser(); + column_mapping->column_indexes_for_input_fields.clear(); + column_mapping->not_presented_columns.clear(); +} + +void RowInputFormatWithNamesAndTypes::tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) +{ + const auto & index = column_mapping->column_indexes_for_input_fields[file_column]; + if (index) + { + checkNullValueForNonNullable(type); + const bool is_last_file_column = file_column + 1 == column_mapping->column_indexes_for_input_fields.size(); + readField(column, type, serializations[*index], is_last_file_column, column_mapping->names_of_columns[file_column]); + } + else + { + skipField(column_mapping->names_of_columns[file_column]); + } +} + +bool RowInputFormatWithNamesAndTypes::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) +{ + if (in->eof()) + { + out << "\n"; + return false; + } + + if (!parseRowStartWithDiagnosticInfo(out)) + return false; + + for (size_t file_column = 0; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column) + { + if (column_mapping->column_indexes_for_input_fields[file_column].has_value()) + { + const auto & header = getPort().getHeader(); + size_t col_idx = column_mapping->column_indexes_for_input_fields[file_column].value(); + if (!deserializeFieldAndPrintDiagnosticInfo(header.getByPosition(col_idx).name, data_types[col_idx], *columns[col_idx], out, file_column)) + return false; + } + else + { + static const String skipped_column_str = ""; + static const DataTypePtr skipped_column_type = std::make_shared(); + static const MutableColumnPtr skipped_column = skipped_column_type->createColumn(); + if (!deserializeFieldAndPrintDiagnosticInfo(skipped_column_str, skipped_column_type, *skipped_column, out, file_column)) + return false; + } + + /// Delimiters + if (file_column + 1 != column_mapping->column_indexes_for_input_fields.size()) + { + if (!parseFieldDelimiterWithDiagnosticInfo(out)) + return false; + } + } + + return parseRowEndWithDiagnosticInfo(out); +} + +void registerInputFormatWithNamesAndTypes(const String & base_format_name, RegisterFormatWithNamesAndTypesFunc register_func) +{ + register_func(base_format_name, false, false); + register_func(base_format_name + "WithNames", true, false); + register_func(base_format_name + "WithNamesAndTypes", true, true); +} + +void registerFileSegmentationEngineForFormatWithNamesAndTypes( + FormatFactory & factory, const String & base_format_name, FormatFactory::FileSegmentationEngine segmentation_engine) +{ + factory.registerFileSegmentationEngine(base_format_name, segmentation_engine); + factory.registerFileSegmentationEngine(base_format_name + "WithNames", segmentation_engine); + factory.registerFileSegmentationEngine(base_format_name + "WithNamesAndTypes", segmentation_engine); +} + + +} diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h new file mode 100644 index 00000000000..d9413b3a9bc --- /dev/null +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -0,0 +1,73 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +/// Base class for input formats with -WithNames and -WithNamesAndTypes suffixes. +class RowInputFormatWithNamesAndTypes : public RowInputFormatWithDiagnosticInfo +{ +public: + /** with_names - in the first line the header with column names + * with_types - in the second line the header with column names + */ + RowInputFormatWithNamesAndTypes( + const Block & header_, + ReadBuffer & in_, + const Params & params_, + bool with_names_, bool with_types_, const FormatSettings & format_settings_); + + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + void readPrefix() override; + void resetParser() override; + +protected: + /// Return false if there was no real value and we inserted default value. + virtual bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) = 0; + + virtual void skipField(const String & column_name) = 0; + virtual void skipRow() = 0; + virtual void skipRowStartDelimiter() {} + virtual void skipFieldDelimiter() {} + virtual void skipRowEndDelimiter() {} + + + /// Methods for parsing with diagnostic info. + virtual void checkNullValueForNonNullable(DataTypePtr /*type*/) {} + virtual bool parseRowStartWithDiagnosticInfo(WriteBuffer & /*out*/) { return true; } + virtual bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) = 0; + virtual bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) = 0; + + /// Read the list of names or types. + virtual std::vector readHeaderRow() = 0; + + const FormatSettings format_settings; + DataTypes data_types; + +private: + bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override; + void tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) override; + + void setupAllColumnsByTableSchema(); + void addInputColumn(const String & column_name, std::vector & read_columns); + void insertDefaultsForNotSeenColumns(MutableColumns & columns, RowReadExtension & ext); + + bool with_names; + bool with_types; + std::unordered_map column_indexes_by_names; +}; + +using RegisterFormatWithNamesAndTypesFunc = std::function; + +void registerInputFormatWithNamesAndTypes(const String & base_format_name, RegisterFormatWithNamesAndTypesFunc register_func); + +void registerFileSegmentationEngineForFormatWithNamesAndTypes( + FormatFactory & factory, const String & base_format_name, FormatFactory::FileSegmentationEngine segmentation_engine); + +} diff --git a/tests/queries/0_stateless/00300_csv.reference b/tests/queries/0_stateless/00300_csv.reference index 9d2fe7233d8..42cd22078c4 100644 --- a/tests/queries/0_stateless/00300_csv.reference +++ b/tests/queries/0_stateless/00300_csv.reference @@ -1,6 +1,10 @@ +"Hello, ""World""",123,"[1,2,3]",456,"['abc','def']","Newline +here" "x","y","z","a","b" "Hello, ""World""",123,"[1,2,3]",456,"['abc','def']","Newline here" +"x","y","z","a","b" +"String","UInt8","Array(UInt8)","Tuple(UInt16, Array(String))","String" "Hello, ""World""",123,"[1,2,3]",456,"['abc','def']","Newline here" 0,"0","[]","2000-01-01","2000-01-01 00:00:00" diff --git a/tests/queries/0_stateless/00300_csv.sql b/tests/queries/0_stateless/00300_csv.sql index 0c761ad0af1..76b1b29df06 100644 --- a/tests/queries/0_stateless/00300_csv.sql +++ b/tests/queries/0_stateless/00300_csv.sql @@ -1,3 +1,4 @@ -SELECT 'Hello, "World"' AS x, 123 AS y, [1, 2, 3] AS z, (456, ['abc', 'def']) AS a, 'Newline\nhere' AS b FORMAT CSVWithNames; SELECT 'Hello, "World"' AS x, 123 AS y, [1, 2, 3] AS z, (456, ['abc', 'def']) AS a, 'Newline\nhere' AS b FORMAT CSV; +SELECT 'Hello, "World"' AS x, 123 AS y, [1, 2, 3] AS z, (456, ['abc', 'def']) AS a, 'Newline\nhere' AS b FORMAT CSVWithNames; +SELECT 'Hello, "World"' AS x, 123 AS y, [1, 2, 3] AS z, (456, ['abc', 'def']) AS a, 'Newline\nhere' AS b FORMAT CSVWithNamesAndTypes; SELECT number, toString(number), range(number), toDate('2000-01-01') + number, toDateTime('2000-01-01 00:00:00') + number FROM system.numbers LIMIT 10 FORMAT CSV; diff --git a/tests/queries/0_stateless/00301_csv.sh b/tests/queries/0_stateless/00301_csv.sh index 0aee9abe25c..39721ce1050 100755 --- a/tests/queries/0_stateless/00301_csv.sh +++ b/tests/queries/0_stateless/00301_csv.sh @@ -13,7 +13,7 @@ Hello "world", 789 ,2016-01-03 "Hello world", 100, 2016-01-04, default,, - default-eof,,' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --query="INSERT INTO csv FORMAT CSV"; + default-eof,,' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --input_format_csv_empty_as_default=1 --query="INSERT INTO csv FORMAT CSV"; $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY d"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; @@ -33,7 +33,7 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE csv (t Nullable(DateTime('Europe/Moscow echo 'NULL, NULL "2016-01-01 01:02:03",NUL -"2016-01-02 01:02:03",Nhello' | $CLICKHOUSE_CLIENT --input_format_csv_unquoted_null_literal_as_null=1 --query="INSERT INTO csv FORMAT CSV"; +"2016-01-02 01:02:03",Nhello' | $CLICKHOUSE_CLIENT --input_format_csv_unquoted_null_literal_as_null=1 --input_format_csv_empty_as_default=1 --query="INSERT INTO csv FORMAT CSV"; $CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY s NULLS LAST"; $CLICKHOUSE_CLIENT --query="DROP TABLE csv"; diff --git a/tests/queries/0_stateless/00938_template_input_format.reference b/tests/queries/0_stateless/00938_template_input_format.reference index ce89532886d..e1f77d9a581 100644 --- a/tests/queries/0_stateless/00938_template_input_format.reference +++ b/tests/queries/0_stateless/00938_template_input_format.reference @@ -23,3 +23,11 @@ cv bn m","","as""df'gh","",456,"2016-01-02" "as""df'gh","","zx cv bn m","",789,"2016-01-04" "qwe,rty","","","",9876543210,"2016-01-03" +==== check raw ==== +"qwe,rty","as""df'gh","","zx +cv bn m",123,"2016-01-01" +"as""df\'gh","","zx +cv bn m","qwe,rty",456,"2016-01-02" +"zx\cv\bn m","qwe,rty","as""df'gh","",789,"2016-01-04" +"","zx +cv bn m","qwe,rty","as""df'gh",9876543210,"2016-01-03" diff --git a/tests/queries/0_stateless/00938_template_input_format.sh b/tests/queries/0_stateless/00938_template_input_format.sh index 75616b35af0..bf7631cf3d5 100755 --- a/tests/queries/0_stateless/00938_template_input_format.sh +++ b/tests/queries/0_stateless/00938_template_input_format.sh @@ -50,6 +50,30 @@ format_template_rows_between_delimiter = ','"; $CLICKHOUSE_CLIENT --query="SELECT * FROM template2 ORDER BY n FORMAT CSV"; +echo "==== check raw ====" + +echo -ne '{prefix} \n${data}\n $$ suffix $$\n' > "$CURDIR"/00938_template_input_format_resultset.tmp +echo -ne 'n:\t${n:Escaped}, s1:\t${0:Raw}\t, s2:\t${1:Quoted}, s3:\t${s3:JSON}, s4:\t${3:CSV}, d:\t${d:Escaped}\t' > "$CURDIR"/00938_template_input_format_row.tmp + + +$CLICKHOUSE_CLIENT --query="TRUNCATE TABLE template1"; + +echo "{prefix}"' '" +n: 123, s1: qwe,rty , s2: 'as\"df\\'gh', s3: \"\", s4: \"zx +cv bn m\", d: 2016-01-01 ; +n: 456, s1: as\"df\\'gh , s2: '', s3: \"zx\\ncv\\tbn m\", s4: \"qwe,rty\", d: 2016-01-02 ; +n: 9876543210, s1: , s2: 'zx\\ncv\\tbn m', s3: \"qwe,rty\", s4: \"as\"\"df'gh\", d: 2016-01-03 ; +n: 789, s1: zx\cv\bn m , s2: 'qwe,rty', s3: \"as\\\"df'gh\", s4: \"\", d: 2016-01-04"$'\t'" + $ suffix $" | $CLICKHOUSE_CLIENT --query="INSERT INTO template1 FORMAT Template SETTINGS \ +format_template_resultset = '$CURDIR/00938_template_input_format_resultset.tmp', \ +format_template_row = '$CURDIR/00938_template_input_format_row.tmp', \ +format_template_rows_between_delimiter = ';\n'"; + +$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT CSV"; + + + $CLICKHOUSE_CLIENT --query="DROP TABLE template1"; $CLICKHOUSE_CLIENT --query="DROP TABLE template2"; rm "$CURDIR"/00938_template_input_format_resultset.tmp "$CURDIR"/00938_template_input_format_row.tmp + diff --git a/tests/queries/0_stateless/01034_JSONCompactEachRow.reference b/tests/queries/0_stateless/01034_JSONCompactEachRow.reference index 6ec53e11fc9..bfc99d688d5 100644 --- a/tests/queries/0_stateless/01034_JSONCompactEachRow.reference +++ b/tests/queries/0_stateless/01034_JSONCompactEachRow.reference @@ -12,6 +12,11 @@ [1, "a"] [2, "b"] [3, "c"] +---------- +["value", "name"] +[1, "a"] +[2, "b"] +[3, "c"] 4 ["name", "c"] ["String", "UInt64"] @@ -31,17 +36,33 @@ 8 ["first", 1, 2, 0] ["second", 2, 0, 6] +["first", 1, 2, 0] +["second", 2, 0, 6] 9 ["first", 1, 2, 8] ["second", 2, 32, 6] +["first", 1, 2, 8] +["second", 2, 32, 6] 10 ["first", 1, 16, 8] ["second", 2, 32, 8] +["first", 1, 16, 8] +["second", 2, 32, 8] 11 ["v1", "v2", "v3", "v4"] ["String", "UInt8", "UInt16", "UInt8"] ["", 2, 3, 1] +["", 2, 3, 1] +---------- +["v1", "v2", "v3", "v4"] +["", 2, 3, 1] +["", 2, 3, 1] 12 ["v1", "n.id", "n.name"] ["UInt8", "Array(UInt8)", "Array(String)"] [16, [15,16,0], ["first","second","third"]] +[16, [15,16,0], ["first","second","third"]] +---------- +["v1", "n.id", "n.name"] +[16, [15,16,0], ["first","second","third"]] +[16, [15,16,0], ["first","second","third"]] diff --git a/tests/queries/0_stateless/01034_JSONCompactEachRow.sql b/tests/queries/0_stateless/01034_JSONCompactEachRow.sql index f5442c90a2a..f71597a60e5 100644 --- a/tests/queries/0_stateless/01034_JSONCompactEachRow.sql +++ b/tests/queries/0_stateless/01034_JSONCompactEachRow.sql @@ -10,8 +10,10 @@ SELECT 2; /* Check Totals */ SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactEachRow; SELECT 3; -/* Check JSONCompactEachRowWithNamesAndTypes Output */ +/* Check JSONCompactEachRowWithNames and JSONCompactEachRowWithNamesAndTypes Output */ SELECT * FROM test_table FORMAT JSONCompactEachRowWithNamesAndTypes; +SELECT '----------'; +SELECT * FROM test_table FORMAT JSONCompactEachRowWithNames; SELECT 4; /* Check Totals */ SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactEachRowWithNamesAndTypes; @@ -35,30 +37,39 @@ INSERT INTO test_table_2 FORMAT JSONCompactEachRow [16, [15, 16, null], ["first" SELECT * FROM test_table_2 FORMAT JSONCompactEachRow; TRUNCATE TABLE test_table_2; SELECT 8; -/* Check JSONCompactEachRowWithNamesAndTypes Output */ +/* Check JSONCompactEachRowWithNamesAndTypes and JSONCompactEachRowWithNamesAndTypes Input */ SET input_format_null_as_default = 0; INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", 1, "2", null]["second", 2, null, 6]; +INSERT INTO test_table FORMAT JSONCompactEachRowWithNames ["v1", "v2", "v3", "v4"]["first", 1, "2", null]["second", 2, null, 6]; SELECT * FROM test_table FORMAT JSONCompactEachRow; TRUNCATE TABLE test_table; SELECT 9; /* Check input_format_null_as_default = 1 */ SET input_format_null_as_default = 1; INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", 1, "2", null] ["second", 2, null, 6]; +INSERT INTO test_table FORMAT JSONCompactEachRowWithNames ["v1", "v2", "v3", "v4"]["first", 1, "2", null] ["second", 2, null, 6]; SELECT * FROM test_table FORMAT JSONCompactEachRow; SELECT 10; /* Check Header */ TRUNCATE TABLE test_table; SET input_format_skip_unknown_fields = 1; INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "v2", "invalid_column"]["String", "UInt8", "UInt8"]["first", 1, 32]["second", 2, "64"]; +INSERT INTO test_table FORMAT JSONCompactEachRowWithNames ["v1", "v2", "invalid_column"]["first", 1, 32]["second", 2, "64"]; SELECT * FROM test_table FORMAT JSONCompactEachRow; SELECT 11; TRUNCATE TABLE test_table; INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v4", "v2", "v3"]["UInt8", "UInt8", "UInt16"][1, 2, 3] +INSERT INTO test_table FORMAT JSONCompactEachRowWithNames ["v4", "v2", "v3"][1, 2, 3] SELECT * FROM test_table FORMAT JSONCompactEachRowWithNamesAndTypes; +SELECT '----------'; +SELECT * FROM test_table FORMAT JSONCompactEachRowWithNames; SELECT 12; /* Check Nested */ INSERT INTO test_table_2 FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "n.id", "n.name"]["UInt8", "Array(UInt8)", "Array(String)"][16, [15, 16, null], ["first", "second", "third"]]; +INSERT INTO test_table_2 FORMAT JSONCompactEachRowWithNames ["v1", "n.id", "n.name"][16, [15, 16, null], ["first", "second", "third"]]; SELECT * FROM test_table_2 FORMAT JSONCompactEachRowWithNamesAndTypes; +SELECT '----------'; +SELECT * FROM test_table_2 FORMAT JSONCompactEachRowWithNames; DROP TABLE IF EXISTS test_table; DROP TABLE IF EXISTS test_table_2; diff --git a/tests/queries/0_stateless/01195_formats_diagnostic_info.reference b/tests/queries/0_stateless/01195_formats_diagnostic_info.reference index 15fc31538ce..eddbb80198d 100644 --- a/tests/queries/0_stateless/01195_formats_diagnostic_info.reference +++ b/tests/queries/0_stateless/01195_formats_diagnostic_info.reference @@ -1,5 +1,5 @@ CSV -Column 2, name: d, type: Decimal(18, 10), parsed text: "123456789"ERROR +Column 2, name: d, type: Decimal(18, 10), parsed text: " 123456789"ERROR ERROR: garbage after DateTime: "7, Hello" ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format. ERROR: There is no line feed. "1" found instead. @@ -28,3 +28,14 @@ ERROR: There is no delimiter before field 1: expected "", got "7Hello< ERROR: There is no delimiter after last field: expected "", got "1" ERROR: There is no delimiter after last field: expected "", got "Hello" Column 0, name: t, type: DateTime, ERROR: text "" is not like DateTime +JSONCompactEachRow +Column 2, name: d, type: Decimal(18, 10), parsed text: " 123456789"ERROR +Column 0, name: t, type: DateTime, parsed text: "2020-04-21 12:34:56"ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format. +ERROR: garbage after DateTime: "7, Hello" +ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format. +ERROR: There is no closing parenthesis (']') at the end of the row. "," found instead. +Column 1, name: s, type: String, parsed text: ERROR +ERROR: There is no '[' before the row. +ERROR: garbage after Decimal(18, 10): ";" +ERROR: There is no comma. ";" found instead. +ERROR: Closing parenthesis (']') found where comma is expected. It's like your file has less columns than expected. diff --git a/tests/queries/0_stateless/01195_formats_diagnostic_info.sh b/tests/queries/0_stateless/01195_formats_diagnostic_info.sh index 6c64b17f719..dde410d95c4 100755 --- a/tests/queries/0_stateless/01195_formats_diagnostic_info.sh +++ b/tests/queries/0_stateless/01195_formats_diagnostic_info.sh @@ -38,3 +38,19 @@ echo -e '2020-04-21 12:34:567\tHello\t123456789' | "${PARSER[@]}" 2>&1| grep "ER echo -e '2020-04-21 12:34:56\tHello\t12345678\t1' | "${PARSER[@]}" 2>&1| grep "ERROR" echo -e '2020-04-21 12:34:56\t\t123Hello' | "${PARSER[@]}" 2>&1| grep "ERROR" echo -e '2020-04-21 12:34:56\tHello\t12345678\n' | "${PARSER[@]}" 2>&1| grep "ERROR" + +PARSER=(${CLICKHOUSE_LOCAL} --query 'SELECT t, s, d FROM table' --structure 't DateTime, s String, d Decimal64(10)' --input-format JSONCompactEachRow) +echo '["2020-04-21 12:34:56", "Hello", 12345678]' | "${PARSER[@]}" 2>&1| grep "ERROR" || echo "JSONCompactEachRow" +echo '["2020-04-21 12:34:56", "Hello", 123456789]' | "${PARSER[@]}" 2>&1| grep "ERROR" +echo '["2020-04-21 12:34:567", "Hello", 123456789]' | "${PARSER[@]}" 2>&1| grep "ERROR" +echo '["2020-04-21 12:34:56"7, "Hello", 123456789]' | "${PARSER[@]}" 2>&1| grep "ERROR" +echo '["2020-04-21 12:34:56", "Hello", 12345678,1]' | "${PARSER[@]}" 2>&1| grep "ERROR" +echo '["2020-04-21 12:34:56",,123Hello]' | "${PARSER[@]}" 2>&1| grep "ERROR" +echo -e '["2020-04-21 12:34:56", "Hello", 12345678\n]' | "${PARSER[@]}" 2>&1| grep "ERROR" +echo -e '"2020-04-21 12:34:56", "Hello", 12345678]' | "${PARSER[@]}" 2>&1| grep "ERROR" +echo -e '["2020-04-21 12:34:56", "Hello", 12345678;' | "${PARSER[@]}" 2>&1| grep "ERROR" +echo -e '["2020-04-21 12:34:56", "Hello", 12345678' | "${PARSER[@]}" 2>&1| grep "ERROR" +echo -e '["2020-04-21 12:34:56", "Hello", 12345678\n' | "${PARSER[@]}" 2>&1| grep "ERROR" +echo -e '["2020-04-21 12:34:56", "Hello"; 12345678\n' | "${PARSER[@]}" 2>&1| grep "ERROR" +echo -e '["2020-04-21 12:34:56", "Hello"\n' | "${PARSER[@]}" 2>&1| grep "ERROR" +echo -e '["2020-04-21 12:34:56", "Hello"]' | "${PARSER[@]}" 2>&1| grep "ERROR" diff --git a/tests/queries/0_stateless/01375_output_format_tsv_csv_with_names.reference b/tests/queries/0_stateless/01375_output_format_tsv_csv_with_names.reference index 6f1974ccd73..ffea4c736dc 100644 --- a/tests/queries/0_stateless/01375_output_format_tsv_csv_with_names.reference +++ b/tests/queries/0_stateless/01375_output_format_tsv_csv_with_names.reference @@ -7,7 +7,21 @@ number UInt64 0 1 +TSVRawWithNames +number +0 +1 +TSVRawWithNamesAndTypes +number +UInt64 +0 +1 CSVWithNames "number" 0 1 +CSVWithNamesAndTypes +"number" +"UInt64" +0 +1 diff --git a/tests/queries/0_stateless/01375_output_format_tsv_csv_with_names.sh b/tests/queries/0_stateless/01375_output_format_tsv_csv_with_names.sh index ad9cc2c53a8..69f3ab1c9a8 100755 --- a/tests/queries/0_stateless/01375_output_format_tsv_csv_with_names.sh +++ b/tests/queries/0_stateless/01375_output_format_tsv_csv_with_names.sh @@ -15,5 +15,14 @@ ${CLICKHOUSE_LOCAL} "${opts[@]}" --format TSVWithNames echo 'TSVWithNamesAndTypes' ${CLICKHOUSE_LOCAL} "${opts[@]}" --format TSVWithNamesAndTypes +echo 'TSVRawWithNames' +${CLICKHOUSE_LOCAL} "${opts[@]}" --format TSVWithNames + +echo 'TSVRawWithNamesAndTypes' +${CLICKHOUSE_LOCAL} "${opts[@]}" --format TSVWithNamesAndTypes + echo 'CSVWithNames' ${CLICKHOUSE_LOCAL} "${opts[@]}" --format CSVWithNames + +echo 'CSVWithNamesAndTypes' +${CLICKHOUSE_LOCAL} "${opts[@]}" --format CSVWithNamesAndTypes diff --git a/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.reference b/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.reference index 4f2a79b9905..78286b89a39 100644 --- a/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.reference +++ b/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.reference @@ -2,6 +2,11 @@ zero rows TSVWithNames TSVWithNamesAndTypes CSVWithNames +CSVWithNamesAndTypes +JSONCompactEachRowWithNames +JSONCompactEachRowWithNamesAndTypes +JSONCompactStringsEachRow +JSONCompactStringsEachRowWithNamesAndTypes multi clickhouse-local one file TSVWithNames 0 @@ -15,3 +20,23 @@ CSVWithNames 0 0 0 +CSVWithNamesAndTypes +0 +0 +0 +JSONCompactEachRowWithNames +0 +0 +0 +JSONCompactEachRowWithNamesAndTypes +0 +0 +0 +JSONCompactStringsEachRow +0 +0 +0 +JSONCompactStringsEachRowWithNamesAndTypes +0 +0 +0 diff --git a/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh b/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh index 469f7e7008b..7731deaa8ff 100755 --- a/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh +++ b/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh @@ -6,26 +6,26 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # zero rows echo 'zero rows' -for format in TSVWithNames TSVWithNamesAndTypes CSVWithNames; do +for format in TSVWithNames TSVWithNamesAndTypes CSVWithNames CSVWithNamesAndTypes JSONCompactEachRowWithNames JSONCompactEachRowWithNamesAndTypes JSONCompactStringsEachRow JSONCompactStringsEachRowWithNamesAndTypes; do echo $format ${CLICKHOUSE_LOCAL} --query=" - CREATE TABLE ${format}_01375 ENGINE File($format, '01375_$format.tsv') AS SELECT * FROM numbers(1) WHERE number < 0; + CREATE TABLE ${format}_01375 ENGINE File($format, '01375_$format') AS SELECT * FROM numbers(1) WHERE number < 0; SELECT * FROM ${format}_01375; DROP TABLE ${format}_01375; " - rm 01375_$format.tsv + rm 01375_$format done # run multiple times to the same file echo 'multi clickhouse-local one file' -for format in TSVWithNames TSVWithNamesAndTypes CSVWithNames; do +for format in TSVWithNames TSVWithNamesAndTypes CSVWithNames CSVWithNamesAndTypes JSONCompactEachRowWithNames JSONCompactEachRowWithNamesAndTypes JSONCompactStringsEachRow JSONCompactStringsEachRowWithNamesAndTypes; do echo $format for _ in {1..2}; do ${CLICKHOUSE_LOCAL} --query=" - CREATE TABLE ${format}_01375 ENGINE File($format, '01375_$format.tsv') AS SELECT * FROM numbers(1); + CREATE TABLE ${format}_01375 ENGINE File($format, '01375_$format') AS SELECT * FROM numbers(1); SELECT * FROM ${format}_01375; DROP TABLE ${format}_01375; " done - rm 01375_$format.tsv + rm 01375_$format done diff --git a/tests/queries/0_stateless/01448_json_compact_strings_each_row.reference b/tests/queries/0_stateless/01448_json_compact_strings_each_row.reference index fb1a066f272..8a69cf26ffd 100644 --- a/tests/queries/0_stateless/01448_json_compact_strings_each_row.reference +++ b/tests/queries/0_stateless/01448_json_compact_strings_each_row.reference @@ -12,6 +12,11 @@ ["1", "a"] ["2", "b"] ["3", "c"] +---------- +["value", "name"] +["1", "a"] +["2", "b"] +["3", "c"] 4 ["name", "c"] ["String", "UInt64"] @@ -31,17 +36,33 @@ 8 ["first", "1", "2", "0"] ["second", "2", "0", "6"] +["first", "1", "2", "0"] +["second", "2", "0", "6"] 9 ["first", "1", "2", "8"] ["second", "2", "32", "6"] +["first", "1", "2", "8"] +["second", "2", "32", "6"] 10 ["first", "1", "16", "8"] ["second", "2", "32", "8"] +["first", "1", "16", "8"] +["second", "2", "32", "8"] 11 ["v1", "v2", "v3", "v4"] ["String", "UInt8", "UInt16", "UInt8"] ["", "2", "3", "1"] +["", "2", "3", "1"] +--------- +["v1", "v2", "v3", "v4"] +["", "2", "3", "1"] +["", "2", "3", "1"] 12 ["v1", "n.id", "n.name"] ["UInt8", "Array(UInt8)", "Array(String)"] ["16", "[15,16,17]", "['first','second','third']"] +["16", "[15,16,17]", "['first','second','third']"] +--------- +["v1", "n.id", "n.name"] +["16", "[15,16,17]", "['first','second','third']"] +["16", "[15,16,17]", "['first','second','third']"] diff --git a/tests/queries/0_stateless/01448_json_compact_strings_each_row.sql b/tests/queries/0_stateless/01448_json_compact_strings_each_row.sql index 925faa3a17f..869041193cf 100644 --- a/tests/queries/0_stateless/01448_json_compact_strings_each_row.sql +++ b/tests/queries/0_stateless/01448_json_compact_strings_each_row.sql @@ -12,8 +12,10 @@ SELECT 2; /* Check Totals */ SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactStringsEachRow; SELECT 3; -/* Check JSONCompactStringsEachRowWithNamesAndTypes Output */ +/* Check JSONCompactStringsEachRowWithNames and JSONCompactStringsEachRowWithNamesAndTypes Output */ SELECT * FROM test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes; +SELECT '----------'; +SELECT * FROM test_table FORMAT JSONCompactStringsEachRowWithNames; SELECT 4; /* Check Totals */ SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactStringsEachRowWithNamesAndTypes; @@ -37,30 +39,39 @@ INSERT INTO test_table_2 FORMAT JSONCompactStringsEachRow ["16", "[15, 16, 17]", SELECT * FROM test_table_2 FORMAT JSONCompactStringsEachRow; TRUNCATE TABLE test_table_2; SELECT 8; -/* Check JSONCompactStringsEachRowWithNamesAndTypes Output */ +/* Check JSONCompactStringsEachRowWithNames and JSONCompactStringsEachRowWithNamesAndTypes Input */ SET input_format_null_as_default = 0; INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"]["second", "2", "null", "6"]; +INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNames ["v1", "v2", "v3", "v4"]["first", "1", "2", "null"]["second", "2", "null", "6"]; SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; TRUNCATE TABLE test_table; SELECT 9; /* Check input_format_null_as_default = 1 */ SET input_format_null_as_default = 1; INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", "1", "2", "null"] ["second", "2", "null", "6"]; +INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNames ["v1", "v2", "v3", "v4"]["first", "1", "2", "null"] ["second", "2", "null", "6"]; SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; SELECT 10; /* Check Header */ TRUNCATE TABLE test_table; SET input_format_skip_unknown_fields = 1; INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "v2", "invalid_column"]["String", "UInt8", "UInt8"]["first", "1", "32"]["second", "2", "64"]; +INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNames ["v1", "v2", "invalid_column"]["first", "1", "32"]["second", "2", "64"]; SELECT * FROM test_table FORMAT JSONCompactStringsEachRow; SELECT 11; TRUNCATE TABLE test_table; INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v4", "v2", "v3"]["UInt8", "UInt8", "UInt16"]["1", "2", "3"] +INSERT INTO test_table FORMAT JSONCompactStringsEachRowWithNames ["v4", "v2", "v3"]["1", "2", "3"] SELECT * FROM test_table FORMAT JSONCompactStringsEachRowWithNamesAndTypes; +SELECT '---------'; +SELECT * FROM test_table FORMAT JSONCompactStringsEachRowWithNames; SELECT 12; /* Check Nested */ INSERT INTO test_table_2 FORMAT JSONCompactStringsEachRowWithNamesAndTypes ["v1", "n.id", "n.name"]["UInt8", "Array(UInt8)", "Array(String)"]["16", "[15, 16, 17]", "['first', 'second', 'third']"]; +INSERT INTO test_table_2 FORMAT JSONCompactStringsEachRowWithNames ["v1", "n.id", "n.name"]["16", "[15, 16, 17]", "['first', 'second', 'third']"]; SELECT * FROM test_table_2 FORMAT JSONCompactStringsEachRowWithNamesAndTypes; +SELECT '---------'; +SELECT * FROM test_table_2 FORMAT JSONCompactStringsEachRowWithNames; DROP TABLE IF EXISTS test_table; DROP TABLE IF EXISTS test_table_2; diff --git a/tests/queries/0_stateless/02097_json_strings_deserialization.reference b/tests/queries/0_stateless/02097_json_strings_deserialization.reference new file mode 100644 index 00000000000..8d7ffe54606 --- /dev/null +++ b/tests/queries/0_stateless/02097_json_strings_deserialization.reference @@ -0,0 +1,4 @@ +test\n\t\0\n test\n\t\0\n +test\n\t\0\n test\n\t\0\n +test\n\t\0\n test\n\t\0\n +test\n\t\0\n test\n\t\0\n diff --git a/tests/queries/0_stateless/02097_json_strings_deserialization.sh b/tests/queries/0_stateless/02097_json_strings_deserialization.sh new file mode 100755 index 00000000000..ae9e1ea7645 --- /dev/null +++ b/tests/queries/0_stateless/02097_json_strings_deserialization.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists test_02097" +$CLICKHOUSE_CLIENT -q "create table test_02097 (s String, f FixedString(8)) engine=Memory()" +echo -e "('test\n\t\0\n', 'test\n\t\0\n')" | $CLICKHOUSE_CLIENT -q "insert into test_02097 format Values" +$CLICKHOUSE_CLIENT -q "select * from test_02097 format JSONStringsEachRow" | $CLICKHOUSE_CLIENT -q "insert into test_02097 format JSONStringsEachRow" +$CLICKHOUSE_CLIENT -q "select * from test_02097 format JSONCompactStringsEachRow" | $CLICKHOUSE_CLIENT -q "insert into test_02097 format JSONCompactStringsEachRow" +$CLICKHOUSE_CLIENT -q "select * from test_02097" +$CLICKHOUSE_CLIENT -q "drop table test_02097" + diff --git a/tests/queries/0_stateless/02098_with_types_use_header.reference b/tests/queries/0_stateless/02098_with_types_use_header.reference new file mode 100644 index 00000000000..c1d70452d1d --- /dev/null +++ b/tests/queries/0_stateless/02098_with_types_use_header.reference @@ -0,0 +1,16 @@ +TSVWithNamesAndTypes +OK +OK +OK +CSVWithNamesAndTypes +OK +OK +OK +JSONCompactEachRowWithNamesAndTypes +OK +OK +OK +JSONCompactStringsEachRowWithNamesAndTypes +OK +OK +OK diff --git a/tests/queries/0_stateless/02098_with_types_use_header.sh b/tests/queries/0_stateless/02098_with_types_use_header.sh new file mode 100755 index 00000000000..cbeb783aed0 --- /dev/null +++ b/tests/queries/0_stateless/02098_with_types_use_header.sh @@ -0,0 +1,33 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_02098" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_02098 (x UInt32, y String, z Date) engine=Memory()" + +echo "TSVWithNamesAndTypes" +echo -e "x\ty\tz\nString\tDate\tUInt32\ntext\t2020-01-01\t1" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT TSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +echo -e "y\tz\tx\nString\tDate\tUInt32\ntext\t2020-01-01\t1" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT TSVWithNamesAndTypes" && echo 'OK' || echo 'FAIL' +echo -e "x\tz\ty\nUInt32\tString\tDate\n1\ttext\t2020-01-01" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT TSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' + + +echo "CSVWithNamesAndTypes" +echo -e "'x','y','z'\n'String','Date','UInt32'\n'text','2020-01-01',1" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +echo -e "'y','z','x'\n'String','Date','UInt32'\n'text','2020-01-01',1" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" && echo 'OK' || echo 'FAIL' +echo -e "'x','z','y'\n'UInt32','String',Date'\n1,'text','2020-01-01'" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' + + +echo "JSONCompactEachRowWithNamesAndTypes" +echo -e '["x","y","z"]\n["String","Date","UInt32"]\n["text","2020-01-01",1]' | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactEachRowWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +echo -e '["y","z","x"]\n["String","Date","UInt32"]\n["text","2020-01-01",1]' | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactEachRowWithNamesAndTypes" && echo 'OK' || echo 'FAIL' +echo -e '["x","z","y"]\n["UInt32", "String", "Date"]\n[1, "text","2020-01-01"]' | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactEachRowWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' + +echo "JSONCompactStringsEachRowWithNamesAndTypes" +echo -e '["x","y","z"]\n["String","Date","UInt32"]\n["text","2020-01-01","1"]' | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactStringsEachRowWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +echo -e '["y","z","x"]\n["String","Date","UInt32"]\n["text","2020-01-01","1"]' | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactStringsEachRowWithNamesAndTypes" && echo 'OK' || echo 'FAIL' +echo -e '["x","z","y"]\n["UInt32", "String", "Date"]\n["1", "text","2020-01-01"]' | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactStringsEachRowWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' + +$CLICKHOUSE_CLIENT -q "DROP TABLE test_02098" diff --git a/tests/queries/0_stateless/02099_tsv_raw_format.reference b/tests/queries/0_stateless/02099_tsv_raw_format.reference new file mode 100644 index 00000000000..de46cf8dff7 --- /dev/null +++ b/tests/queries/0_stateless/02099_tsv_raw_format.reference @@ -0,0 +1,113 @@ +TSVRaw +0 0 1970-01-01 +1 1 1970-01-02 +2 2 1970-01-03 +0 0 1970-01-01 +1 1 1970-01-02 +2 2 1970-01-03 +0 0 1970-01-01 +1 1 1970-01-02 +2 2 1970-01-03 +TSVRawWithNames +number string date +0 0 1970-01-01 +1 1 1970-01-02 +2 2 1970-01-03 +0 0 1970-01-01 +1 1 1970-01-02 +2 2 1970-01-03 +0 0 1970-01-01 +1 1 1970-01-02 +2 2 1970-01-03 +TSVRawWithNamesAndTypes +number string date +UInt64 String Date +0 0 1970-01-01 +1 1 1970-01-02 +2 2 1970-01-03 +0 0 1970-01-01 +1 1 1970-01-02 +2 2 1970-01-03 +0 0 1970-01-01 +1 1 1970-01-02 +2 2 1970-01-03 +TabSeparatedRaw +0 0 1970-01-01 +1 1 1970-01-02 +2 2 1970-01-03 +0 0 1970-01-01 +1 1 1970-01-02 +2 2 1970-01-03 +0 0 1970-01-01 +1 1 1970-01-02 +2 2 1970-01-03 +TabSeparatedRawWithNames +number string date +0 0 1970-01-01 +1 1 1970-01-02 +2 2 1970-01-03 +0 0 1970-01-01 +1 1 1970-01-02 +2 2 1970-01-03 +0 0 1970-01-01 +1 1 1970-01-02 +2 2 1970-01-03 +TabSeparatedRawWithNamesAndTypes +number string date +UInt64 String Date +0 0 1970-01-01 +1 1 1970-01-02 +2 2 1970-01-03 +0 0 1970-01-01 +1 1 1970-01-02 +2 2 1970-01-03 +0 0 1970-01-01 +1 1 1970-01-02 +2 2 1970-01-03 +0 +\N +2 +\N +0 +\N +2 +\N +0 +\N +2 +\N +0 +\N +2 +\N +0 +\N +2 +\N +0 +\N +2 +\N +0 +\N +2 +\N +0 +\N +2 +\N +0 +\N +2 +\N +0 +\N +2 +\N +0 +\N +2 +\N +nSome text +b1cad4eb4be08a40387c9de70d02fcc2 - +b1cad4eb4be08a40387c9de70d02fcc2 - diff --git a/tests/queries/0_stateless/02099_tsv_raw_format.sh b/tests/queries/0_stateless/02099_tsv_raw_format.sh new file mode 100755 index 00000000000..ef59e399bdf --- /dev/null +++ b/tests/queries/0_stateless/02099_tsv_raw_format.sh @@ -0,0 +1,59 @@ +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_02099" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_02099 (number UInt64, string String, date Date) ENGINE=Memory()" + +FORMATS=('TSVRaw' 'TSVRawWithNames' 'TSVRawWithNamesAndTypes' 'TabSeparatedRaw' 'TabSeparatedRawWithNames' 'TabSeparatedRawWithNamesAndTypes') + +for format in "${FORMATS[@]}" +do + echo $format + $CLICKHOUSE_CLIENT -q "INSERT INTO test_02099 SELECT number, toString(number), toDate(number) FROM numbers(3)" + $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02099 FORMAT $format" + + $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02099 FORMAT $format" | $CLICKHOUSE_CLIENT -q "INSERT INTO test_02099 FORMAT $format" + $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02099" + + $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02099" +done + +$CLICKHOUSE_CLIENT -q "DROP TABLE test_02099" + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_nullable_02099" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_nullable_02099 ENGINE=Memory() AS SELECT number % 2 ? NULL : number from numbers(4)"; + +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_nullable_02099 FORMAT TSVRaw" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_nullable_02099 FORMAT TSVRaw" | $CLICKHOUSE_CLIENT -q "INSERT INTO test_nullable_02099 FORMAT TSVRaw" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_nullable_02099" + + +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_nullable_02099 FORMAT TSV" | $CLICKHOUSE_CLIENT -q "INSERT INTO test_nullable_02099 FORMAT TSVRaw" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_nullable_02099 FORMAT TSVRaw" | $CLICKHOUSE_CLIENT -q "INSERT INTO test_nullable_02099 FORMAT TSV" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_nullable_02099" + +$CLICKHOUSE_CLIENT -q "DROP TABLE test_nullable_02099" + + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_nullable_string_02099" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_nullable_string_02099 (s Nullable(String)) ENGINE=Memory()"; + +echo 'nSome text' | $CLICKHOUSE_CLIENT -q "INSERT INTO test_nullable_string_02099 FORMAT TSVRaw" + +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_nullable_string_02099" +$CLICKHOUSE_CLIENT -q "DROP TABLE test_nullable_string_02099" + + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_parallel_parsing_02099" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_parallel_parsing_02099 (x UInt64, a Array(UInt64), s String) ENGINE=Memory()"; +$CLICKHOUSE_CLIENT -q "SELECT number AS x, range(number % 50) AS a, toString(a) AS s FROM numbers(1000000) FORMAT TSVRaw" | $CLICKHOUSE_CLIENT --input_format_parallel_parsing=0 -q "INSERT INTO test_parallel_parsing_02099 FORMAT TSVRaw" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_parallel_parsing_02099 ORDER BY x" | md5sum + +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_parallel_parsing_02099" + +$CLICKHOUSE_CLIENT -q "SELECT number AS x, range(number % 50) AS a, toString(a) AS s FROM numbers(1000000) FORMAT TSVRaw" | $CLICKHOUSE_CLIENT --input_format_parallel_parsing=1 -q "INSERT INTO test_parallel_parsing_02099 FORMAT TSVRaw" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_parallel_parsing_02099 ORDER BY x" | md5sum + +$CLICKHOUSE_CLIENT -q "DROP TABLE test_parallel_parsing_02099" + diff --git a/tests/queries/0_stateless/02100_low_cardinality_nullable_null_default.reference b/tests/queries/0_stateless/02100_low_cardinality_nullable_null_default.reference new file mode 100644 index 00000000000..12b4d6ad854 --- /dev/null +++ b/tests/queries/0_stateless/02100_low_cardinality_nullable_null_default.reference @@ -0,0 +1,14 @@ +CSV +\N +TSV +\N +TSVRaw +\N +TSKV +\N +JSONCompactEachRow +\N +JSONEachRow +\N +Values +\N diff --git a/tests/queries/0_stateless/02100_low_cardinality_nullable_null_default.sh b/tests/queries/0_stateless/02100_low_cardinality_nullable_null_default.sh new file mode 100755 index 00000000000..d380e784229 --- /dev/null +++ b/tests/queries/0_stateless/02100_low_cardinality_nullable_null_default.sh @@ -0,0 +1,21 @@ +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_02100" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_02100 (x LowCardinality(Nullable(String)) DEFAULT 'default') ENGINE=Memory()" + +FORMATS=('CSV' 'TSV' 'TSVRaw' 'TSKV' 'JSONCompactEachRow' 'JSONEachRow' 'Values') + +for format in "${FORMATS[@]}" +do + echo $format + $CLICKHOUSE_CLIENT -q "SELECT NULL as x FORMAT $format" | $CLICKHOUSE_CLIENT -q "INSERT INTO test_02100 FORMAT $format" + + $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02100" + + $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02100" +done + +$CLICKHOUSE_CLIENT -q "DROP TABLE test_02100" + diff --git a/tests/queries/0_stateless/02101_empty_as_default_and_omitted_fields.reference b/tests/queries/0_stateless/02101_empty_as_default_and_omitted_fields.reference new file mode 100644 index 00000000000..61444c7a238 --- /dev/null +++ b/tests/queries/0_stateless/02101_empty_as_default_and_omitted_fields.reference @@ -0,0 +1,16 @@ +TSV +1 42 +2 0 +3 42 +4 0 +CSV +1 42 +2 0 +3 42 +4 0 +JSONEachRow +1 42 +2 0 +JSONCompactEachRow +1 42 +2 0 diff --git a/tests/queries/0_stateless/02101_empty_as_default_and_omitted_fields.sh b/tests/queries/0_stateless/02101_empty_as_default_and_omitted_fields.sh new file mode 100755 index 00000000000..344982bcd84 --- /dev/null +++ b/tests/queries/0_stateless/02101_empty_as_default_and_omitted_fields.sh @@ -0,0 +1,39 @@ +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_02101" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_02101 (x UInt64, y UInt64 DEFAULT 42) ENGINE=Memory()" + +echo 'TSV' +echo -e 'x\ty\n1\t' | $CLICKHOUSE_CLIENT --input_format_tsv_empty_as_default=1 --input_format_defaults_for_omitted_fields=1 -q "INSERT INTO test_02101 FORMAT TSVWithNames" +echo -e 'x\ty\n2\t' | $CLICKHOUSE_CLIENT --input_format_tsv_empty_as_default=1 --input_format_defaults_for_omitted_fields=0 -q "INSERT INTO test_02101 FORMAT TSVWithNames" +echo -e 'x\tz\n3\t123' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --input_format_skip_unknown_fields=1 -q "INSERT INTO test_02101 FORMAT TSVWithNames" +echo -e 'x\tz\n4\t123' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_skip_unknown_fields=1 -q "INSERT INTO test_02101 FORMAT TSVWithNames" + +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02101 ORDER BY x" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02101" + +echo 'CSV' +echo -e '"x","y"\n1,' | $CLICKHOUSE_CLIENT --input_format_csv_empty_as_default=1 --input_format_defaults_for_omitted_fields=1 -q "INSERT INTO test_02101 FORMAT CSVWithNames" +echo -e '"x","y"\n2,' | $CLICKHOUSE_CLIENT --input_format_csv_empty_as_default=1 --input_format_defaults_for_omitted_fields=0 -q "INSERT INTO test_02101 FORMAT CSVWithNames" +echo -e '"x","z"\n3,123' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --input_format_skip_unknown_fields=1 -q "INSERT INTO test_02101 FORMAT CSVWithNames" +echo -e '"x","z"\n4,123' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_skip_unknown_fields=1 -q "INSERT INTO test_02101 FORMAT CSVWithNames" + +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02101 ORDER BY x" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02101" + +echo 'JSONEachRow' +echo -e '{"x" : 1, "z" : 123}' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --input_format_skip_unknown_fields=1 -q "INSERT INTO test_02101 FORMAT JSONEachRow" +echo -e '{"x" : 2, "z" : 123}' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_skip_unknown_fields=1 -q "INSERT INTO test_02101 FORMAT JSONEachRow" + +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02101 ORDER BY x" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02101" + +echo 'JSONCompactEachRow' +echo -e '["x", "z"], [1, 123]' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --input_format_skip_unknown_fields=1 -q "INSERT INTO test_02101 FORMAT JSONCompactEachRowWithNames" +echo -e '["x", "z"], [2, 123]' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_skip_unknown_fields=1 -q "INSERT INTO test_02101 FORMAT JSONCompactEachRowWithNames" + +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02101 ORDER BY x" +$CLICKHOUSE_CLIENT -q "DROP TABLE test_02101" + diff --git a/tests/queries/1_stateful/00159_parallel_formatting_json_and_friends.reference b/tests/queries/1_stateful/00159_parallel_formatting_json_and_friends.reference index 6d663c33057..7ad5359a30e 100644 --- a/tests/queries/1_stateful/00159_parallel_formatting_json_and_friends.reference +++ b/tests/queries/1_stateful/00159_parallel_formatting_json_and_friends.reference @@ -1,12 +1,28 @@ JSONEachRow, false -7251839681e559f5a92db107571bb357 - +e0a3c9978a92a277f2fff4664f3c1749 - JSONEachRow, true -7251839681e559f5a92db107571bb357 - +e0a3c9978a92a277f2fff4664f3c1749 - JSONCompactEachRow, false -ba1081a754a06ef6563840b2d8d4d327 - +0c1efbbc25a5bd90a2ecea559d283667 - JSONCompactEachRow, true -ba1081a754a06ef6563840b2d8d4d327 - +0c1efbbc25a5bd90a2ecea559d283667 - +JSONCompactStringsEachRow, false +0c1efbbc25a5bd90a2ecea559d283667 - +JSONCompactStringsEachRow, true +0c1efbbc25a5bd90a2ecea559d283667 - +JSONCompactEachRowWithNames, false +b9e4f8ecadbb650245d1762f4187ee0a - +JSONCompactEachRowWithNames, true +b9e4f8ecadbb650245d1762f4187ee0a - +JSONCompactStringsEachRowWithNames, false +b9e4f8ecadbb650245d1762f4187ee0a - +JSONCompactStringsEachRowWithNames, true +b9e4f8ecadbb650245d1762f4187ee0a - +JSONCompactEachRowWithNamesAndTypes, false +8b41f7375999b53d4c9607398456fe5b - +JSONCompactEachRowWithNamesAndTypes, true +8b41f7375999b53d4c9607398456fe5b - JSONCompactStringsEachRowWithNamesAndTypes, false -31ded3cd9971b124450fb5a44a8bce63 - +8b41f7375999b53d4c9607398456fe5b - JSONCompactStringsEachRowWithNamesAndTypes, true -31ded3cd9971b124450fb5a44a8bce63 - +8b41f7375999b53d4c9607398456fe5b - diff --git a/tests/queries/1_stateful/00159_parallel_formatting_json_and_friends.sh b/tests/queries/1_stateful/00159_parallel_formatting_json_and_friends.sh index 5d54328e45d..f6c87eabfde 100755 --- a/tests/queries/1_stateful/00159_parallel_formatting_json_and_friends.sh +++ b/tests/queries/1_stateful/00159_parallel_formatting_json_and_friends.sh @@ -6,15 +6,15 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -FORMATS=('JSONEachRow' 'JSONCompactEachRow' 'JSONCompactStringsEachRowWithNamesAndTypes') +FORMATS=('JSONEachRow' 'JSONCompactEachRow' 'JSONCompactStringsEachRow' 'JSONCompactEachRowWithNames' 'JSONCompactStringsEachRowWithNames' 'JSONCompactEachRowWithNamesAndTypes' 'JSONCompactStringsEachRowWithNamesAndTypes') for format in "${FORMATS[@]}" do echo "$format, false"; $CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \ - "SELECT ClientEventTime::DateTime('Europe/Moscow') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum + "SELECT ClientEventTime::DateTime('Europe/Moscow') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c LIMIT 3000000 Format $format" | md5sum echo "$format, true"; $CLICKHOUSE_CLIENT --output_format_parallel_formatting=true -q \ - "SELECT ClientEventTime::DateTime('Europe/Moscow') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c Format $format" | md5sum + "SELECT ClientEventTime::DateTime('Europe/Moscow') as a, MobilePhoneModel as b, ClientIP6 as c FROM test.hits ORDER BY a, b, c LIMIT 3000000 Format $format" | md5sum done diff --git a/tests/queries/1_stateful/00167_parallel_parsing_with_names_and_types.reference b/tests/queries/1_stateful/00167_parallel_parsing_with_names_and_types.reference new file mode 100644 index 00000000000..0c0367694b2 --- /dev/null +++ b/tests/queries/1_stateful/00167_parallel_parsing_with_names_and_types.reference @@ -0,0 +1,20 @@ +TSVWithNamesAndTypes, false +7c1feeaae418e502d66fcc8e31946f2e - +TSVWithNamesAndTypes, true +7c1feeaae418e502d66fcc8e31946f2e - +CSVWithNamesAndTypes, false +7c1feeaae418e502d66fcc8e31946f2e - +CSVWithNamesAndTypes, true +7c1feeaae418e502d66fcc8e31946f2e - +JSONStringsEachRow, false +7c1feeaae418e502d66fcc8e31946f2e - +JSONStringsEachRow, true +7c1feeaae418e502d66fcc8e31946f2e - +JSONCompactEachRowWithNamesAndTypes, false +7c1feeaae418e502d66fcc8e31946f2e - +JSONCompactEachRowWithNamesAndTypes, true +7c1feeaae418e502d66fcc8e31946f2e - +JSONCompactStringsEachRowWithNamesAndTypes, false +7c1feeaae418e502d66fcc8e31946f2e - +JSONCompactStringsEachRowWithNamesAndTypes, true +7c1feeaae418e502d66fcc8e31946f2e - diff --git a/tests/queries/1_stateful/00167_parallel_parsing_with_names_and_types.sh b/tests/queries/1_stateful/00167_parallel_parsing_with_names_and_types.sh new file mode 100755 index 00000000000..9fdca20d097 --- /dev/null +++ b/tests/queries/1_stateful/00167_parallel_parsing_with_names_and_types.sh @@ -0,0 +1,32 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +FORMATS=('TSVWithNamesAndTypes' 'CSVWithNamesAndTypes' 'JSONStringsEachRow' 'JSONCompactEachRowWithNamesAndTypes' 'JSONCompactStringsEachRowWithNamesAndTypes') +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names" + +for format in "${FORMATS[@]}" +do + # Columns are permuted + $CLICKHOUSE_CLIENT -q "CREATE TABLE parsing_with_names(c FixedString(16), a DateTime('Europe/Moscow'), b String) ENGINE=Memory()" + + echo "$format, false"; + $CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \ + "SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Europe/Moscow') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 5000 Format $format" | \ + $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=false -q "INSERT INTO parsing_with_names FORMAT $format SETTINGS input_format_null_as_default=0" + + $CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum + $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names" + + + $CLICKHOUSE_CLIENT -q "CREATE TABLE parsing_with_names(c FixedString(16), a DateTime('Europe/Moscow'), b String) ENGINE=Memory()" + echo "$format, true"; + $CLICKHOUSE_CLIENT --output_format_parallel_formatting=false -q \ + "SELECT URLRegions as d, toTimeZone(ClientEventTime, 'Europe/Moscow') as a, MobilePhoneModel as b, ParamPrice as e, ClientIP6 as c FROM test.hits LIMIT 5000 Format $format" | \ + $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_parallel_parsing=true -q "INSERT INTO parsing_with_names FORMAT $format SETTINGS input_format_null_as_default=0" + + $CLICKHOUSE_CLIENT -q "SELECT * FROM parsing_with_names;" | md5sum + $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parsing_with_names" +done From 400a31db06c7cc497d6fde7dbb8f4a4b43efb9d4 Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 14 Oct 2021 05:06:21 +0000 Subject: [PATCH 156/919] reduce poll timeout fix --- src/Storages/FileLog/DirectoryWatcherBase.cpp | 5 ++--- src/Storages/FileLog/ReadBufferFromFileLog.cpp | 8 ++++++-- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index 4f6e429e1fd..9ea2e7e1824 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -16,8 +16,7 @@ namespace ErrorCodes extern const int IO_SETUP_ERROR; } -static constexpr int event_size = sizeof(struct inotify_event); -static constexpr int buffer_size = 1024 * (NAME_MAX + event_size + 1); +static constexpr int buffer_size = 4096; DirectoryWatcherBase::DirectoryWatcherBase( FileLogDirectoryWatcher & owner_, const std::string & path_, ContextPtr context_, int event_mask_) @@ -64,7 +63,7 @@ void DirectoryWatcherBase::watchFunc() pfd.events = POLLIN; while (!stopped) { - if (poll(&pfd, 1, 5000) > 0 && pfd.revents & POLLIN) + if (poll(&pfd, 1, 500) > 0 && pfd.revents & POLLIN) { int n = read(fd, buffer.data(), buffer.size()); int i = 0; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 745d8c46a63..f935decd7f8 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -110,16 +110,20 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Ifstream for file {} does not initialized", file_meta.file_name); auto & reader = file_ctx.reader.value(); + StorageFileLog::assertStreamGood(reader); + Record record; while (read_records_size < need_records_size && static_cast(reader.tellg()) < file_meta.last_open_end) { /// Need to get offset before reading record from stream record.offset = reader.tellg(); - record.file_name = file_name; - StorageFileLog::assertStreamGood(reader); + record.file_name = file_name; + + std::getline(reader, record.data); + StorageFileLog::assertStreamGood(reader); new_records.emplace_back(record); ++read_records_size; From 2979c0dee787318ac7f8c49639414a815dd1b30d Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 14 Oct 2021 15:05:49 +0300 Subject: [PATCH 157/919] Support columns mapping and types checking in RowBinaryWithNamesAndTypes format --- src/Core/Settings.h | 8 +- .../Formats/Impl/BinaryRowInputFormat.cpp | 97 +++++++++++-------- .../Formats/Impl/BinaryRowInputFormat.h | 28 ++++-- .../Formats/Impl/CSVRowInputFormat.cpp | 2 +- .../Formats/Impl/CSVRowInputFormat.h | 11 ++- .../Impl/JSONCompactEachRowRowInputFormat.cpp | 9 +- .../Impl/JSONCompactEachRowRowInputFormat.h | 10 +- .../Impl/TabSeparatedRowInputFormat.cpp | 2 +- .../Formats/Impl/TabSeparatedRowInputFormat.h | 10 +- .../RowInputFormatWithNamesAndTypes.cpp | 12 +-- .../Formats/RowInputFormatWithNamesAndTypes.h | 19 ++-- .../02098_with_types_use_header.sh | 24 ++--- ..._row_binary_with_names_and_types.reference | 8 ++ .../02102_row_binary_with_names_and_types.sh | 52 ++++++++++ 14 files changed, 201 insertions(+), 91 deletions(-) create mode 100644 tests/queries/0_stateless/02102_row_binary_with_names_and_types.reference create mode 100755 tests/queries/0_stateless/02102_row_binary_with_names_and_types.sh diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9d1f8163dfe..50486a30cae 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -555,11 +555,11 @@ class IColumn; M(Bool, input_format_csv_unquoted_null_literal_as_null, false, "Consider unquoted NULL literal as \\N", 0) \ M(Bool, input_format_csv_enum_as_number, false, "Treat inserted enum values in CSV formats as enum indices \\N", 0) \ M(Bool, input_format_csv_arrays_as_nested_csv, false, R"(When reading Array from CSV, expect that its elements were serialized in nested CSV and then put into string. Example: "[""Hello"", ""world"", ""42"""" TV""]". Braces around array can be omitted.)", 0) \ - M(Bool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow, CSVWithNames, TSVWithNames and TSKV formats).", 0) \ - M(Bool, input_format_with_names_use_header, true, "For TSVWithNames and CSVWithNames input formats this controls whether format parser is to assume that column data appear in the input exactly as they are specified in the header.", 0) \ - M(Bool, input_format_with_types_use_header, true, "For TSVWithNamesTypes and CSVWithNamesAndTypes input formats this controls whether format parser should check if data types from the input match data types from the header.", 0) \ + M(Bool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow, -WithNames, -WithNamesAndTypes and TSKV formats).", 0) \ + M(Bool, input_format_with_names_use_header, true, "For -WithNames input formats this controls whether format parser is to assume that column data appear in the input exactly as they are specified in the header.", 0) \ + M(Bool, input_format_with_types_use_header, true, "For -WithNamesAndTypes input formats this controls whether format parser should check if data types from the input match data types from the header.", 0) \ M(Bool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).", 0) \ - M(Bool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, JSONCompactEachRow, CSV and TSV formats).", IMPORTANT) \ + M(Bool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, -WithNames, -WithNamesAndTypes formats).", IMPORTANT) \ M(Bool, input_format_csv_empty_as_default, true, "Treat empty fields in CSV input as default values.", 0) \ M(Bool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \ M(Bool, input_format_tsv_enum_as_number, false, "Treat inserted enum values in TSV formats as enum indices \\N", 0) \ diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp index c122b9eea1a..d3cea169eab 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp @@ -2,59 +2,80 @@ #include #include #include +#include namespace DB { -BinaryRowInputFormat::BinaryRowInputFormat(ReadBuffer & in_, Block header, Params params_, bool with_names_, bool with_types_) - : IRowInputFormat(std::move(header), in_, params_), with_names(with_names_), with_types(with_types_) +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +BinaryRowInputFormat::BinaryRowInputFormat(ReadBuffer & in_, Block header, Params params_, bool with_names_and_types, const FormatSettings & format_settings_) + : RowInputFormatWithNamesAndTypes(std::move(header), in_, std::move(params_), with_names_and_types, with_names_and_types, format_settings_) { } -bool BinaryRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) +std::vector BinaryRowInputFormat::readHeaderRow() { - if (in->eof()) - return false; + std::vector fields; + String field; + for (size_t i = 0; i < read_columns; ++i) + { + readStringBinary(field, *in); + fields.push_back(field); + } + return fields; +} - size_t num_columns = columns.size(); - for (size_t i = 0; i < num_columns; ++i) - serializations[i]->deserializeBinary(*columns[i], *in); +std::vector BinaryRowInputFormat::readNames() +{ + readVarUInt(read_columns, *in); + return readHeaderRow(); +} +std::vector BinaryRowInputFormat::readTypes() +{ + auto types = readHeaderRow(); + for (const auto & type_name : types) + read_data_types.push_back(DataTypeFactory::instance().get(type_name)); + return types; +} + +bool BinaryRowInputFormat::readField(IColumn & column, const DataTypePtr & /*type*/, const SerializationPtr & serialization, bool /*is_last_file_column*/, const String & /*column_name*/) +{ + serialization->deserializeBinary(column, *in); return true; } - -void BinaryRowInputFormat::readPrefix() +void BinaryRowInputFormat::skipHeaderRow() { - /// NOTE: The header is completely ignored. This can be easily improved. - - UInt64 columns = 0; String tmp; - - if (with_names || with_types) - { - readVarUInt(columns, *in); - } - - if (with_names) - { - for (size_t i = 0; i < columns; ++i) - { - readStringBinary(tmp, *in); - } - } - - if (with_types) - { - for (size_t i = 0; i < columns; ++i) - { - readStringBinary(tmp, *in); - } - } + for (size_t i = 0; i < read_columns; ++i) + readStringBinary(tmp, *in); } +void BinaryRowInputFormat::skipNames() +{ + readVarUInt(read_columns, *in); + skipHeaderRow(); +} + +void BinaryRowInputFormat::skipTypes() +{ + skipHeaderRow(); +} + +void BinaryRowInputFormat::skipField(size_t file_column) +{ + if (file_column >= read_data_types.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot skip field in Binary format, because it's type is unknown"); + Field field; + read_data_types[file_column]->getDefaultSerialization()->deserializeBinary(field, *in); +} void registerInputFormatRowBinary(FormatFactory & factory) { @@ -62,18 +83,18 @@ void registerInputFormatRowBinary(FormatFactory & factory) ReadBuffer & buf, const Block & sample, const IRowInputFormat::Params & params, - const FormatSettings &) + const FormatSettings & settings) { - return std::make_shared(buf, sample, params, false, false); + return std::make_shared(buf, sample, params, false, settings); }); factory.registerInputFormat("RowBinaryWithNamesAndTypes", []( ReadBuffer & buf, const Block & sample, const IRowInputFormat::Params & params, - const FormatSettings &) + const FormatSettings & settings) { - return std::make_shared(buf, sample, params, true, true); + return std::make_shared(buf, sample, params, true, settings); }); } diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.h b/src/Processors/Formats/Impl/BinaryRowInputFormat.h index e96a516c1a7..020be4f4db6 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.h +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -12,19 +13,32 @@ class ReadBuffer; /** A stream for inputting data in a binary line-by-line format. */ -class BinaryRowInputFormat : public IRowInputFormat +class BinaryRowInputFormat : public RowInputFormatWithNamesAndTypes { public: - BinaryRowInputFormat(ReadBuffer & in_, Block header, Params params_, bool with_names_, bool with_types_); - - bool readRow(MutableColumns & columns, RowReadExtension &) override; - void readPrefix() override; + BinaryRowInputFormat(ReadBuffer & in_, Block header, Params params_, bool with_names_and_types, const FormatSettings & format_settings_); String getName() const override { return "BinaryRowInputFormat"; } + /// RowInputFormatWithNamesAndTypes implements logic with DiagnosticInfo, but + /// in this format we cannot provide any DiagnosticInfo, because here we have + /// just binary data. + std::string getDiagnosticInfo() override { return {}; } + private: - bool with_names; - bool with_types; + bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override; + void skipField(size_t file_column) override; + + void skipNames() override; + void skipTypes() override; + void skipHeaderRow(); + + std::vector readNames() override; + std::vector readTypes() override; + std::vector readHeaderRow(); + + DataTypes read_data_types; + UInt64 read_columns; }; } diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 824c33858d6..12bea39b5cd 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -106,7 +106,7 @@ void CSVRowInputFormat::skipRowEndDelimiter() skipEndOfLine(*in); } -void CSVRowInputFormat::skipRow() +void CSVRowInputFormat::skipHeaderRow() { do { diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index 2e036fa2318..f239464485a 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -38,14 +38,19 @@ private: bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override; - void skipField(const String & /*column_name*/) override { skipField(); } + void skipField(size_t /*file_column*/) override { skipField(); } void skipField(); - void skipRow() override; + void skipHeaderRow() ; + void skipNames() override { skipHeaderRow(); } + void skipTypes() override { skipHeaderRow(); } void skipFieldDelimiter() override; void skipRowEndDelimiter() override; - Names readHeaderRow() override; + std::vector readHeaderRow(); + std::vector readNames() override { return readHeaderRow(); } + std::vector readTypes() override { return readHeaderRow(); } + String readFieldIntoString(); }; diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index c551597ca5f..1c5843ce97b 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include #include @@ -64,13 +63,13 @@ String JSONCompactEachRowRowInputFormat::readFieldIntoString() return field; } -void JSONCompactEachRowRowInputFormat::skipField(const String & column_name) +void JSONCompactEachRowRowInputFormat::skipField(size_t file_column) { skipWhitespaceIfAny(*in); - skipJSONField(*in, column_name); + skipJSONField(*in, column_mapping->names_of_columns[file_column]); } -void JSONCompactEachRowRowInputFormat::skipRow() +void JSONCompactEachRowRowInputFormat::skipHeaderRow() { skipRowStartDelimiter(); size_t i = 0; @@ -78,7 +77,7 @@ void JSONCompactEachRowRowInputFormat::skipRow() { if (i >= column_mapping->names_of_columns.size()) throw Exception(ErrorCodes::INCORRECT_DATA, "The number of columns in a row differs from the number of column names"); - skipField(column_mapping->names_of_columns[i++]); + skipField(i++); skipWhitespaceIfAny(*in); } while (checkChar(',', *in)); diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h index fe8fc2acda3..373eb04f06c 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h @@ -45,13 +45,17 @@ private: bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override; - void skipField(const String & column_name) override; - void skipRow() override; + void skipField(size_t file_column) override; + void skipHeaderRow(); + void skipNames() override { skipHeaderRow(); } + void skipTypes() override { skipHeaderRow(); } void skipRowStartDelimiter() override; void skipFieldDelimiter() override; void skipRowEndDelimiter() override; - Names readHeaderRow() override; + std::vector readHeaderRow(); + std::vector readNames() override { return readHeaderRow(); } + std::vector readTypes() override { return readHeaderRow(); } String readFieldIntoString(); bool yield_strings; diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index ec6dfef4f0c..ac94793a511 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -72,7 +72,7 @@ void TabSeparatedRowInputFormat::skipField() readEscapedStringInto(null_sink, *in); } -void TabSeparatedRowInputFormat::skipRow() +void TabSeparatedRowInputFormat::skipHeaderRow() { do { diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h index 31e6e12400a..11a788bc900 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h @@ -30,13 +30,17 @@ private: bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override; - void skipField(const String & /*column_name*/) override { skipField(); } + void skipField(size_t /*file_column*/) override { skipField(); } void skipField(); - void skipRow() override; + void skipHeaderRow(); + void skipNames() override { skipHeaderRow(); } + void skipTypes() override { skipHeaderRow(); } void skipFieldDelimiter() override; void skipRowEndDelimiter() override; - Names readHeaderRow() override; + std::vector readHeaderRow(); + std::vector readNames() override { return readHeaderRow(); } + std::vector readTypes() override { return readHeaderRow(); } String readFieldIntoString(); void checkNullValueForNonNullable(DataTypePtr type) override; diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index 614ec27c0d5..bb1d95c0634 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -89,7 +89,7 @@ void RowInputFormatWithNamesAndTypes::readPrefix() if (format_settings.with_names_use_header) { std::vector read_columns(data_types.size(), false); - auto column_names = readHeaderRow(); + auto column_names = readNames(); for (const auto & name : column_names) addInputColumn(name, read_columns); @@ -102,7 +102,7 @@ void RowInputFormatWithNamesAndTypes::readPrefix() else { setupAllColumnsByTableSchema(); - skipRow(); + skipNames(); } } else if (!column_mapping->is_set) @@ -112,7 +112,7 @@ void RowInputFormatWithNamesAndTypes::readPrefix() { if (format_settings.with_types_use_header) { - auto types = readHeaderRow(); + auto types = readTypes(); if (types.size() != column_mapping->column_indexes_for_input_fields.size()) throw Exception( ErrorCodes::INCORRECT_DATA, @@ -133,7 +133,7 @@ void RowInputFormatWithNamesAndTypes::readPrefix() } } else - skipRow(); + skipTypes(); } } @@ -167,7 +167,7 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE is_last_file_column, column_mapping->names_of_columns[file_column]); else - skipField(column_mapping->names_of_columns[file_column]); + skipField(file_column); if (!is_last_file_column) skipFieldDelimiter(); @@ -202,7 +202,7 @@ void RowInputFormatWithNamesAndTypes::tryDeserializeField(const DataTypePtr & ty } else { - skipField(column_mapping->names_of_columns[file_column]); + skipField(file_column); } } diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index d9413b3a9bc..53a73be1818 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -28,21 +28,24 @@ protected: /// Return false if there was no real value and we inserted default value. virtual bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) = 0; - virtual void skipField(const String & column_name) = 0; - virtual void skipRow() = 0; + virtual void skipField(size_t file_column) = 0; + virtual void skipNames() = 0; + virtual void skipTypes() = 0; virtual void skipRowStartDelimiter() {} virtual void skipFieldDelimiter() {} virtual void skipRowEndDelimiter() {} /// Methods for parsing with diagnostic info. - virtual void checkNullValueForNonNullable(DataTypePtr /*type*/) {} - virtual bool parseRowStartWithDiagnosticInfo(WriteBuffer & /*out*/) { return true; } - virtual bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) = 0; - virtual bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) = 0; + virtual void checkNullValueForNonNullable(DataTypePtr) {} + virtual bool parseRowStartWithDiagnosticInfo(WriteBuffer &) { return true; } + virtual bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer &) { return true; } + virtual bool parseRowEndWithDiagnosticInfo(WriteBuffer &) { return true;} + bool isGarbageAfterField(size_t, ReadBuffer::Position) override {return false; } + + virtual std::vector readNames() = 0; + virtual std::vector readTypes() = 0; - /// Read the list of names or types. - virtual std::vector readHeaderRow() = 0; const FormatSettings format_settings; DataTypes data_types; diff --git a/tests/queries/0_stateless/02098_with_types_use_header.sh b/tests/queries/0_stateless/02098_with_types_use_header.sh index cbeb783aed0..d72adf5e681 100755 --- a/tests/queries/0_stateless/02098_with_types_use_header.sh +++ b/tests/queries/0_stateless/02098_with_types_use_header.sh @@ -9,25 +9,25 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_02098" $CLICKHOUSE_CLIENT -q "CREATE TABLE test_02098 (x UInt32, y String, z Date) engine=Memory()" echo "TSVWithNamesAndTypes" -echo -e "x\ty\tz\nString\tDate\tUInt32\ntext\t2020-01-01\t1" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT TSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' -echo -e "y\tz\tx\nString\tDate\tUInt32\ntext\t2020-01-01\t1" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT TSVWithNamesAndTypes" && echo 'OK' || echo 'FAIL' -echo -e "x\tz\ty\nUInt32\tString\tDate\n1\ttext\t2020-01-01" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT TSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +echo -e "x\ty\tz\nString\tDate\tUInt32\ntext\t2020-01-01\t1" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT TSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +echo -e "y\tz\tx\nString\tDate\tUInt32\ntext\t2020-01-01\t1" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT TSVWithNamesAndTypes" && echo 'OK' || echo 'FAIL' +echo -e "x\tz\ty\nUInt32\tString\tDate\n1\ttext\t2020-01-01" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT TSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' echo "CSVWithNamesAndTypes" -echo -e "'x','y','z'\n'String','Date','UInt32'\n'text','2020-01-01',1" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' -echo -e "'y','z','x'\n'String','Date','UInt32'\n'text','2020-01-01',1" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" && echo 'OK' || echo 'FAIL' -echo -e "'x','z','y'\n'UInt32','String',Date'\n1,'text','2020-01-01'" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +echo -e "'x','y','z'\n'String','Date','UInt32'\n'text','2020-01-01',1" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +echo -e "'y','z','x'\n'String','Date','UInt32'\n'text','2020-01-01',1" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" && echo 'OK' || echo 'FAIL' +echo -e "'x','z','y'\n'UInt32','String',Date'\n1,'text','2020-01-01'" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' echo "JSONCompactEachRowWithNamesAndTypes" -echo -e '["x","y","z"]\n["String","Date","UInt32"]\n["text","2020-01-01",1]' | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactEachRowWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' -echo -e '["y","z","x"]\n["String","Date","UInt32"]\n["text","2020-01-01",1]' | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactEachRowWithNamesAndTypes" && echo 'OK' || echo 'FAIL' -echo -e '["x","z","y"]\n["UInt32", "String", "Date"]\n[1, "text","2020-01-01"]' | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactEachRowWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +echo -e '["x","y","z"]\n["String","Date","UInt32"]\n["text","2020-01-01",1]' | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactEachRowWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +echo -e '["y","z","x"]\n["String","Date","UInt32"]\n["text","2020-01-01",1]' | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactEachRowWithNamesAndTypes" && echo 'OK' || echo 'FAIL' +echo -e '["x","z","y"]\n["UInt32", "String", "Date"]\n[1, "text","2020-01-01"]' | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactEachRowWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' echo "JSONCompactStringsEachRowWithNamesAndTypes" -echo -e '["x","y","z"]\n["String","Date","UInt32"]\n["text","2020-01-01","1"]' | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactStringsEachRowWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' -echo -e '["y","z","x"]\n["String","Date","UInt32"]\n["text","2020-01-01","1"]' | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactStringsEachRowWithNamesAndTypes" && echo 'OK' || echo 'FAIL' -echo -e '["x","z","y"]\n["UInt32", "String", "Date"]\n["1", "text","2020-01-01"]' | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactStringsEachRowWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +echo -e '["x","y","z"]\n["String","Date","UInt32"]\n["text","2020-01-01","1"]' | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactStringsEachRowWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +echo -e '["y","z","x"]\n["String","Date","UInt32"]\n["text","2020-01-01","1"]' | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactStringsEachRowWithNamesAndTypes" && echo 'OK' || echo 'FAIL' +echo -e '["x","z","y"]\n["UInt32", "String", "Date"]\n["1", "text","2020-01-01"]' | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT JSONCompactStringsEachRowWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' $CLICKHOUSE_CLIENT -q "DROP TABLE test_02098" diff --git a/tests/queries/0_stateless/02102_row_binary_with_names_and_types.reference b/tests/queries/0_stateless/02102_row_binary_with_names_and_types.reference new file mode 100644 index 00000000000..fc1da360fd6 --- /dev/null +++ b/tests/queries/0_stateless/02102_row_binary_with_names_and_types.reference @@ -0,0 +1,8 @@ +1 text 2020-01-01 +1 text 2020-01-01 +1 text 2020-01-01 +1 default 1970-01-01 +1 1970-01-01 +1 default 1970-01-01 +OK +OK diff --git a/tests/queries/0_stateless/02102_row_binary_with_names_and_types.sh b/tests/queries/0_stateless/02102_row_binary_with_names_and_types.sh new file mode 100755 index 00000000000..c9a0511dee9 --- /dev/null +++ b/tests/queries/0_stateless/02102_row_binary_with_names_and_types.sh @@ -0,0 +1,52 @@ +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_02102" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_02102 (x UInt32, y String DEFAULT 'default', z Date) engine=Memory()" + + +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" + +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102" + + +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=0 --input_format_with_types_use_header=0 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" + +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102" + + +$CLICKHOUSE_CLIENT -q "SELECT 'text' AS y, toDate('2020-01-01') AS z, toUInt32(1) AS x FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" + +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102" + + +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" + +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102" + +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" + +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102" + + + +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, [[1, 2, 3], [4, 5], []] as a FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" + +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102" + + + +$CLICKHOUSE_CLIENT -q "SELECT 'text' AS x, toDate('2020-01-01') AS y, toUInt32(1) AS z FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' + +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' as z, toDate('2020-01-01') AS y FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' + +$CLICKHOUSE_CLIENT -q "DROP TABLE test_02102" + From 476d7a411f37666adb627206e4a1e11705dea688 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 14 Oct 2021 16:44:28 +0300 Subject: [PATCH 158/919] allow to remove SAMPLE BY expression --- src/Databases/DatabaseOnDisk.cpp | 2 + src/Interpreters/InterpreterAlterQuery.cpp | 1 + src/Parsers/ASTAlterQuery.cpp | 4 ++ src/Parsers/ASTAlterQuery.h | 1 + src/Parsers/ParserAlterQuery.cpp | 5 +++ src/Storages/AlterCommands.cpp | 17 ++++++- src/Storages/AlterCommands.h | 3 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 11 ++++- .../02097_remove_sample_by.reference | 3 ++ .../0_stateless/02097_remove_sample_by.sql | 44 +++++++++++++++++++ 11 files changed, 88 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02097_remove_sample_by.reference create mode 100644 tests/queries/0_stateless/02097_remove_sample_by.sql diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 97e59f53f64..638aef7186c 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -180,6 +180,8 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo if (metadata.sampling_key.definition_ast) storage_ast.set(storage_ast.sample_by, metadata.sampling_key.definition_ast); + else if (storage_ast.sample_by != nullptr) /// SAMPLE BY was removed + storage_ast.sample_by = nullptr; if (metadata.table_ttl.definition_ast) storage_ast.set(storage_ast.ttl_table, metadata.table_ttl.definition_ast); diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 1c613758ecc..e595bd580b3 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -270,6 +270,7 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS required_access.emplace_back(AccessType::ALTER_ORDER_BY, database, table); break; } + case ASTAlterCommand::REMOVE_SAMPLE_BY: case ASTAlterCommand::MODIFY_SAMPLE_BY: { required_access.emplace_back(AccessType::ALTER_SAMPLE_BY, database, table); diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 53d53bf5ae1..d3153952114 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -157,6 +157,10 @@ void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState & settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY SAMPLE BY " << (settings.hilite ? hilite_none : ""); sample_by->formatImpl(settings, state, frame); } + else if (type == ASTAlterCommand::REMOVE_SAMPLE_BY) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << "REMOVE SAMPLE BY" << (settings.hilite ? hilite_none : ""); + } else if (type == ASTAlterCommand::ADD_INDEX) { settings.ostr << (settings.hilite ? hilite_keyword : "") << "ADD INDEX " << (if_not_exists ? "IF NOT EXISTS " : "") diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 9b40586e09f..3e0d9219549 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -41,6 +41,7 @@ public: RESET_SETTING, MODIFY_QUERY, REMOVE_TTL, + REMOVE_SAMPLE_BY, ADD_INDEX, DROP_INDEX, diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 2eade2079da..1ea64d94fe7 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -104,6 +104,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_ttl("TTL"); ParserKeyword s_remove_ttl("REMOVE TTL"); + ParserKeyword s_remove_sample_by("REMOVE SAMPLE BY"); ParserCompoundIdentifier parser_name; ParserStringLiteral parser_string_literal; @@ -669,6 +670,10 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->type = ASTAlterCommand::MODIFY_SAMPLE_BY; } + else if (s_remove_sample_by.ignore(pos, expected)) + { + command->type = ASTAlterCommand::REMOVE_SAMPLE_BY; + } else if (s_delete.ignore(pos, expected)) { if (s_in_partition.ignore(pos, expected)) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 1d057d1bb10..c5101f162ee 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -205,6 +205,13 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ command.sample_by = command_ast->sample_by; return command; } + else if (command_ast->type == ASTAlterCommand::REMOVE_SAMPLE_BY) + { + AlterCommand command; + command.ast = command_ast->clone(); + command.type = AlterCommand::REMOVE_SAMPLE_BY; + return command; + } else if (command_ast->type == ASTAlterCommand::ADD_INDEX) { AlterCommand command; @@ -463,6 +470,10 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) { metadata.sampling_key.recalculateWithNewAST(sample_by, metadata.columns, context); } + else if (type == REMOVE_SAMPLE_BY) + { + metadata.sampling_key = {}; + } else if (type == COMMENT_COLUMN) { metadata.columns.modify(column_name, @@ -745,7 +756,7 @@ bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metada return false; /// We remove properties on metadata level - if (isRemovingProperty() || type == REMOVE_TTL) + if (isRemovingProperty() || type == REMOVE_TTL || type == REMOVE_SAMPLE_BY) return false; if (type == DROP_COLUMN || type == DROP_INDEX || type == DROP_PROJECTION || type == RENAME_COLUMN) @@ -1208,6 +1219,10 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, ContextPt { throw Exception{"Table doesn't have any table TTL expression, cannot remove", ErrorCodes::BAD_ARGUMENTS}; } + else if (command.type == AlterCommand::REMOVE_SAMPLE_BY && !metadata.hasSamplingKey()) + { + throw Exception{"Table doesn't have SAMPLE BY, cannot remove", ErrorCodes::BAD_ARGUMENTS}; + } /// Collect default expressions for MODIFY and ADD comands if (command.type == AlterCommand::MODIFY_COLUMN || command.type == AlterCommand::ADD_COLUMN) diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index 046238bd5f5..dce6b496741 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -45,7 +45,8 @@ struct AlterCommand RENAME_COLUMN, REMOVE_TTL, MODIFY_DATABASE_SETTING, - COMMENT_TABLE + COMMENT_TABLE, + REMOVE_SAMPLE_BY, }; /// Which property user wants to remove from column diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 790b95a9fa9..3866f760a36 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -249,7 +249,7 @@ MergeTreeData::MergeTreeData( { /// This is for backward compatibility. checkSampleExpression(metadata_, attach || settings->compatibility_allow_sampling_expression_not_in_primary_key, - settings->check_sample_column_is_correct); + settings->check_sample_column_is_correct && !attach); } checkTTLExpressions(metadata_, metadata_); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 416d37cd351..5677ae6604f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1016,8 +1016,15 @@ void StorageReplicatedMergeTree::setTableStructure( if (metadata_diff.sampling_expression_changed) { - auto sample_by_ast = parse_key_expr(metadata_diff.new_sampling_expression); - new_metadata.sampling_key.recalculateWithNewAST(sample_by_ast, new_metadata.columns, getContext()); + if (!metadata_diff.new_sampling_expression.empty()) + { + auto sample_by_ast = parse_key_expr(metadata_diff.new_sampling_expression); + new_metadata.sampling_key.recalculateWithNewAST(sample_by_ast, new_metadata.columns, getContext()); + } + else /// SAMPLE BY was removed + { + new_metadata.sampling_key = {}; + } } if (metadata_diff.skip_indices_changed) diff --git a/tests/queries/0_stateless/02097_remove_sample_by.reference b/tests/queries/0_stateless/02097_remove_sample_by.reference new file mode 100644 index 00000000000..0747bbd5d1f --- /dev/null +++ b/tests/queries/0_stateless/02097_remove_sample_by.reference @@ -0,0 +1,3 @@ +CREATE TABLE default.t_remove_sample_by\n(\n `id` UInt64\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 +CREATE TABLE default.t_remove_sample_by\n(\n `id` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/t_remove_sample_by\', \'1\')\nORDER BY id\nSETTINGS index_granularity = 8192 +CREATE TABLE default.t_remove_sample_by\n(\n `id` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/02097_remove_sample_by.sql b/tests/queries/0_stateless/02097_remove_sample_by.sql new file mode 100644 index 00000000000..89fbfe0c4c5 --- /dev/null +++ b/tests/queries/0_stateless/02097_remove_sample_by.sql @@ -0,0 +1,44 @@ +-- Tags: zookeeper + +DROP TABLE IF EXISTS t_remove_sample_by; + +CREATE TABLE t_remove_sample_by(id UInt64) ENGINE = MergeTree ORDER BY id SAMPLE BY id; + +ALTER TABLE t_remove_sample_by REMOVE SAMPLE BY; +SHOW CREATE TABLE t_remove_sample_by; + +ALTER TABLE t_remove_sample_by REMOVE SAMPLE BY; -- { serverError 36 } +SELECT * FROM t_remove_sample_by SAMPLE 1 / 10; -- { serverError 141 } + +DROP TABLE t_remove_sample_by; + +CREATE TABLE t_remove_sample_by(id UInt64) +ENGINE = ReplicatedMergeTree('/clickhouse/{database}/t_remove_sample_by', '1') +ORDER BY id SAMPLE BY id; + +ALTER TABLE t_remove_sample_by REMOVE SAMPLE BY; +SHOW CREATE TABLE t_remove_sample_by; + +DROP TABLE t_remove_sample_by; + +CREATE TABLE t_remove_sample_by(id UInt64) ENGINE = Memory; +ALTER TABLE t_remove_sample_by REMOVE SAMPLE BY; -- { serverError 36 } + +DROP TABLE t_remove_sample_by; + +CREATE TABLE t_remove_sample_by(id String) +ENGINE = MergeTree ORDER BY id SAMPLE BY id +SETTINGS check_sample_column_is_correct = 0; + +ALTER TABLE t_remove_sample_by RESET SETTING check_sample_column_is_correct; + +DETACH TABLE t_remove_sample_by; +ATTACH TABLE t_remove_sample_by; + +INSERT INTO t_remove_sample_by VALUES (1); +SELECT * FROM t_remove_sample_by SAMPLE 1 / 10; -- { serverError 59 } + +ALTER TABLE t_remove_sample_by REMOVE SAMPLE BY; +SHOW CREATE TABLE t_remove_sample_by; + +DROP TABLE t_remove_sample_by; From 252634c6e9f4048f70b68e22d2131edd004b1b2d Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 14 Oct 2021 22:01:06 +0300 Subject: [PATCH 159/919] Fix tests --- tests/queries/0_stateless/02098_with_types_use_header.sh | 2 +- tests/queries/0_stateless/02099_tsv_raw_format.sh | 2 ++ .../0_stateless/02100_low_cardinality_nullable_null_default.sh | 2 ++ .../0_stateless/02101_empty_as_default_and_omitted_fields.sh | 2 ++ .../0_stateless/02102_row_binary_with_names_and_types.sh | 2 ++ 5 files changed, 9 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02098_with_types_use_header.sh b/tests/queries/0_stateless/02098_with_types_use_header.sh index d72adf5e681..846696d18c0 100755 --- a/tests/queries/0_stateless/02098_with_types_use_header.sh +++ b/tests/queries/0_stateless/02098_with_types_use_header.sh @@ -16,7 +16,7 @@ echo -e "x\tz\ty\nUInt32\tString\tDate\n1\ttext\t2020-01-01" | $CLICKHOUSE_CLIEN echo "CSVWithNamesAndTypes" echo -e "'x','y','z'\n'String','Date','UInt32'\n'text','2020-01-01',1" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' -echo -e "'y','z','x'\n'String','Date','UInt32'\n'text','2020-01-01',1" | $CLICKHOUSE_CLIENT --input_format_with_types_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" && echo 'OK' || echo 'FAIL' +echo -e "'y','z','x'\n'String','Date','UInt32'\n'text','2020-01-01',1" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" && echo 'OK' || echo 'FAIL' echo -e "'x','z','y'\n'UInt32','String',Date'\n1,'text','2020-01-01'" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' diff --git a/tests/queries/0_stateless/02099_tsv_raw_format.sh b/tests/queries/0_stateless/02099_tsv_raw_format.sh index ef59e399bdf..16b695e4037 100755 --- a/tests/queries/0_stateless/02099_tsv_raw_format.sh +++ b/tests/queries/0_stateless/02099_tsv_raw_format.sh @@ -1,3 +1,5 @@ +#!/usr/bin/env bash + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh diff --git a/tests/queries/0_stateless/02100_low_cardinality_nullable_null_default.sh b/tests/queries/0_stateless/02100_low_cardinality_nullable_null_default.sh index d380e784229..2fd1f130b7a 100755 --- a/tests/queries/0_stateless/02100_low_cardinality_nullable_null_default.sh +++ b/tests/queries/0_stateless/02100_low_cardinality_nullable_null_default.sh @@ -1,3 +1,5 @@ +#!/usr/bin/env bash + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh diff --git a/tests/queries/0_stateless/02101_empty_as_default_and_omitted_fields.sh b/tests/queries/0_stateless/02101_empty_as_default_and_omitted_fields.sh index 344982bcd84..4f03c72cac3 100755 --- a/tests/queries/0_stateless/02101_empty_as_default_and_omitted_fields.sh +++ b/tests/queries/0_stateless/02101_empty_as_default_and_omitted_fields.sh @@ -1,3 +1,5 @@ +#!/usr/bin/env bash + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh diff --git a/tests/queries/0_stateless/02102_row_binary_with_names_and_types.sh b/tests/queries/0_stateless/02102_row_binary_with_names_and_types.sh index c9a0511dee9..8c9ad5abac5 100755 --- a/tests/queries/0_stateless/02102_row_binary_with_names_and_types.sh +++ b/tests/queries/0_stateless/02102_row_binary_with_names_and_types.sh @@ -1,3 +1,5 @@ +#!/usr/bin/env bash + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh From 3995506d3731b417c601b680636d4b24e7e71387 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 15 Oct 2021 11:36:26 +0300 Subject: [PATCH 160/919] Adjust range reader for remote fs reads --- src/Common/ProfileEvents.cpp | 1 + .../CachedCompressedReadBuffer.cpp | 6 + src/Compression/CachedCompressedReadBuffer.h | 3 + src/Compression/CompressedReadBufferBase.cpp | 8 ++ src/Compression/CompressedReadBufferBase.h | 6 + .../CompressedReadBufferFromFile.cpp | 12 +- .../CompressedReadBufferFromFile.h | 2 + src/Disks/DiskWebServer.cpp | 2 +- src/Disks/HDFS/DiskHDFS.cpp | 2 +- src/Disks/ReadBufferFromRemoteFSGather.cpp | 43 +++++-- src/Disks/ReadBufferFromRemoteFSGather.h | 34 ++++-- src/Disks/S3/DiskS3.cpp | 1 + src/Disks/S3/registerDiskS3.cpp | 32 ++--- ...chronousReadIndirectBufferFromRemoteFS.cpp | 68 +++++++---- ...ynchronousReadIndirectBufferFromRemoteFS.h | 6 +- src/IO/ReadBufferFromS3.cpp | 24 +++- src/IO/ReadBufferFromS3.h | 5 +- src/IO/ReadBufferFromWebServer.cpp | 4 +- src/IO/ReadBufferFromWebServer.h | 3 +- src/Storages/HDFS/ReadBufferFromHDFS.cpp | 2 +- src/Storages/HDFS/ReadBufferFromHDFS.h | 4 +- src/Storages/MergeTree/IMergeTreeReader.h | 3 +- .../MergeTree/MergeTreeRangeReader.cpp | 32 +++-- src/Storages/MergeTree/MergeTreeRangeReader.h | 8 +- .../MergeTree/MergeTreeReaderCompact.cpp | 3 +- .../MergeTree/MergeTreeReaderCompact.h | 3 +- .../MergeTree/MergeTreeReaderInMemory.cpp | 3 +- .../MergeTree/MergeTreeReaderInMemory.h | 3 +- .../MergeTree/MergeTreeReaderStream.cpp | 111 +++++++++++------- .../MergeTree/MergeTreeReaderStream.h | 9 +- .../MergeTree/MergeTreeReaderWide.cpp | 20 ++-- src/Storages/MergeTree/MergeTreeReaderWide.h | 7 +- .../MergeTree/MergeTreeSequentialSource.cpp | 3 +- 33 files changed, 318 insertions(+), 155 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 41ae41b38d4..f9645875a6c 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -262,6 +262,7 @@ M(RemoteFSNewReaders, "Number of created impl objects") \ M(RemoteFSAsyncBuffers, "Total number of AsycnhronousReadIndirectBufferFromREmoteFS buffers") \ M(RemoteFSSimpleBuffers, "Total number of ReadIndirectBufferFromREmoteFS buffers") \ + M(RemoteFSRedundantlyReadBytes, "") \ \ M(SleepFunctionCalls, "Number of times a sleep function (sleep, sleepEachRow) has been called.") \ M(SleepFunctionMicroseconds, "Time spent sleeping due to a sleep function call.") \ diff --git a/src/Compression/CachedCompressedReadBuffer.cpp b/src/Compression/CachedCompressedReadBuffer.cpp index 4a583773b4b..c19e854dd45 100644 --- a/src/Compression/CachedCompressedReadBuffer.cpp +++ b/src/Compression/CachedCompressedReadBuffer.cpp @@ -28,6 +28,12 @@ void CachedCompressedReadBuffer::initInput() } +void CachedCompressedReadBuffer::prefetch() +{ + file_in->prefetch(); +} + + bool CachedCompressedReadBuffer::nextImpl() { /// Let's check for the presence of a decompressed block in the cache, grab the ownership of this block, if it exists. diff --git a/src/Compression/CachedCompressedReadBuffer.h b/src/Compression/CachedCompressedReadBuffer.h index bb24f699eed..6eedf66a487 100644 --- a/src/Compression/CachedCompressedReadBuffer.h +++ b/src/Compression/CachedCompressedReadBuffer.h @@ -33,8 +33,11 @@ private: UncompressedCache::MappedPtr owned_cell; void initInput(); + bool nextImpl() override; + void prefetch() override; + /// Passed into file_in. ReadBufferFromFileBase::ProfileCallback profile_callback; clockid_t clock_type {}; diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index 81e49e445a7..b0609b9ba26 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include @@ -107,6 +108,13 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c } +void CompressedReadBufferBase::setRightOffset(size_t offset) +{ + if (auto * async_in = dynamic_cast(compressed_in)) + async_in->setRightOffset(offset); +} + + /// Read compressed data into compressed_buffer. Get size of decompressed data from block header. Checksum if need. /// Returns number of compressed bytes read. size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, size_t & size_compressed_without_checksum, bool always_copy) diff --git a/src/Compression/CompressedReadBufferBase.h b/src/Compression/CompressedReadBufferBase.h index c32a169aecc..0f788ec445d 100644 --- a/src/Compression/CompressedReadBufferBase.h +++ b/src/Compression/CompressedReadBufferBase.h @@ -60,6 +60,12 @@ public: disable_checksum = true; } + /** + * For asynchronous range reading from remote fs need to update last offset for current task, + * when newer tasks read behind previous task last mark. + */ + void setRightOffset(size_t offset); + public: CompressionCodecPtr codec; }; diff --git a/src/Compression/CompressedReadBufferFromFile.cpp b/src/Compression/CompressedReadBufferFromFile.cpp index 2cfd6d65c1c..d5aae38cf34 100644 --- a/src/Compression/CompressedReadBufferFromFile.cpp +++ b/src/Compression/CompressedReadBufferFromFile.cpp @@ -44,12 +44,6 @@ bool CompressedReadBufferFromFile::nextImpl() } -void CompressedReadBufferFromFile::prefetch() -{ - file_in.prefetch(); -} - - CompressedReadBufferFromFile::CompressedReadBufferFromFile(std::unique_ptr buf, bool allow_different_codecs_) : BufferWithOwnMemory(0), p_file_in(std::move(buf)), file_in(*p_file_in) { @@ -72,6 +66,12 @@ CompressedReadBufferFromFile::CompressedReadBufferFromFile( } +void CompressedReadBufferFromFile::prefetch() +{ + file_in.prefetch(); +} + + void CompressedReadBufferFromFile::seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block) { /// Nothing to do if we already at required position diff --git a/src/Compression/CompressedReadBufferFromFile.h b/src/Compression/CompressedReadBufferFromFile.h index 5f027851da3..125e80a0078 100644 --- a/src/Compression/CompressedReadBufferFromFile.h +++ b/src/Compression/CompressedReadBufferFromFile.h @@ -42,6 +42,7 @@ private: /* size_t nextimpl_working_buffer_offset; */ bool nextImpl() override; + void prefetch() override; public: @@ -61,6 +62,7 @@ public: { file_in.setProfileCallback(profile_callback_, clock_type_); } + }; } diff --git a/src/Disks/DiskWebServer.cpp b/src/Disks/DiskWebServer.cpp index d6fae0aa7dc..148e34cf9c5 100644 --- a/src/Disks/DiskWebServer.cpp +++ b/src/Disks/DiskWebServer.cpp @@ -168,7 +168,7 @@ std::unique_ptr DiskWebServer::readFile(const String & p bool threadpool_read = read_settings.remote_fs_method == RemoteFSReadMethod::read_threadpool; - auto web_impl = std::make_unique(url, meta, getContext(), threadpool_read, read_settings); + auto web_impl = std::make_unique(path, url, meta, getContext(), threadpool_read, read_settings); if (threadpool_read) { diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index 9ed861a5f34..cb3e1e00277 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -73,7 +73,7 @@ std::unique_ptr DiskHDFS::readFile(const String & path, "Read from file by path: {}. Existing HDFS objects: {}", backQuote(metadata_path + path), metadata.remote_fs_objects.size()); - auto hdfs_impl = std::make_unique(config, remote_fs_root_path, metadata, read_settings.remote_fs_buffer_size); + auto hdfs_impl = std::make_unique(path, config, remote_fs_root_path, metadata, read_settings.remote_fs_buffer_size); if (read_settings.remote_fs_method == RemoteFSReadMethod::read_threadpool) { diff --git a/src/Disks/ReadBufferFromRemoteFSGather.cpp b/src/Disks/ReadBufferFromRemoteFSGather.cpp index 5ffb8b9f589..67ba4448d20 100644 --- a/src/Disks/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/ReadBufferFromRemoteFSGather.cpp @@ -12,6 +12,7 @@ #include #endif +#include #include #include @@ -27,31 +28,32 @@ namespace ErrorCodes #if USE_AWS_S3 -SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const String & path) const +SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const String & path, size_t last_offset) const { return std::make_unique(client_ptr, bucket, - fs::path(metadata.remote_fs_root_path) / path, max_single_read_retries, settings, threadpool_read); + fs::path(metadata.remote_fs_root_path) / path, max_single_read_retries, settings, threadpool_read, last_offset); } #endif -SeekableReadBufferPtr ReadBufferFromWebServerGather::createImplementationBuffer(const String & path) const +SeekableReadBufferPtr ReadBufferFromWebServerGather::createImplementationBuffer(const String & path, size_t last_offset) const { - return std::make_unique(fs::path(uri) / path, context, settings, threadpool_read); + return std::make_unique(fs::path(uri) / path, context, settings, threadpool_read, last_offset); } #if USE_HDFS -SeekableReadBufferPtr ReadBufferFromHDFSGather::createImplementationBuffer(const String & path) const +SeekableReadBufferPtr ReadBufferFromHDFSGather::createImplementationBuffer(const String & path, size_t last_offset) const { - return std::make_unique(hdfs_uri, fs::path(hdfs_directory) / path, config, buf_size); + return std::make_unique(hdfs_uri, fs::path(hdfs_directory) / path, config, buf_size, last_offset); } #endif -ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather(const RemoteMetadata & metadata_) +ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather(const RemoteMetadata & metadata_, const String & path_) : ReadBuffer(nullptr, 0) , metadata(metadata_) + , path(path_) { } @@ -91,7 +93,7 @@ void ReadBufferFromRemoteFSGather::initialize() /// Do not create a new buffer if we already have what we need. if (!current_buf || buf_idx != i) { - current_buf = createImplementationBuffer(file_path); + current_buf = createImplementationBuffer(file_path, last_offset); buf_idx = i; } @@ -126,8 +128,8 @@ bool ReadBufferFromRemoteFSGather::nextImpl() ++current_buf_idx; - const auto & path = metadata.remote_fs_objects[current_buf_idx].first; - current_buf = createImplementationBuffer(path); + const auto & current_path = metadata.remote_fs_objects[current_buf_idx].first; + current_buf = createImplementationBuffer(current_path, last_offset); return readImpl(); } @@ -145,6 +147,7 @@ bool ReadBufferFromRemoteFSGather::readImpl() if (bytes_to_ignore) current_buf->ignore(bytes_to_ignore); + LOG_DEBUG(&Poco::Logger::get("Gather"), "Reading from path: {}", path); auto result = current_buf->next(); swap(*current_buf); @@ -158,8 +161,17 @@ bool ReadBufferFromRemoteFSGather::readImpl() void ReadBufferFromRemoteFSGather::seek(off_t offset) { + current_buf.reset(); absolute_position = offset; - initialize(); + // initialize(); +} + + +void ReadBufferFromRemoteFSGather::setRightOffset(size_t offset) +{ + assert(last_offset < offset); + current_buf.reset(); + last_offset = offset; } @@ -168,4 +180,13 @@ void ReadBufferFromRemoteFSGather::reset() current_buf.reset(); } + +String ReadBufferFromRemoteFSGather::getFileName() const +{ + return path; + // if (current_buf) + // return fs::path(metadata.metadata_file_path) / metadata.remote_fs_objects[buf_idx].first; + // return metadata.metadata_file_path; +} + } diff --git a/src/Disks/ReadBufferFromRemoteFSGather.h b/src/Disks/ReadBufferFromRemoteFSGather.h index efd3d7a2483..3285d7190d2 100644 --- a/src/Disks/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/ReadBufferFromRemoteFSGather.h @@ -13,28 +13,31 @@ namespace Aws namespace S3 { class S3Client; -}} +} +} + namespace DB { class ReadBufferFromRemoteFSGather : public ReadBuffer { -friend class ThreadPoolRemoteFSReader; friend class ReadIndirectBufferFromRemoteFS; public: - explicit ReadBufferFromRemoteFSGather(const RemoteMetadata & metadata_); + explicit ReadBufferFromRemoteFSGather(const RemoteMetadata & metadata_, const String & path_); - String getFileName() const { return metadata.metadata_file_path; } + String getFileName() const; void reset(); void seek(off_t offset); /// SEEK_SET only. -protected: + void setRightOffset(size_t offset); + size_t readInto(char * data, size_t size, size_t offset, size_t ignore = 0); - virtual SeekableReadBufferPtr createImplementationBuffer(const String & path) const = 0; +protected: + virtual SeekableReadBufferPtr createImplementationBuffer(const String & path, size_t last_offset) const = 0; RemoteMetadata metadata; @@ -54,6 +57,10 @@ private: size_t buf_idx = 0; size_t bytes_to_ignore = 0; + + size_t last_offset = 0; + + String path; }; @@ -63,13 +70,14 @@ class ReadBufferFromS3Gather final : public ReadBufferFromRemoteFSGather { public: ReadBufferFromS3Gather( + const String & path_, std::shared_ptr client_ptr_, const String & bucket_, IDiskRemote::Metadata metadata_, size_t max_single_read_retries_, const ReadSettings & settings_, bool threadpool_read_ = false) - : ReadBufferFromRemoteFSGather(metadata_) + : ReadBufferFromRemoteFSGather(metadata_, path_) , client_ptr(std::move(client_ptr_)) , bucket(bucket_) , max_single_read_retries(max_single_read_retries_) @@ -78,7 +86,7 @@ public: { } - SeekableReadBufferPtr createImplementationBuffer(const String & path) const override; + SeekableReadBufferPtr createImplementationBuffer(const String & path, size_t last_offset) const override; private: std::shared_ptr client_ptr; @@ -94,12 +102,13 @@ class ReadBufferFromWebServerGather final : public ReadBufferFromRemoteFSGather { public: ReadBufferFromWebServerGather( + const String & path_, const String & uri_, RemoteMetadata metadata_, ContextPtr context_, size_t threadpool_read_, const ReadSettings & settings_) - : ReadBufferFromRemoteFSGather(metadata_) + : ReadBufferFromRemoteFSGather(metadata_, path_) , uri(uri_) , context(context_) , threadpool_read(threadpool_read_) @@ -107,7 +116,7 @@ public: { } - SeekableReadBufferPtr createImplementationBuffer(const String & path) const override; + SeekableReadBufferPtr createImplementationBuffer(const String & path, size_t last_offset) const override; private: String uri; @@ -123,11 +132,12 @@ class ReadBufferFromHDFSGather final : public ReadBufferFromRemoteFSGather { public: ReadBufferFromHDFSGather( + const String & path_, const Poco::Util::AbstractConfiguration & config_, const String & hdfs_uri_, IDiskRemote::Metadata metadata_, size_t buf_size_) - : ReadBufferFromRemoteFSGather(metadata_) + : ReadBufferFromRemoteFSGather(metadata_, path_) , config(config_) , buf_size(buf_size_) { @@ -136,7 +146,7 @@ public: hdfs_uri = hdfs_uri_.substr(0, begin_of_path); } - SeekableReadBufferPtr createImplementationBuffer(const String & path) const override; + SeekableReadBufferPtr createImplementationBuffer(const String & path, size_t last_offset) const override; private: const Poco::Util::AbstractConfiguration & config; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index c22bc32c84b..b6c94ca0802 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -205,6 +205,7 @@ std::unique_ptr DiskS3::readFile(const String & path, co bool threadpool_read = read_settings.remote_fs_method == RemoteFSReadMethod::read_threadpool; auto s3_impl = std::make_unique( + path, settings->client, bucket, metadata, settings->s3_max_single_read_retries, read_settings, threadpool_read); diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index 5eabbfff5f8..1db9c3938c6 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -200,27 +200,27 @@ void registerDiskS3(DiskFactory & factory) s3disk->startup(); - bool cache_enabled = config.getBool(config_prefix + ".cache_enabled", true); + // bool cache_enabled = config.getBool(config_prefix + ".cache_enabled", true); - if (cache_enabled) - { - String cache_path = config.getString(config_prefix + ".cache_path", context->getPath() + "disks/" + name + "/cache/"); + // if (cache_enabled) + // { + // String cache_path = config.getString(config_prefix + ".cache_path", context->getPath() + "disks/" + name + "/cache/"); - if (metadata_path == cache_path) - throw Exception("Metadata and cache path should be different: " + metadata_path, ErrorCodes::BAD_ARGUMENTS); + // if (metadata_path == cache_path) + // throw Exception("Metadata and cache path should be different: " + metadata_path, ErrorCodes::BAD_ARGUMENTS); - auto cache_disk = std::make_shared("s3-cache", cache_path, 0); - auto cache_file_predicate = [] (const String & path) - { - return path.ends_with("idx") // index files. - || path.ends_with("mrk") || path.ends_with("mrk2") || path.ends_with("mrk3") // mark files. - || path.ends_with("txt") || path.ends_with("dat"); - }; + // auto cache_disk = std::make_shared("s3-cache", cache_path, 0); + // auto cache_file_predicate = [] (const String & path) + // { + // return path.ends_with("idx") // index files. + // || path.ends_with("mrk") || path.ends_with("mrk2") || path.ends_with("mrk3") // mark files. + // || path.ends_with("txt") || path.ends_with("dat"); + // }; - s3disk = std::make_shared(s3disk, cache_disk, cache_file_predicate); - } + // s3disk = std::make_shared(s3disk, cache_disk, cache_file_predicate); + // } - return std::make_shared(s3disk); + return s3disk; }; factory.registerDiskType("s3", creator); } diff --git a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp index c69eb893663..9dd3aeb4625 100644 --- a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -2,6 +2,7 @@ #include #include +#include namespace CurrentMetrics @@ -35,14 +36,16 @@ AsynchronousReadIndirectBufferFromRemoteFS::AsynchronousReadIndirectBufferFromRe Int32 priority_, std::shared_ptr impl_, size_t buf_size_, - size_t min_bytes_for_seek_) + size_t /* min_bytes_for_seek_ */) : ReadBufferFromFileBase(buf_size_, nullptr, 0) , reader(reader_) , priority(priority_) , impl(impl_) , prefetch_buffer(buf_size_) - , min_bytes_for_seek(min_bytes_for_seek_) + // , min_bytes_for_seek(min_bytes_for_seek_) { + ProfileEvents::increment(ProfileEvents::RemoteFSAsyncBuffers); + buffer_events += impl->getFileName() + " : "; } @@ -66,15 +69,31 @@ std::future AsynchronousReadIndirectBufferFromRemot void AsynchronousReadIndirectBufferFromRemoteFS::prefetch() { - if (hasPendingData()) - return; + if (hasPendingData()) + return; + if (prefetch_future.valid()) + return; + + prefetch_future = readInto(prefetch_buffer.data(), prefetch_buffer.size()); + ProfileEvents::increment(ProfileEvents::RemoteFSPrefetches); + buffer_events += "-- Prefetch (" + toString(absolute_position) + ") --"; +} + + +void AsynchronousReadIndirectBufferFromRemoteFS::setRightOffset(size_t offset) +{ + buffer_events += "-- Set last offset " + toString(offset) + "--"; if (prefetch_future.valid()) - return; + { + buffer_events += "-- Cancelling because of offset update --"; + ProfileEvents::increment(ProfileEvents::RemoteFSSeekCancelledPrefetches); + prefetch_future.wait(); + prefetch_future = {}; + } - prefetch_future = readInto(prefetch_buffer.data(), prefetch_buffer.size()); - ProfileEvents::increment(ProfileEvents::RemoteFSPrefetches); - buffer_events += "-- Prefetch --"; + last_offset = offset; + impl->setRightOffset(offset); } @@ -86,7 +105,6 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() if (prefetch_future.valid()) { ProfileEvents::increment(ProfileEvents::RemoteFSPrefetchReads); - buffer_events += "-- Read from prefetch --"; CurrentMetrics::Increment metric_increment{CurrentMetrics::AsynchronousReadWait}; Stopwatch watch; @@ -100,13 +118,17 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() absolute_position += size; } } + + buffer_events += fmt::format("-- Read from prefetch from offset: {}, upper bound: {}, actually read: {} --", + toString(absolute_position), toString(last_offset), toString(size)); watch.stop(); ProfileEvents::increment(ProfileEvents::AsynchronousReadWaitMicroseconds, watch.elapsedMicroseconds()); } else { - buffer_events += "-- Read without prefetch --"; size = readInto(memory.data(), memory.size()).get(); + buffer_events += fmt::format("-- Read without prefetch from offset: {}, upper bound: {}, actually read: {} --", + toString(absolute_position), toString(last_offset), toString(size)); if (size) { set(memory.data(), memory.size()); @@ -115,7 +137,6 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() } } - buffer_events += " + " + toString(size) + " + "; prefetch_future = {}; return size; } @@ -163,6 +184,7 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence if (prefetch_future.valid()) { + buffer_events += "-- cancelling prefetch because of seek --"; ProfileEvents::increment(ProfileEvents::RemoteFSSeekCancelledPrefetches); prefetch_future.wait(); prefetch_future = {}; @@ -170,16 +192,18 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence pos = working_buffer.end(); - if (static_cast(absolute_position) >= getPosition() - && static_cast(absolute_position) < getPosition() + static_cast(min_bytes_for_seek)) - { - /** - * Lazy ignore. Save number of bytes to ignore and ignore it either for prefetch buffer or current buffer. - */ - bytes_to_ignore = absolute_position - getPosition(); - } - else + // if (static_cast(absolute_position) >= getPosition() + // && static_cast(absolute_position) < getPosition() + static_cast(min_bytes_for_seek)) + // { + // /** + // * Lazy ignore. Save number of bytes to ignore and ignore it either for prefetch buffer or current buffer. + // */ + // // bytes_to_ignore = absolute_position - getPosition(); + // impl->seek(absolute_position); /// SEEK_SET. + // } + // else { + buffer_events += "-- Impl seek --"; impl->seek(absolute_position); /// SEEK_SET. } @@ -189,14 +213,14 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence void AsynchronousReadIndirectBufferFromRemoteFS::finalize() { - std::cerr << "\n\n\nBuffer events: " << buffer_events << std::endl; - if (prefetch_future.valid()) { + buffer_events += "-- cancelling prefetch in finalize --"; ProfileEvents::increment(ProfileEvents::RemoteFSUnusedCancelledPrefetches); prefetch_future.wait(); prefetch_future = {}; } + std::cerr << "Buffer events: " << buffer_events << std::endl; } diff --git a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h index 3c600562252..af67efe1218 100644 --- a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -47,6 +47,8 @@ public: void prefetch() override; + void setRightOffset(size_t offset); + private: bool nextImpl() override; @@ -68,8 +70,10 @@ private: String buffer_events; - size_t min_bytes_for_seek; + // size_t min_bytes_for_seek; + size_t bytes_to_ignore = 0; + Int64 last_offset = 0; }; } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index bf974440be2..40bda7d6bfe 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -32,7 +32,7 @@ namespace ErrorCodes ReadBufferFromS3::ReadBufferFromS3( std::shared_ptr client_ptr_, const String & bucket_, const String & key_, - UInt64 max_single_read_retries_, const ReadSettings & settings_, bool use_external_buffer_) + UInt64 max_single_read_retries_, const ReadSettings & settings_, bool use_external_buffer_, size_t last_offset_) : SeekableReadBuffer(nullptr, 0) , client_ptr(std::move(client_ptr_)) , bucket(bucket_) @@ -40,11 +40,22 @@ ReadBufferFromS3::ReadBufferFromS3( , max_single_read_retries(max_single_read_retries_) , read_settings(settings_) , use_external_buffer(use_external_buffer_) + , last_offset(last_offset_) { } bool ReadBufferFromS3::nextImpl() { + if (last_offset) + { + if (static_cast(last_offset) == offset) + { + impl.reset(); + working_buffer.resize(0); + return false; + } + } + bool next_result = false; /// `impl` has been initialized earlier and now we're at the end of the current portion of data. @@ -162,16 +173,17 @@ std::unique_ptr ReadBufferFromS3::initialize() req.SetBucket(bucket); req.SetKey(key); - auto right_offset = read_settings.remote_read_right_offset; - if (right_offset) + // auto right_offset = read_settings.remote_read_right_offset; + + if (last_offset) { - req.SetRange(fmt::format("bytes={}-{}", offset, right_offset)); - LOG_TRACE(log, "Read S3 object. Bucket: {}, Key: {}, Range: {}-{}", bucket, key, offset, right_offset); + req.SetRange(fmt::format("bytes={}-{}", offset, last_offset - 1)); + LOG_DEBUG(log, "Read S3 object. Bucket: {}, Key: {}, Range: {}-{}", bucket, key, offset, last_offset - 1); } else { req.SetRange(fmt::format("bytes={}-", offset)); - LOG_TRACE(log, "Read S3 object. Bucket: {}, Key: {}, Offset: {}", bucket, key, offset); + LOG_DEBUG(log, "Read S3 object. Bucket: {}, Key: {}, Offset: {}", bucket, key, offset); } Aws::S3::Model::GetObjectOutcome outcome = client_ptr->GetObject(req); diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index b27de8aa0b4..336893ca5b0 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -45,8 +45,10 @@ public: const String & key_, UInt64 max_single_read_retries_, const ReadSettings & settings_, - bool use_external_buffer = false); + bool use_external_buffer = false, + size_t last_offset_ = 0); + size_t right = 0; bool nextImpl() override; off_t seek(off_t off, int whence) override; @@ -57,6 +59,7 @@ private: ReadSettings read_settings; bool use_external_buffer; + size_t last_offset; }; } diff --git a/src/IO/ReadBufferFromWebServer.cpp b/src/IO/ReadBufferFromWebServer.cpp index bfbca078248..41b4c80b2c8 100644 --- a/src/IO/ReadBufferFromWebServer.cpp +++ b/src/IO/ReadBufferFromWebServer.cpp @@ -23,7 +23,7 @@ namespace ErrorCodes static constexpr size_t HTTP_MAX_TRIES = 10; ReadBufferFromWebServer::ReadBufferFromWebServer( - const String & url_, ContextPtr context_, const ReadSettings & settings_, bool use_external_buffer_) + const String & url_, ContextPtr context_, const ReadSettings & settings_, bool use_external_buffer_, size_t) : SeekableReadBuffer(nullptr, 0) , log(&Poco::Logger::get("ReadBufferFromWebServer")) , context(context_) @@ -108,7 +108,7 @@ void ReadBufferFromWebServer::initializeWithRetry() if (i == num_tries - 1) throw; - LOG_ERROR(&Poco::Logger::get("ReadBufferFromWeb"), e.what()); + LOG_ERROR(&Poco::Logger::get("ReadBufferFromWeb"), "Error: {}, code: {}", e.what(), e.code()); sleepForMilliseconds(milliseconds_to_wait); milliseconds_to_wait *= 2; } diff --git a/src/IO/ReadBufferFromWebServer.h b/src/IO/ReadBufferFromWebServer.h index 780a4b16442..c4d847b9f39 100644 --- a/src/IO/ReadBufferFromWebServer.h +++ b/src/IO/ReadBufferFromWebServer.h @@ -20,7 +20,8 @@ public: explicit ReadBufferFromWebServer( const String & url_, ContextPtr context_, const ReadSettings & settings_ = {}, - bool use_external_buffer_ = false); + bool use_external_buffer_ = false, + size_t last_offset = 0); bool nextImpl() override; diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index 96d67ad0e08..f24705d7f65 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -125,7 +125,7 @@ ReadBufferFromHDFS::ReadBufferFromHDFS( const String & hdfs_uri_, const String & hdfs_file_path_, const Poco::Util::AbstractConfiguration & config_, - size_t buf_size_) + size_t buf_size_, size_t) : SeekableReadBuffer(nullptr, 0) , impl(std::make_unique(hdfs_uri_, hdfs_file_path_, config_, buf_size_)) { diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.h b/src/Storages/HDFS/ReadBufferFromHDFS.h index 035a55bd0fa..38c8047ba93 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.h +++ b/src/Storages/HDFS/ReadBufferFromHDFS.h @@ -27,7 +27,9 @@ struct ReadBufferFromHDFSImpl; public: ReadBufferFromHDFS(const String & hdfs_uri_, const String & hdfs_file_path_, - const Poco::Util::AbstractConfiguration & config_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE); + const Poco::Util::AbstractConfiguration & config_, + size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE, + size_t last_offset = 0); ~ReadBufferFromHDFS() override; diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 696cc2f105b..bcb51f2fce6 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -31,7 +31,8 @@ public: /// Return the number of rows has been read or zero if there is no columns to read. /// If continue_reading is true, continue reading from last state, otherwise seek to from_mark - virtual size_t readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) = 0; + virtual size_t readRows(size_t from_mark, size_t current_task_last_mark, + bool continue_reading, size_t max_rows_to_read, Columns & res_columns) = 0; virtual bool canReadIncompleteGranules() const = 0; diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 2f6bc10e472..31e8fe6454f 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -55,8 +55,11 @@ static void filterColumns(Columns & columns, const ColumnPtr & filter) MergeTreeRangeReader::DelayedStream::DelayedStream( - size_t from_mark, IMergeTreeReader * merge_tree_reader_) + size_t from_mark, + size_t current_task_last_mark_, + IMergeTreeReader * merge_tree_reader_) : current_mark(from_mark), current_offset(0), num_delayed_rows(0) + , current_task_last_mark(current_task_last_mark_) , merge_tree_reader(merge_tree_reader_) , index_granularity(&(merge_tree_reader->data_part->index_granularity)) , continue_reading(false), is_finished(false) @@ -73,7 +76,8 @@ size_t MergeTreeRangeReader::DelayedStream::readRows(Columns & columns, size_t n { if (num_rows) { - size_t rows_read = merge_tree_reader->readRows(current_mark, continue_reading, num_rows, columns); + size_t rows_read = merge_tree_reader->readRows( + current_mark, current_task_last_mark, continue_reading, num_rows, columns); continue_reading = true; /// Zero rows_read maybe either because reading has finished @@ -151,13 +155,13 @@ size_t MergeTreeRangeReader::DelayedStream::finalize(Columns & columns) MergeTreeRangeReader::Stream::Stream( - size_t from_mark, size_t to_mark, IMergeTreeReader * merge_tree_reader_) + size_t from_mark, size_t to_mark, size_t current_task_last_mark, IMergeTreeReader * merge_tree_reader_) : current_mark(from_mark), offset_after_current_mark(0) , last_mark(to_mark) , merge_tree_reader(merge_tree_reader_) , index_granularity(&(merge_tree_reader->data_part->index_granularity)) , current_mark_index_granularity(index_granularity->getMarkRows(from_mark)) - , stream(from_mark, merge_tree_reader) + , stream(from_mark, current_task_last_mark, merge_tree_reader) { size_t marks_count = index_granularity->getMarksCount(); if (from_mark >= marks_count) @@ -280,9 +284,9 @@ void MergeTreeRangeReader::ReadResult::adjustLastGranule() throw Exception("Can't adjust last granule because no granules were added.", ErrorCodes::LOGICAL_ERROR); if (num_rows_to_subtract > rows_per_granule.back()) - throw Exception("Can't adjust last granule because it has " + toString(rows_per_granule.back()) - + " rows, but try to subtract " + toString(num_rows_to_subtract) + " rows.", - ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Can't adjust last granule because it has {} rows, but try to subtract {} rows.", + toString(rows_per_granule.back()), toString(num_rows_to_subtract)); rows_per_granule.back() -= num_rows_to_subtract; total_rows_per_granule -= num_rows_to_subtract; @@ -750,6 +754,16 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t ReadResult result; result.columns.resize(merge_tree_reader->getColumns().size()); + auto current_task_last_mark_range = std::max_element(ranges.begin(), ranges.end(), + [&](const MarkRange & range1, const MarkRange & range2) + { + return range1.end < range2.end; + }); + + size_t current_task_last_mark = 0; + if (current_task_last_mark_range != ranges.end()) + current_task_last_mark = current_task_last_mark_range->end; + /// Stream is lazy. result.num_added_rows is the number of rows added to block which is not equal to /// result.num_rows_read until call to stream.finalize(). Also result.num_added_rows may be less than /// result.num_rows_read if the last granule in range also the last in part (so we have to adjust last granule). @@ -760,7 +774,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t if (stream.isFinished()) { result.addRows(stream.finalize(result.columns)); - stream = Stream(ranges.front().begin, ranges.front().end, merge_tree_reader); + stream = Stream(ranges.front().begin, ranges.front().end, current_task_last_mark, merge_tree_reader); result.addRange(ranges.front()); ranges.pop_front(); } @@ -818,7 +832,7 @@ Columns MergeTreeRangeReader::continueReadingChain(ReadResult & result, size_t & num_rows += stream.finalize(columns); const auto & range = started_ranges[next_range_to_start].range; ++next_range_to_start; - stream = Stream(range.begin, range.end, merge_tree_reader); + stream = Stream(range.begin, range.end, 0, merge_tree_reader); } bool last = i + 1 == size; diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index d099d2475d2..c913b476b73 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -62,7 +62,7 @@ public: { public: DelayedStream() = default; - DelayedStream(size_t from_mark, IMergeTreeReader * merge_tree_reader); + DelayedStream(size_t from_mark, size_t current_task_last_mark_, IMergeTreeReader * merge_tree_reader); /// Read @num_rows rows from @from_mark starting from @offset row /// Returns the number of rows added to block. @@ -81,6 +81,8 @@ public: size_t current_offset = 0; /// Num of rows we have to read size_t num_delayed_rows = 0; + /// Last mark from all ranges of current task. + size_t current_task_last_mark = 0; /// Actual reader of data from disk IMergeTreeReader * merge_tree_reader = nullptr; @@ -99,7 +101,8 @@ public: { public: Stream() = default; - Stream(size_t from_mark, size_t to_mark, IMergeTreeReader * merge_tree_reader); + Stream(size_t from_mark, size_t to_mark, + size_t current_task_last_mark, IMergeTreeReader * merge_tree_reader); /// Returns the number of rows added to block. size_t read(Columns & columns, size_t num_rows, bool skip_remaining_rows_in_current_granule); @@ -122,6 +125,7 @@ public: /// Invariant: offset_after_current_mark + skipped_rows_after_offset < index_granularity size_t offset_after_current_mark = 0; + /// Last mark in current range. size_t last_mark = 0; IMergeTreeReader * merge_tree_reader = nullptr; diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index c898874f737..35da14319ba 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -121,7 +121,8 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( } } -size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) +size_t MergeTreeReaderCompact::readRows( + size_t from_mark, size_t /* current_task_last_mark */, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) { if (continue_reading) from_mark = next_mark; diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index dbfaa7868fa..5a419a23642 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -32,7 +32,8 @@ public: /// Return the number of rows has been read or zero if there is no columns to read. /// If continue_reading is true, continue reading from last state, otherwise seek to from_mark - size_t readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) override; + size_t readRows(size_t from_mark, size_t current_task_last_mark, + bool continue_reading, size_t max_rows_to_read, Columns & res_columns) override; bool canReadIncompleteGranules() const override { return false; } diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index 3e81fec5145..8a69183e858 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -37,7 +37,8 @@ MergeTreeReaderInMemory::MergeTreeReaderInMemory( } } -size_t MergeTreeReaderInMemory::readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) +size_t MergeTreeReaderInMemory::readRows( + size_t from_mark, size_t /* current_task_last_mark */, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) { if (!continue_reading) total_rows_read = 0; diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.h b/src/Storages/MergeTree/MergeTreeReaderInMemory.h index 4526b19c4a8..ff6eb92d9c3 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.h +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.h @@ -23,7 +23,8 @@ public: /// Return the number of rows has been read or zero if there is no columns to read. /// If continue_reading is true, continue reading from last state, otherwise seek to from_mark - size_t readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) override; + size_t readRows(size_t from_mark, size_t current_tasl_last_mark, + bool continue_reading, size_t max_rows_to_read, Columns & res_columns) override; bool canReadIncompleteGranules() const override { return true; } diff --git a/src/Storages/MergeTree/MergeTreeReaderStream.cpp b/src/Storages/MergeTree/MergeTreeReaderStream.cpp index 04b1411d939..707a8c85c73 100644 --- a/src/Storages/MergeTree/MergeTreeReaderStream.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderStream.cpp @@ -1,6 +1,7 @@ #include #include +#include #include @@ -12,68 +13,38 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; } - MergeTreeReaderStream::MergeTreeReaderStream( DiskPtr disk_, const String & path_prefix_, const String & data_file_extension_, size_t marks_count_, const MarkRanges & all_mark_ranges, const MergeTreeReaderSettings & settings, MarkCache * mark_cache_, - UncompressedCache * uncompressed_cache, size_t file_size, + UncompressedCache * uncompressed_cache, size_t file_size_, const MergeTreeIndexGranularityInfo * index_granularity_info_, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type) - : disk(std::move(disk_)), path_prefix(path_prefix_), data_file_extension(data_file_extension_), marks_count(marks_count_) - , mark_cache(mark_cache_), save_marks_in_cache(settings.save_marks_in_cache) - , index_granularity_info(index_granularity_info_) - , marks_loader(disk, mark_cache, index_granularity_info->getMarksFilePath(path_prefix), - marks_count, *index_granularity_info, save_marks_in_cache) + : disk(std::move(disk_)) + , path_prefix(path_prefix_) + , data_file_extension(data_file_extension_) + , marks_count(marks_count_) + , file_size(file_size_) + , mark_cache(mark_cache_) + , save_marks_in_cache(settings.save_marks_in_cache) + , index_granularity_info(index_granularity_info_) + , marks_loader(disk, mark_cache, index_granularity_info->getMarksFilePath(path_prefix), + marks_count, *index_granularity_info, save_marks_in_cache) { /// Compute the size of the buffer. size_t max_mark_range_bytes = 0; size_t sum_mark_range_bytes = 0; - /// Rightmost bound to read. - size_t right_bound = 0; for (const auto & mark_range : all_mark_ranges) { size_t left_mark = mark_range.begin; size_t right_mark = mark_range.end; - - /// NOTE: if we are reading the whole file, then right_mark == marks_count - /// and we will use max_read_buffer_size for buffer size, thus avoiding the need to load marks. - - /// If the end of range is inside the block, we will need to read it too. - if (right_mark < marks_count && marks_loader.getMark(right_mark).offset_in_decompressed_block > 0) - { - auto indices = collections::range(right_mark, marks_count); - auto it = std::upper_bound(indices.begin(), indices.end(), right_mark, [this](size_t i, size_t j) - { - return marks_loader.getMark(i).offset_in_compressed_file < marks_loader.getMark(j).offset_in_compressed_file; - }); - - right_mark = (it == indices.end() ? marks_count : *it); - } - - size_t mark_range_bytes; - size_t current_right_offset; - - /// If there are no marks after the end of range, just use file size - if (right_mark >= marks_count - || (right_mark + 1 == marks_count - && marks_loader.getMark(right_mark).offset_in_compressed_file == marks_loader.getMark(mark_range.end).offset_in_compressed_file)) - { - mark_range_bytes = file_size - (left_mark < marks_count ? marks_loader.getMark(left_mark).offset_in_compressed_file : 0); - current_right_offset = file_size; - } - else - { - mark_range_bytes = marks_loader.getMark(right_mark).offset_in_compressed_file - marks_loader.getMark(left_mark).offset_in_compressed_file; - current_right_offset = marks_loader.getMark(right_mark).offset_in_compressed_file; - } + auto [right_offset, mark_range_bytes] = getRightOffsetAndBytesRange(left_mark, right_mark); max_mark_range_bytes = std::max(max_mark_range_bytes, mark_range_bytes); sum_mark_range_bytes += mark_range_bytes; - right_bound = std::max(right_bound, current_right_offset); } /// Avoid empty buffer. May happen while reading dictionary for DataTypeLowCardinality. @@ -82,9 +53,6 @@ MergeTreeReaderStream::MergeTreeReaderStream( if (max_mark_range_bytes != 0) read_settings = read_settings.adjustBufferSize(max_mark_range_bytes); - /// Set bound for reading from remote disk. - read_settings.remote_read_right_offset = right_bound; - /// Initialize the objects that shall be used to perform read operations. if (uncompressed_cache) { @@ -128,6 +96,45 @@ MergeTreeReaderStream::MergeTreeReaderStream( } +std::pair MergeTreeReaderStream::getRightOffsetAndBytesRange(size_t left_mark, size_t right_mark) +{ + /// NOTE: if we are reading the whole file, then right_mark == marks_count + /// and we will use max_read_buffer_size for buffer size, thus avoiding the need to load marks. + + /// If the end of range is inside the block, we will need to read it too. + size_t result_right_mark = right_mark; + if (right_mark < marks_count && marks_loader.getMark(right_mark).offset_in_decompressed_block > 0) + { + auto indices = collections::range(right_mark, marks_count); + auto it = std::upper_bound(indices.begin(), indices.end(), right_mark, [this](size_t i, size_t j) + { + return marks_loader.getMark(i).offset_in_compressed_file < marks_loader.getMark(j).offset_in_compressed_file; + }); + + result_right_mark = (it == indices.end() ? marks_count : *it); + } + + size_t right_offset; + size_t mark_range_bytes; + + /// If there are no marks after the end of range, just use file size + if (result_right_mark >= marks_count + || (result_right_mark + 1 == marks_count + && marks_loader.getMark(result_right_mark).offset_in_compressed_file == marks_loader.getMark(right_mark).offset_in_compressed_file)) + { + right_offset = file_size; + mark_range_bytes = right_offset - (left_mark < marks_count ? marks_loader.getMark(left_mark).offset_in_compressed_file : 0); + } + else + { + right_offset = marks_loader.getMark(result_right_mark).offset_in_compressed_file; + mark_range_bytes = right_offset - marks_loader.getMark(left_mark).offset_in_compressed_file; + } + + return std::make_pair(right_offset, mark_range_bytes); +} + + void MergeTreeReaderStream::seekToMark(size_t index) { MarkInCompressedFile mark = marks_loader.getMark(index); @@ -172,4 +179,18 @@ void MergeTreeReaderStream::seekToStart() } } + +void MergeTreeReaderStream::adjustForRange(size_t left_mark, size_t right_mark) +{ + auto [right_offset, mark_range_bytes] = getRightOffsetAndBytesRange(left_mark, right_mark); + if (right_offset > last_right_offset) + { + last_right_offset = right_offset; + if (cached_buffer) + cached_buffer->setRightOffset(last_right_offset); + if (non_cached_buffer) + non_cached_buffer->setRightOffset(last_right_offset); + } +} + } diff --git a/src/Storages/MergeTree/MergeTreeReaderStream.h b/src/Storages/MergeTree/MergeTreeReaderStream.h index 32b9c45ccab..b8244d6252f 100644 --- a/src/Storages/MergeTree/MergeTreeReaderStream.h +++ b/src/Storages/MergeTree/MergeTreeReaderStream.h @@ -23,25 +23,32 @@ public: const MarkRanges & all_mark_ranges, const MergeTreeReaderSettings & settings_, MarkCache * mark_cache, UncompressedCache * uncompressed_cache, - size_t file_size, const MergeTreeIndexGranularityInfo * index_granularity_info_, + size_t file_size_, const MergeTreeIndexGranularityInfo * index_granularity_info_, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type); void seekToMark(size_t index); void seekToStart(); + void adjustForRange(size_t left_mark, size_t right_mark); + ReadBuffer * data_buffer; private: + std::pair getRightOffsetAndBytesRange(size_t left_mark, size_t right_mark); + DiskPtr disk; std::string path_prefix; std::string data_file_extension; size_t marks_count; + size_t file_size; MarkCache * mark_cache; bool save_marks_in_cache; + size_t last_right_offset = 0; + const MergeTreeIndexGranularityInfo * index_granularity_info; std::unique_ptr cached_buffer; diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 206469da7be..a47563397be 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -61,7 +61,8 @@ MergeTreeReaderWide::MergeTreeReaderWide( } -size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) +size_t MergeTreeReaderWide::readRows( + size_t from_mark, size_t current_task_last_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) { size_t read_rows = 0; try @@ -87,7 +88,7 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si try { auto & cache = caches[column_from_part.getNameInStorage()]; - prefetch(column_from_part, from_mark, continue_reading, cache, prefetched_streams); + prefetch(column_from_part, from_mark, continue_reading, current_task_last_mark, cache, prefetched_streams); } catch (Exception & e) { @@ -117,7 +118,7 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si auto & cache = caches[column_from_part.getNameInStorage()]; readData( - column_from_part, column, from_mark, continue_reading, + column_from_part, column, from_mark, continue_reading, current_task_last_mark, max_rows_to_read, cache, /* was_prefetched =*/ !prefetched_streams.empty()); /// For elements of Nested, column_size_before_reading may be greater than column size @@ -199,6 +200,7 @@ static ReadBuffer * getStream( MergeTreeReaderWide::FileStreams & streams, const NameAndTypePair & name_and_type, size_t from_mark, bool seek_to_mark, + size_t current_task_last_mark, ISerialization::SubstreamsCache & cache) { /// If substream have already been read. @@ -212,6 +214,7 @@ static ReadBuffer * getStream( return nullptr; MergeTreeReaderStream & stream = *it->second; + stream.adjustForRange(seek_to_start ? 0 : from_mark, current_task_last_mark); if (seek_to_start) stream.seekToStart(); @@ -226,6 +229,7 @@ void MergeTreeReaderWide::prefetch( const NameAndTypePair & name_and_type, size_t from_mark, bool continue_reading, + size_t current_task_last_mark, ISerialization::SubstreamsCache & cache, std::unordered_set & prefetched_streams) { @@ -239,7 +243,7 @@ void MergeTreeReaderWide::prefetch( if (!prefetched_streams.count(stream_name)) { bool seek_to_mark = !continue_reading; - if (ReadBuffer * buf = getStream(false, substream_path, streams, name_and_type, from_mark, seek_to_mark, cache)) + if (ReadBuffer * buf = getStream(false, substream_path, streams, name_and_type, from_mark, seek_to_mark, current_task_last_mark, cache)) buf->prefetch(); prefetched_streams.insert(stream_name); @@ -250,8 +254,8 @@ void MergeTreeReaderWide::prefetch( void MergeTreeReaderWide::readData( const NameAndTypePair & name_and_type, ColumnPtr & column, - size_t from_mark, bool continue_reading, size_t max_rows_to_read, - ISerialization::SubstreamsCache & cache, bool was_prefetched) + size_t from_mark, bool continue_reading, size_t current_task_last_mark, + size_t max_rows_to_read, ISerialization::SubstreamsCache & cache, bool was_prefetched) { double & avg_value_size_hint = avg_value_size_hints[name_and_type.name]; ISerialization::DeserializeBinaryBulkSettings deserialize_settings; @@ -264,7 +268,7 @@ void MergeTreeReaderWide::readData( { deserialize_settings.getter = [&](const ISerialization::SubstreamPath & substream_path) { - return getStream(/* seek_to_start = */true, substream_path, streams, name_and_type, from_mark, /* seek_to_mark = */false, cache); + return getStream(/* seek_to_start = */true, substream_path, streams, name_and_type, from_mark, /* seek_to_mark = */false, current_task_last_mark, cache); }; serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, deserialize_binary_bulk_state_map[name]); } @@ -275,7 +279,7 @@ void MergeTreeReaderWide::readData( return getStream( /* seek_to_start = */false, substream_path, streams, name_and_type, from_mark, - seek_to_mark, cache); + seek_to_mark, current_task_last_mark, cache); }; deserialize_settings.continuous_reading = continue_reading; auto & deserialize_state = deserialize_binary_bulk_state_map[name]; diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.h b/src/Storages/MergeTree/MergeTreeReaderWide.h index 08d743370a9..a71475acd60 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.h +++ b/src/Storages/MergeTree/MergeTreeReaderWide.h @@ -28,7 +28,8 @@ public: /// Return the number of rows has been read or zero if there is no columns to read. /// If continue_reading is true, continue reading from last state, otherwise seek to from_mark - size_t readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) override; + size_t readRows(size_t from_mark, size_t current_task_last_mark, + bool continue_reading, size_t max_rows_to_read, Columns & res_columns) override; bool canReadIncompleteGranules() const override { return true; } @@ -39,13 +40,14 @@ private: FileStreams streams; Serializations serializations; DiskPtr disk; + std::map> marks; void addStreams(const NameAndTypePair & name_and_type, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type); void readData( const NameAndTypePair & name_and_type, ColumnPtr & column, - size_t from_mark, bool continue_reading, size_t max_rows_to_read, + size_t from_mark, bool continue_reading, size_t current_task_last_mark, size_t max_rows_to_read, ISerialization::SubstreamsCache & cache, bool was_prefetched); /// Make next readData more simple by calling 'prefetch' of all related ReadBuffers (column streams). @@ -53,6 +55,7 @@ private: const NameAndTypePair & name_and_type, size_t from_mark, bool continue_reading, + size_t current_task_last_mark, ISerialization::SubstreamsCache & cache, std::unordered_set & prefetched_streams); /// if stream was already prefetched do nothing }; diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 9ea9383c7f0..df8d6a7c127 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -78,7 +78,8 @@ try const auto & sample = reader->getColumns(); Columns columns(sample.size()); - size_t rows_read = reader->readRows(current_mark, continue_reading, rows_to_read, columns); + /// TODO: pass stream size instead of zero? + size_t rows_read = reader->readRows(current_mark, 0, continue_reading, rows_to_read, columns); if (rows_read) { From c1c574e9ca3476d16260ef1c9de2815ebf7395ad Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 15 Oct 2021 13:13:38 +0300 Subject: [PATCH 161/919] Better --- src/Disks/ReadBufferFromRemoteFSGather.cpp | 1 - ...chronousReadIndirectBufferFromRemoteFS.cpp | 75 +++++++++++----- ...ynchronousReadIndirectBufferFromRemoteFS.h | 5 +- src/IO/ReadBufferFromS3.cpp | 87 +++++++++---------- 4 files changed, 98 insertions(+), 70 deletions(-) diff --git a/src/Disks/ReadBufferFromRemoteFSGather.cpp b/src/Disks/ReadBufferFromRemoteFSGather.cpp index 67ba4448d20..500d6b651b1 100644 --- a/src/Disks/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/ReadBufferFromRemoteFSGather.cpp @@ -163,7 +163,6 @@ void ReadBufferFromRemoteFSGather::seek(off_t offset) { current_buf.reset(); absolute_position = offset; - // initialize(); } diff --git a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp index 9dd3aeb4625..907b75f0417 100644 --- a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -36,13 +36,13 @@ AsynchronousReadIndirectBufferFromRemoteFS::AsynchronousReadIndirectBufferFromRe Int32 priority_, std::shared_ptr impl_, size_t buf_size_, - size_t /* min_bytes_for_seek_ */) + size_t min_bytes_for_seek_) : ReadBufferFromFileBase(buf_size_, nullptr, 0) , reader(reader_) , priority(priority_) , impl(impl_) , prefetch_buffer(buf_size_) - // , min_bytes_for_seek(min_bytes_for_seek_) + , min_bytes_for_seek(min_bytes_for_seek_) { ProfileEvents::increment(ProfileEvents::RemoteFSAsyncBuffers); buffer_events += impl->getFileName() + " : "; @@ -69,15 +69,23 @@ std::future AsynchronousReadIndirectBufferFromRemot void AsynchronousReadIndirectBufferFromRemoteFS::prefetch() { - if (hasPendingData()) - return; + if (prefetch_future.valid()) + return; - if (prefetch_future.valid()) - return; + /// Everything is already read. + if (absolute_position == last_offset) + return; - prefetch_future = readInto(prefetch_buffer.data(), prefetch_buffer.size()); - ProfileEvents::increment(ProfileEvents::RemoteFSPrefetches); - buffer_events += "-- Prefetch (" + toString(absolute_position) + ") --"; + if (absolute_position > last_offset) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Read beyond last offset ({} > {})", + absolute_position, last_offset); + + /// Prefetch even in case hasPendingData() == true. + prefetch_future = readInto(prefetch_buffer.data(), prefetch_buffer.size()); + ProfileEvents::increment(ProfileEvents::RemoteFSPrefetches); + + buffer_events += fmt::format("-- PREFETCH from offset: {}, upper bound: {} --", + toString(absolute_position), toString(last_offset)); } @@ -86,10 +94,15 @@ void AsynchronousReadIndirectBufferFromRemoteFS::setRightOffset(size_t offset) buffer_events += "-- Set last offset " + toString(offset) + "--"; if (prefetch_future.valid()) { - buffer_events += "-- Cancelling because of offset update --"; - ProfileEvents::increment(ProfileEvents::RemoteFSSeekCancelledPrefetches); - prefetch_future.wait(); - prefetch_future = {}; + std::cerr << buffer_events << std::endl; + /// TODO: Planning to put logical error here after more testing, + // because seems like future is never supposed to be valid at this point. + std::terminate(); + + // buffer_events += "-- Cancelling because of offset update --"; + // ProfileEvents::increment(ProfileEvents::RemoteFSSeekCancelledPrefetches); + // prefetch_future.wait(); + // prefetch_future = {}; } last_offset = offset; @@ -99,6 +112,14 @@ void AsynchronousReadIndirectBufferFromRemoteFS::setRightOffset(size_t offset) bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() { + /// Everything is already read. + if (absolute_position == last_offset) + return false; + + if (absolute_position > last_offset) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Read beyond last offset ({} > {})", + absolute_position, last_offset); + ProfileEvents::increment(ProfileEvents::RemoteFSAsyncBufferReads); size_t size = 0; @@ -138,6 +159,13 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() } prefetch_future = {}; + + /// TODO: it does not really seem to improve anything to call prefecth() here, + /// but it does not make any worse at the same time. + /// Need to test, it might be useful because in fact sometimes (minority of cases though) + /// we can read without prefetching several times in a row. + prefetch(); + return size; } @@ -192,16 +220,17 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence pos = working_buffer.end(); - // if (static_cast(absolute_position) >= getPosition() - // && static_cast(absolute_position) < getPosition() + static_cast(min_bytes_for_seek)) - // { - // /** - // * Lazy ignore. Save number of bytes to ignore and ignore it either for prefetch buffer or current buffer. - // */ - // // bytes_to_ignore = absolute_position - getPosition(); - // impl->seek(absolute_position); /// SEEK_SET. - // } - // else + /// Note: we read in range [absolute_position, last_offset). + if (absolute_position < last_offset + && static_cast(absolute_position) >= getPosition() + && static_cast(absolute_position) < getPosition() + static_cast(min_bytes_for_seek)) + { + /** + * Lazy ignore. Save number of bytes to ignore and ignore it either for prefetch buffer or current buffer. + */ + bytes_to_ignore = absolute_position - getPosition(); + } + else { buffer_events += "-- Impl seek --"; impl->seek(absolute_position); /// SEEK_SET. diff --git a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h index af67efe1218..2dcd89016ed 100644 --- a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -70,10 +70,11 @@ private: String buffer_events; - // size_t min_bytes_for_seek; + size_t min_bytes_for_seek; size_t bytes_to_ignore = 0; - Int64 last_offset = 0; + + size_t last_offset = 0; }; } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 40bda7d6bfe..f01fd9a87a1 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -2,15 +2,17 @@ #if USE_AWS_S3 -# include -# include -# include +#include +#include +#include -# include -# include -# include +#include +#include -# include +#include +#include + +#include namespace ProfileEvents @@ -27,6 +29,7 @@ namespace ErrorCodes extern const int S3_ERROR; extern const int CANNOT_SEEK_THROUGH_FILE; extern const int SEEK_POSITION_OUT_OF_BOUND; + extern const int LOGICAL_ERROR; } @@ -49,19 +52,29 @@ bool ReadBufferFromS3::nextImpl() if (last_offset) { if (static_cast(last_offset) == offset) - { - impl.reset(); - working_buffer.resize(0); return false; - } + + if (static_cast(last_offset) < offset) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyound right offset ({} > {})", offset, last_offset - 1); } bool next_result = false; - /// `impl` has been initialized earlier and now we're at the end of the current portion of data. if (impl) { - if (!use_external_buffer) + if (use_external_buffer) + { + /** + * use_external_buffer -- means we read into the buffer which + * was passed to us from somewhere else. We do not check whether + * previously returned buffer was read or not, because this branch + * means we are prefetching data. + */ + impl->set(internal_buffer.begin(), internal_buffer.size()); + assert(working_buffer.begin() != nullptr); + assert(!internal_buffer.empty()); + } + else { /** * use_external_buffer -- means we read into the buffer which @@ -74,32 +87,25 @@ bool ReadBufferFromS3::nextImpl() assert(!impl->hasPendingData()); } } - else - { - /// `impl` is not initialized and we're about to read the first portion of data. - impl = initialize(); - next_result = impl->hasPendingData(); - } - if (use_external_buffer) - { - /** - * use_external_buffer -- means we read into the buffer which - * was passed to us from somewhere else. We do not check whether - * previously returned buffer was read or not, because this branch - * means we are prefetching data. - */ - impl->set(internal_buffer.begin(), internal_buffer.size()); - assert(working_buffer.begin() != nullptr); - assert(!internal_buffer.empty()); - } - - auto sleep_time_with_backoff_milliseconds = std::chrono::milliseconds(100); + size_t sleep_time_with_backoff_milliseconds = 100; for (size_t attempt = 0; (attempt < max_single_read_retries) && !next_result; ++attempt) { Stopwatch watch; try { + if (!impl) + { + impl = initialize(); + + if (use_external_buffer) + { + impl->set(internal_buffer.begin(), internal_buffer.size()); + assert(working_buffer.begin() != nullptr); + assert(!internal_buffer.empty()); + } + } + /// Try to read a next portion of data. next_result = impl->next(); watch.stop(); @@ -119,19 +125,11 @@ bool ReadBufferFromS3::nextImpl() throw; /// Pause before next attempt. - std::this_thread::sleep_for(sleep_time_with_backoff_milliseconds); + sleepForMilliseconds(sleep_time_with_backoff_milliseconds); sleep_time_with_backoff_milliseconds *= 2; /// Try to reinitialize `impl`. impl.reset(); - impl = initialize(); - if (use_external_buffer) - { - impl->set(internal_buffer.begin(), internal_buffer.size()); - assert(working_buffer.begin() != nullptr); - assert(!internal_buffer.empty()); - } - next_result = impl->hasPendingData(); } } @@ -173,10 +171,11 @@ std::unique_ptr ReadBufferFromS3::initialize() req.SetBucket(bucket); req.SetKey(key); - // auto right_offset = read_settings.remote_read_right_offset; - if (last_offset) { + if (offset >= static_cast(last_offset)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyound right offset ({} > {})", offset, last_offset - 1); + req.SetRange(fmt::format("bytes={}-{}", offset, last_offset - 1)); LOG_DEBUG(log, "Read S3 object. Bucket: {}, Key: {}, Range: {}-{}", bucket, key, offset, last_offset - 1); } From edfb1a5521d83dd9722d2d705dba5284b6534270 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 15 Oct 2021 15:07:39 +0300 Subject: [PATCH 162/919] Better performance for smaller number of threads --- src/Common/ProfileEvents.cpp | 2 +- ...chronousReadIndirectBufferFromRemoteFS.cpp | 4 +-- src/IO/ReadBufferFromS3.cpp | 10 +++---- src/IO/ReadBufferFromS3.h | 2 +- src/IO/ReadBufferFromWebServer.cpp | 26 +++++++++++++++---- src/IO/ReadBufferFromWebServer.h | 2 ++ src/IO/ReadSettings.h | 2 -- src/IO/ReadWriteBufferFromHTTP.h | 10 +++---- src/Storages/MergeTree/MergeTreeReadPool.cpp | 15 ++++++++--- src/Storages/MergeTree/MergeTreeReadPool.h | 2 ++ 10 files changed, 51 insertions(+), 24 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index f9645875a6c..24dcbacebf9 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -255,7 +255,7 @@ M(RemoteFSSeeks, "Total number of seeks for async buffer") \ M(RemoteFSPrefetches, "Total number of prefetches") \ M(RemoteFSSeekCancelledPrefetches, "Number of cancelled prefecthes because of seek") \ - M(RemoteFSUnusedCancelledPrefetches, "Number of prefetches prending in buffer desctructor") \ + M(RemoteFSUnusedCancelledPrefetches, "Number of prefetches pending in buffer destructor") \ M(RemoteFSPrefetchReads, "Total number of reads from prefecthed buffer") \ M(RemoteFSAsyncBufferReads, "Number of nextImpl() calls for async buffer") \ M(RemoteFSSimpleBufferReads, "Number of nextImpl() calls for non-async buffer") \ diff --git a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp index 907b75f0417..0325d7f79b7 100644 --- a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -27,7 +27,7 @@ namespace DB namespace ErrorCodes { - extern const int CANNOT_SEEK_THROUGH_FILE; + extern const int LOGICAL_ERROR; } @@ -160,7 +160,7 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() prefetch_future = {}; - /// TODO: it does not really seem to improve anything to call prefecth() here, + /// TODO: it does not really seem to improve anything to call prefetch() here, /// but it does not make any worse at the same time. /// Need to test, it might be useful because in fact sometimes (minority of cases though) /// we can read without prefetching several times in a row. diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index f01fd9a87a1..bf578373535 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -51,11 +51,11 @@ bool ReadBufferFromS3::nextImpl() { if (last_offset) { - if (static_cast(last_offset) == offset) + if (last_offset == offset) return false; - if (static_cast(last_offset) < offset) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyound right offset ({} > {})", offset, last_offset - 1); + if (last_offset < offset) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, last_offset - 1); } bool next_result = false; @@ -173,8 +173,8 @@ std::unique_ptr ReadBufferFromS3::initialize() if (last_offset) { - if (offset >= static_cast(last_offset)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyound right offset ({} > {})", offset, last_offset - 1); + if (offset >= last_offset) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, last_offset - 1); req.SetRange(fmt::format("bytes={}-{}", offset, last_offset - 1)); LOG_DEBUG(log, "Read S3 object. Bucket: {}, Key: {}, Range: {}-{}", bucket, key, offset, last_offset - 1); diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 336893ca5b0..fae3938bf72 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -59,7 +59,7 @@ private: ReadSettings read_settings; bool use_external_buffer; - size_t last_offset; + off_t last_offset; }; } diff --git a/src/IO/ReadBufferFromWebServer.cpp b/src/IO/ReadBufferFromWebServer.cpp index 41b4c80b2c8..245364a9896 100644 --- a/src/IO/ReadBufferFromWebServer.cpp +++ b/src/IO/ReadBufferFromWebServer.cpp @@ -23,7 +23,11 @@ namespace ErrorCodes static constexpr size_t HTTP_MAX_TRIES = 10; ReadBufferFromWebServer::ReadBufferFromWebServer( - const String & url_, ContextPtr context_, const ReadSettings & settings_, bool use_external_buffer_, size_t) + const String & url_, + ContextPtr context_, + const ReadSettings & settings_, + bool use_external_buffer_, + size_t last_offset_) : SeekableReadBuffer(nullptr, 0) , log(&Poco::Logger::get("ReadBufferFromWebServer")) , context(context_) @@ -31,6 +35,7 @@ ReadBufferFromWebServer::ReadBufferFromWebServer( , buf_size(settings_.remote_fs_buffer_size) , read_settings(settings_) , use_external_buffer(use_external_buffer_) + , last_offset(last_offset_) { } @@ -41,11 +46,13 @@ std::unique_ptr ReadBufferFromWebServer::initialize() ReadWriteBufferFromHTTP::HTTPHeaderEntries headers; - auto right_offset = read_settings.remote_read_right_offset; - if (right_offset) + if (last_offset) { - headers.emplace_back(std::make_pair("Range", fmt::format("bytes={}-{}", offset, right_offset))); - LOG_DEBUG(log, "Reading with range: {}-{}", offset, right_offset); + if (last_offset < offset) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyound right offset ({} > {})", offset, last_offset - 1); + + headers.emplace_back(std::make_pair("Range", fmt::format("bytes={}-{}", offset, last_offset - 1))); + LOG_DEBUG(log, "Reading with range: {}-{}", offset, last_offset); } else { @@ -120,6 +127,15 @@ void ReadBufferFromWebServer::initializeWithRetry() bool ReadBufferFromWebServer::nextImpl() { + if (last_offset) + { + if (last_offset == offset) + return false; + + if (last_offset < offset) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyound right offset ({} > {})", offset, last_offset - 1); + } + if (impl) { if (use_external_buffer) diff --git a/src/IO/ReadBufferFromWebServer.h b/src/IO/ReadBufferFromWebServer.h index c4d847b9f39..1ffb8589392 100644 --- a/src/IO/ReadBufferFromWebServer.h +++ b/src/IO/ReadBufferFromWebServer.h @@ -47,6 +47,8 @@ private: ReadSettings read_settings; bool use_external_buffer; + + off_t last_offset = 0; }; } diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index 0548659e16e..edd5463bd7c 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -77,8 +77,6 @@ struct ReadSettings size_t remote_fs_read_max_backoff_ms = 10000; size_t remote_fs_read_backoff_max_tries = 4; - size_t remote_read_right_offset = 0; /// Right offset for range reading. - bool http_retriable_read = true; size_t http_max_tries = 1; size_t http_retry_initial_backoff_ms = 100; diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index f28fcb60497..ca44fc2583d 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -265,6 +265,9 @@ namespace detail if (next_callback) next_callback(count()); + if (total_bytes_to_read && bytes_read == total_bytes_to_read.value()) + return false; + if (impl) { if (use_external_buffer) @@ -284,7 +287,7 @@ namespace detail { /** * impl was initialized before, pass position() to it to make - * sure there is no pending data which was not read, becuase + * sure there is no pending data which was not read, because * this branch means we read sequentially. */ if (!working_buffer.empty()) @@ -292,9 +295,6 @@ namespace detail } } - if (total_bytes_to_read && bytes_read == total_bytes_to_read.value()) - return false; - if (impl && !working_buffer.empty()) impl->position() = position(); @@ -322,7 +322,7 @@ namespace detail || (bytes_read && !settings.http_retriable_read)) throw; - LOG_ERROR(&Poco::Logger::get("ReadBufferFromHTTP"), e.what()); + LOG_ERROR(&Poco::Logger::get("ReadBufferFromHTTP"), "Error: {}, code: {}", e.what(), e.code()); impl.reset(); sleepForMilliseconds(milliseconds_to_wait); diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index f23c58bc2b3..f9157317d2d 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -42,6 +42,12 @@ MergeTreeReadPool::MergeTreeReadPool( { /// parts don't contain duplicate MergeTreeDataPart's. const auto per_part_sum_marks = fillPerPartInfo(parts_ranges, check_columns_); + auto min_marks_for_concurrent_read = min_marks_for_concurrent_read_; + if (stored_on_remote_disk) + { + do_not_steal_tasks = true; + min_marks_for_concurrent_read = std::max(min_marks_for_concurrent_read, sum_marks_ / threads_); + } fillPerThreadInfo(threads_, sum_marks_, per_part_sum_marks, parts_ranges, min_marks_for_concurrent_read_); } @@ -89,8 +95,11 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(const size_t min_marks_to_read, auto & part = parts_with_idx[part_idx]; auto & marks_in_part = thread_tasks.sum_marks_in_parts.back(); - /// Get whole part to read if it is small enough. - auto need_marks = std::min(marks_in_part, min_marks_to_read); + size_t need_marks; + if (stored_on_remote_disk) /// For better performance with remote disks + need_marks = marks_in_part; + else /// Get whole part to read if it is small enough. + need_marks = std::min(marks_in_part, min_marks_to_read); /// Do not leave too little rows in part for next time. if (marks_in_part > need_marks && @@ -223,7 +232,7 @@ std::vector MergeTreeReadPool::fillPerPartInfo( /// Turn off tasks stealing in case there is remote disk. if (part.data_part->isStoredOnRemoteDisk()) - do_not_steal_tasks = true; + stored_on_remote_disk = true; /// Read marks for every data part. size_t sum_marks = 0; diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index 9949bdf86f8..db04208aa7a 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -139,6 +139,8 @@ private: mutable std::mutex mutex; Poco::Logger * log = &Poco::Logger::get("MergeTreeReadPool"); + + bool stored_on_remote_disk = false; }; using MergeTreeReadPoolPtr = std::shared_ptr; From c67e04b3a5d3d7a5e6a626ac8b35f8ee5e6b011e Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 15 Oct 2021 16:09:34 +0300 Subject: [PATCH 163/919] Fix Raw serialization for LowCardinality --- .../Serializations/SerializationLowCardinality.cpp | 11 +++++++++++ .../Serializations/SerializationLowCardinality.h | 2 ++ 2 files changed, 13 insertions(+) diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.cpp b/src/DataTypes/Serializations/SerializationLowCardinality.cpp index e9bb62f74c5..8abbb56d116 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.cpp +++ b/src/DataTypes/Serializations/SerializationLowCardinality.cpp @@ -802,6 +802,7 @@ void SerializationLowCardinality::serializeTextJSON(const IColumn & column, size { serializeImpl(column, row_num, &ISerialization::serializeTextJSON, ostr, settings); } + void SerializationLowCardinality::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { deserializeImpl(column, &ISerialization::deserializeTextJSON, istr, settings); @@ -812,6 +813,16 @@ void SerializationLowCardinality::serializeTextXML(const IColumn & column, size_ serializeImpl(column, row_num, &ISerialization::serializeTextXML, ostr, settings); } +void SerializationLowCardinality::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + deserializeImpl(column, &ISerialization::deserializeTextJSON, istr, settings); +} + +void SerializationLowCardinality::serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + serializeImpl(column, row_num, &ISerialization::serializeTextXML, ostr, settings); +} + template void SerializationLowCardinality::serializeImpl( const IColumn & column, size_t row_num, SerializationLowCardinality::SerializeFunctionPtr func, Args &&... args) const diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.h b/src/DataTypes/Serializations/SerializationLowCardinality.h index e9ca0349e38..b9978985953 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.h +++ b/src/DataTypes/Serializations/SerializationLowCardinality.h @@ -60,6 +60,8 @@ public: void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; private: template From 72a1b928cf80d8e3e6471021875e80dc0aef2980 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 15 Oct 2021 14:04:31 +0000 Subject: [PATCH 164/919] Update install.sh --- tests/config/install.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/install.sh b/tests/config/install.sh index 06a4f1b3f88..04b655d8769 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -48,6 +48,7 @@ ln -sf $SRC_PATH/users.d/database_atomic_drop_detach_sync.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/users.d/opentelemetry.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/remote_queries.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/session_log_test.xml $DEST_SERVER_PATH/users.d/ +ln -sf $SRC_PATH/users.d/memory_profiler.xml $DEST_SERVER_PATH/users.d/ # FIXME DataPartsExchange may hang for http_send_timeout seconds # when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"), From 759f747080cc264ca292f66e37ddadfa24f16dd8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 15 Oct 2021 17:12:34 +0300 Subject: [PATCH 165/919] Fix build --- src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp | 1 + src/Storages/MergeTree/MergeTreeReaderWide.h | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp index 0325d7f79b7..ce836082130 100644 --- a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -28,6 +28,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int CANNOT_SEEK_THROUGH_FILE; } diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.h b/src/Storages/MergeTree/MergeTreeReaderWide.h index a71475acd60..e27dd85643b 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.h +++ b/src/Storages/MergeTree/MergeTreeReaderWide.h @@ -40,7 +40,6 @@ private: FileStreams streams; Serializations serializations; DiskPtr disk; - std::map> marks; void addStreams(const NameAndTypePair & name_and_type, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type); From 370019e13d8b0a784466f7bca7667c0d51648845 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 15 Oct 2021 18:27:18 +0300 Subject: [PATCH 166/919] Fix typos --- src/DataTypes/Serializations/SerializationLowCardinality.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.cpp b/src/DataTypes/Serializations/SerializationLowCardinality.cpp index 8abbb56d116..9867ded84cd 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.cpp +++ b/src/DataTypes/Serializations/SerializationLowCardinality.cpp @@ -815,12 +815,12 @@ void SerializationLowCardinality::serializeTextXML(const IColumn & column, size_ void SerializationLowCardinality::deserializeTextRaw(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - deserializeImpl(column, &ISerialization::deserializeTextJSON, istr, settings); + deserializeImpl(column, &ISerialization::deserializeTextRaw, istr, settings); } void SerializationLowCardinality::serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - serializeImpl(column, row_num, &ISerialization::serializeTextXML, ostr, settings); + serializeImpl(column, row_num, &ISerialization::serializeTextRaw, ostr, settings); } template From 97d228f64b830833e309f57b11e0add6e21531eb Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 15 Oct 2021 23:03:49 +0300 Subject: [PATCH 167/919] Update mapPopulateSeries Update mapPopulateSeries --- .../functions/tuple-map-functions.md | 13 ++--- .../functions/tuple-map-functions.md | 50 ++++++++++++++++--- 2 files changed, 51 insertions(+), 12 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index 6ddac9a0530..5510fbef226 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -22,7 +22,7 @@ map(key1, value1[, key2, value2, ...]) **Returned value** -- Data structure as `key:value` pairs. +- Data structure as `key:value` pairs. Type: [Map(key, value)](../../sql-reference/data-types/map.md). @@ -165,9 +165,6 @@ Result: ## mapPopulateSeries {#function-mappopulateseries} Fills missing keys in the maps (key and value array pair), where keys are integers. Also, it supports specifying the max key, which is used to extend the keys array. -Arguments are [maps](../../sql-reference/data-types/map.md) or two [arrays](../../sql-reference/data-types/array.md#data-type-array), where the first array represent keys, and the second array contains values for the each key. - -For array arguments the number of elements in `keys` and `values` must be the same for each row. **Syntax** @@ -176,8 +173,12 @@ mapPopulateSeries(keys, values[, max]) mapPopulateSeries(map[, max]) ``` +Arguments are [maps](../../sql-reference/data-types/map.md) or two [arrays](../../sql-reference/data-types/array.md#data-type-array), where the first array represent keys, and the second array contains values for the each key. + Generates a map (a tuple with two arrays or a value of `Map` type, depending on the arguments), where keys are a series of numbers, from minimum to maximum keys (or `max` argument if it specified) taken from the map with a step size of one, and corresponding values. If the value is not specified for the key, then it uses the default value in the resulting map. For repeated keys, only the first value (in order of appearing) gets associated with the key. +For array arguments the number of elements in `keys` and `values` must be the same for each row. + **Arguments** Mapped arrays: @@ -191,14 +192,14 @@ or **Returned value** -- Depending on the arguments returns a [map](../../sql-reference/data-types/map.md) or a [tuple](../../sql-reference/data-types/tuple.md#tuplet1-t2) of two [arrays](../../sql-reference/data-types/array.md#data-type-array): keys in sorted order, and values the corresponding keys. +- Depending on the arguments returns a [map](../../sql-reference/data-types/map.md) or a [tuple](../../sql-reference/data-types/tuple.md#tuplet1-t2) of two [arrays](../../sql-reference/data-types/array.md#data-type-array): keys in sorted order, and values the corresponding keys. **Example** Query with mapped arrays: ```sql -select mapPopulateSeries([1,2,4], [11,22,44], 5) as res, toTypeName(res) as type; +SELECT mapPopulateSeries([1,2,4], [11,22,44], 5) AS res, toTypeName(res) AS type; ``` Result: diff --git a/docs/ru/sql-reference/functions/tuple-map-functions.md b/docs/ru/sql-reference/functions/tuple-map-functions.md index e4cc1fefab4..8c4ffed38b4 100644 --- a/docs/ru/sql-reference/functions/tuple-map-functions.md +++ b/docs/ru/sql-reference/functions/tuple-map-functions.md @@ -108,7 +108,7 @@ SELECT mapAdd(([toUInt8(1), 2], [1, 1]), ([toUInt8(1), 2], [1, 1])) as res, toTy SELECT mapAdd(map(1,1), map(1,1)); ``` -Result: +Результат: ```text ┌─mapAdd(map(1, 1), map(1, 1))─┐ @@ -128,13 +128,13 @@ mapSubtract(Tuple(Array, Array), Tuple(Array, Array) [, ...]) **Аргументы** -Аргументами являются [кортежи](../../sql-reference/data-types/tuple.md#tuplet1-t2) из двух [массивов](../../sql-reference/data-types/array.md#data-type-array), где элементы в первом массиве представляют ключи, а второй массив содержит значения для каждого ключа. +Аргументами являются [maps](../../sql-reference/data-types/map.md) или [кортежи](../../sql-reference/data-types/tuple.md#tuplet1-t2) из двух [массивов](../../sql-reference/data-types/array.md#data-type-array), где элементы в первом массиве представляют ключи, а второй массив содержит значения для каждого ключа. Все массивы ключей должны иметь один и тот же тип, а все массивы значений должны содержать элементы, которые можно приводить к одному типу ([Int64](../../sql-reference/data-types/int-uint.md#int-ranges), [UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges) или [Float64](../../sql-reference/data-types/float.md#float32-float64)). Общий приведенный тип используется в качестве типа для результирующего массива. **Возвращаемое значение** -- Возвращает один [tuple](../../sql-reference/data-types/tuple.md#tuplet1-t2), в котором первый массив содержит отсортированные ключи, а второй - значения. +- В зависимости от аргумента возвращает [map] или [кортеж](../../sql-reference/data-types/tuple.md#tuplet1-t2), в котором первый массив содержит отсортированные ключи, а второй - значения. **Пример** @@ -152,14 +152,31 @@ SELECT mapSubtract(([toUInt8(1), 2], [toInt32(1), 1]), ([toUInt8(1), 2], [toInt3 └────────────────┴───────────────────────────────────┘ ``` +Запрос с типом `Map`: + +```sql +SELECT mapSubtract(map(1,1), map(1,1)); +``` + +Результат: + +```text +┌─mapSubtract(map(1, 1), map(1, 1))─┐ +│ {1:0} │ +└───────────────────────────────────┘ +``` + ## mapPopulateSeries {#function-mappopulateseries} Заполняет недостающие ключи в контейнере map (пара массивов ключей и значений), где ключи являются целыми числами. Кроме того, он поддерживает указание максимального ключа, который используется для расширения массива ключей. +Аргументами являются [map](../../sql-reference/data-types/map.m) или два [массива](../../sql-reference/data-types/array.md#data-type-array), где первый массив представляет ключи, а второй массив содержит значения для каждого ключа. + **Синтаксис** ``` sql mapPopulateSeries(keys, values[, max]) +mapPopulateSeries(map[, max]) ``` Генерирует контейнер map, где ключи - это серия чисел, от минимального до максимального ключа (или аргумент `max`, если он указан), взятых из массива `keys` с размером шага один, и соответствующие значения, взятые из массива `values`. Если значение не указано для ключа, то в результирующем контейнере используется значение по умолчанию. @@ -168,19 +185,25 @@ mapPopulateSeries(keys, values[, max]) **Аргументы** +Сопоставленные массивы: + - `keys` — массив ключей [Array](../../sql-reference/data-types/array.md#data-type-array)([Int](../../sql-reference/data-types/int-uint.md#int-ranges)). - `values` — массив значений. [Array](../../sql-reference/data-types/array.md#data-type-array)([Int](../../sql-reference/data-types/int-uint.md#int-ranges)). +или + +- `map` — карта с целочисленными ключами. [Map](../../sql-reference/data-types/map.md). + **Возвращаемое значение** -- Возвращает [кортеж](../../sql-reference/data-types/tuple.md#tuplet1-t2) из двух [массивов](../../sql-reference/data-types/array.md#data-type-array): ключи отсортированные по порядку и значения соответствующих ключей. +- В зависимости от аргумента возвращает [map](../../sql-reference/data-types/map.md) или [кортеж](../../sql-reference/data-types/tuple.md#tuplet1-t2) из двух [массивов](../../sql-reference/data-types/array.md#data-type-array): ключи отсортированные по порядку и значения соответствующих ключей. **Пример** -Запрос: +Запрос с сопоставленными массивами: ```sql -select mapPopulateSeries([1,2,4], [11,22,44], 5) as res, toTypeName(res) as type; +SELECT mapPopulateSeries([1,2,4], [11,22,44], 5) AS res, toTypeName(res) AS type; ``` Результат: @@ -191,6 +214,20 @@ select mapPopulateSeries([1,2,4], [11,22,44], 5) as res, toTypeName(res) as type └──────────────────────────────┴───────────────────────────────────┘ ``` +Запрос с типом `Map`: + +```sql +SELECT mapPopulateSeries(map(1, 10, 5, 20), 6); +``` + +Результат: + +```text +┌─mapPopulateSeries(map(1, 10, 5, 20), 6)─┐ +│ {1:10,2:0,3:0,4:0,5:20,6:0} │ +└─────────────────────────────────────────┘ +``` + ## mapContains {#mapcontains} Определяет, содержит ли контейнер `map` ключ `key`. @@ -320,3 +357,4 @@ SELECT mapValues(a) FROM test; └──────────────────┘ ``` +[Оригинальная статья](https://clickhouse.com/docs/ru/sql-reference/functions/tuple-map-functions/) From 7b02b6cc2adb89fd4f507a7d3d3780b89033494c Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 15 Oct 2021 23:08:35 +0300 Subject: [PATCH 168/919] Update settings.md Update distributed_push_down_limit in RU --- docs/ru/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 67b23808f61..592a669ccd6 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1709,7 +1709,7 @@ ClickHouse генерирует исключение Включает или отключает [LIMIT](#limit), применяемый к каждому шарду по отдельности. -Обычно, её включение не требуется, так как это будет сделано автоматически, если это возможно. +Обычно, включение данной настройки не требуется, так как это будет сделано автоматически, если это возможно, например, для простого запроса SELECT FROM LIMIT. Возможные значения: From 024d98b482d0c61ff000d559df203d28f16cf890 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 15 Oct 2021 23:12:55 +0300 Subject: [PATCH 169/919] Update settings.md Comment encryption settings. --- .../en/operations/server-configuration-parameters/settings.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 425853553ed..635c5313174 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -69,6 +69,8 @@ If no conditions met for a data part, ClickHouse uses the `lz4` compression. ``` + + ## custom_settings_prefixes {#custom_settings_prefixes} List of prefixes for [custom settings](../../operations/settings/index.md#custom_settings). The prefixes must be separated with commas. From 502183c4aa9dc37d4939684e7eaaf89f1524f4b3 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 15 Oct 2021 23:17:01 +0300 Subject: [PATCH 170/919] Update tuple-map-functions.md Fix link --- docs/ru/sql-reference/functions/tuple-map-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/tuple-map-functions.md b/docs/ru/sql-reference/functions/tuple-map-functions.md index 8c4ffed38b4..0d4cae7f2d5 100644 --- a/docs/ru/sql-reference/functions/tuple-map-functions.md +++ b/docs/ru/sql-reference/functions/tuple-map-functions.md @@ -170,7 +170,7 @@ SELECT mapSubtract(map(1,1), map(1,1)); Заполняет недостающие ключи в контейнере map (пара массивов ключей и значений), где ключи являются целыми числами. Кроме того, он поддерживает указание максимального ключа, который используется для расширения массива ключей. -Аргументами являются [map](../../sql-reference/data-types/map.m) или два [массива](../../sql-reference/data-types/array.md#data-type-array), где первый массив представляет ключи, а второй массив содержит значения для каждого ключа. +Аргументами являются [map](../../sql-reference/data-types/map.md) или два [массива](../../sql-reference/data-types/array.md#data-type-array), где первый массив представляет ключи, а второй массив содержит значения для каждого ключа. **Синтаксис** From c928ce8369ceea17653c0fc5d35fded80b3ddcee Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 16 Oct 2021 04:25:05 +0300 Subject: [PATCH 171/919] improve performance of aggregation in order of primary key --- .../FinishAggregatingInOrderAlgorithm.cpp | 45 ++++++++++++++++--- .../FinishAggregatingInOrderAlgorithm.h | 12 ++++- .../FinishAggregatingInOrderTransform.h | 6 ++- src/Processors/QueryPlan/AggregatingStep.cpp | 3 +- 4 files changed, 54 insertions(+), 12 deletions(-) diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp index f488ee61cc3..99e3adf893c 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp @@ -25,12 +25,14 @@ FinishAggregatingInOrderAlgorithm::FinishAggregatingInOrderAlgorithm( size_t num_inputs_, AggregatingTransformParamsPtr params_, SortDescription description_, - size_t max_block_size_) + size_t max_block_size_, + size_t merge_threads_) : header(header_) , num_inputs(num_inputs_) , params(params_) , description(std::move(description_)) , max_block_size(max_block_size_) + , pool(merge_threads_) { /// Replace column names in description to positions. for (auto & column_description : description) @@ -58,6 +60,12 @@ void FinishAggregatingInOrderAlgorithm::consume(Input & input, size_t source_num IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge() { + if (finished) + { + auto res = popResult(); + return Status(std::move(res), results.empty()); + } + if (!inputs_to_update.empty()) { Status status(inputs_to_update.back()); @@ -81,7 +89,13 @@ IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge() } if (!best_input) - return Status{aggregate(), true}; + { + aggregate(); + pool.wait(); + finished = true; + auto res = popResult(); + return Status(std::move(res), results.empty()); + } /// Chunk at best_input will be aggregated entirely. auto & best_state = states[*best_input]; @@ -112,17 +126,34 @@ IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge() /// Do not merge blocks, if there are too few rows. if (accumulated_rows >= max_block_size) - status.chunk = aggregate(); + aggregate(); + status.chunk = popResult(); return status; } -Chunk FinishAggregatingInOrderAlgorithm::aggregate() +Chunk FinishAggregatingInOrderAlgorithm::popResult() +{ + std::lock_guard lock(results_mutex); + + if (results.empty()) + return {}; + + auto res = std::move(results.back()); + results.pop_back(); + return res; +} + +void FinishAggregatingInOrderAlgorithm::aggregate() { - auto aggregated = params->aggregator.mergeBlocks(blocks, false); - blocks.clear(); accumulated_rows = 0; - return {aggregated.getColumns(), aggregated.rows()}; + pool.scheduleOrThrowOnError([this, blocks_list = std::move(blocks)]() mutable + { + auto aggregated = params->aggregator.mergeBlocks(blocks_list, false); + + std::lock_guard lock(results_mutex); + results.emplace_back(aggregated.getColumns(), aggregated.rows()); + }); } void FinishAggregatingInOrderAlgorithm::addToAggregation() diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h index 119aefb0ab0..29209f462e4 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -38,15 +39,17 @@ public: size_t num_inputs_, AggregatingTransformParamsPtr params_, SortDescription description_, - size_t max_block_size_); + size_t max_block_size_, + size_t max_threads_); void initialize(Inputs inputs) override; void consume(Input & input, size_t source_num) override; Status merge() override; private: - Chunk aggregate(); + void aggregate(); void addToAggregation(); + Chunk popResult(); struct State { @@ -69,12 +72,17 @@ private: AggregatingTransformParamsPtr params; SortDescription description; size_t max_block_size; + ThreadPool pool; + + std::mutex results_mutex; + std::vector results; Inputs current_inputs; std::vector states; std::vector inputs_to_update; BlocksList blocks; size_t accumulated_rows = 0; + bool finished = false; }; } diff --git a/src/Processors/Merges/FinishAggregatingInOrderTransform.h b/src/Processors/Merges/FinishAggregatingInOrderTransform.h index 6d5e334311f..3a388a36727 100644 --- a/src/Processors/Merges/FinishAggregatingInOrderTransform.h +++ b/src/Processors/Merges/FinishAggregatingInOrderTransform.h @@ -17,14 +17,16 @@ public: size_t num_inputs, AggregatingTransformParamsPtr params, SortDescription description, - size_t max_block_size) + size_t max_block_size, + size_t merge_threads) : IMergingTransform( num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false, header, num_inputs, params, std::move(description), - max_block_size) + max_block_size, + merge_threads) { } diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 023f9016cc4..4942757cbd8 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -101,7 +101,8 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B pipeline.getNumStreams(), transform_params, group_by_sort_description, - max_block_size); + max_block_size, + merge_threads); pipeline.addTransform(std::move(transform)); aggregating_sorted = collector.detachProcessors(1); From 5db75f9e77c6604c09ee0b386572693c38615ef6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 16 Oct 2021 14:28:57 +0300 Subject: [PATCH 172/919] Better exception --- programs/client/Client.cpp | 9 +++-- programs/local/LocalServer.cpp | 67 +++++++++++++++++++--------------- src/Client/ClientBase.cpp | 2 - src/Client/ClientBase.h | 1 + 4 files changed, 44 insertions(+), 35 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 11459ff4a4a..30a1b28f669 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -95,7 +95,6 @@ void Client::processError(const String & query) const { if (server_exception) { - bool print_stack_trace = config().getBool("stacktrace", false); fmt::print(stderr, "Received exception from server (version {}):\n{}\n", server_version, getExceptionMessage(*server_exception, print_stack_trace, true)); @@ -232,9 +231,12 @@ bool Client::executeMultiQuery(const String & all_queries_text) } catch (...) { + if (!is_interactive) + throw; + // Surprisingly, this is a client error. A server error would // have been reported w/o throwing (see onReceiveSeverException()). - client_exception = std::make_unique(getCurrentExceptionMessage(true), getCurrentExceptionCode()); + client_exception = std::make_unique(getCurrentExceptionMessage(print_stack_trace), getCurrentExceptionCode()); have_error = true; } // Check whether the error (or its absence) matches the test hints @@ -822,7 +824,7 @@ bool Client::processWithFuzzing(const String & full_query) // uniformity. // Surprisingly, this is a client exception, because we get the // server exception w/o throwing (see onReceiveException()). - client_exception = std::make_unique(getCurrentExceptionMessage(true), getCurrentExceptionCode()); + client_exception = std::make_unique(getCurrentExceptionMessage(print_stack_trace), getCurrentExceptionCode()); have_error = true; } @@ -1196,6 +1198,7 @@ void Client::processConfig() if (!query_id.empty()) global_context->setCurrentQueryId(query_id); } + print_stack_trace = config().getBool("stacktrace", false); if (config().has("multiquery")) is_multiquery = true; diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 30082caaac1..35397968879 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -73,7 +73,6 @@ void LocalServer::processError(const String &) const String message; if (server_exception) { - bool print_stack_trace = config().getBool("stacktrace", false); message = getExceptionMessage(*server_exception, print_stack_trace, true); } else if (client_exception) @@ -143,9 +142,12 @@ bool LocalServer::executeMultiQuery(const String & all_queries_text) } catch (...) { + if (!is_interactive) + throw; + // Surprisingly, this is a client error. A server error would // have been reported w/o throwing (see onReceiveSeverException()). - client_exception = std::make_unique(getCurrentExceptionMessage(true), getCurrentExceptionCode()); + client_exception = std::make_unique(getCurrentExceptionMessage(print_stack_trace), getCurrentExceptionCode()); have_error = true; } @@ -299,23 +301,30 @@ void LocalServer::tryInitPath() void LocalServer::cleanup() { - connection.reset(); - - if (global_context) + try { - global_context->shutdown(); - global_context.reset(); + connection.reset(); + + if (global_context) + { + global_context->shutdown(); + global_context.reset(); + } + + status.reset(); + + // Delete the temporary directory if needed. + if (temporary_directory_to_delete) + { + const auto dir = *temporary_directory_to_delete; + temporary_directory_to_delete.reset(); + LOG_DEBUG(&logger(), "Removing temporary directory: {}", dir.string()); + remove_all(dir); + } } - - status.reset(); - - // Delete the temporary directory if needed. - if (temporary_directory_to_delete) + catch (...) { - const auto dir = *temporary_directory_to_delete; - temporary_directory_to_delete.reset(); - LOG_DEBUG(&logger(), "Removing temporary directory: {}", dir.string()); - remove_all(dir); + tryLogCurrentException(__PRETTY_FUNCTION__); } } @@ -458,23 +467,20 @@ try cleanup(); return Application::EXIT_OK; } +catch (const DB::Exception & e) +{ + cleanup(); + + bool print_stack_trace = config().getBool("stacktrace", false); + std::cerr << getExceptionMessage(e, print_stack_trace, true) << std::endl << std::endl; + return e.code() ? e.code() : -1; +} catch (...) { - try - { - cleanup(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + cleanup(); - if (!ignore_error) - std::cerr << getCurrentExceptionMessage(config().hasOption("stacktrace")) << '\n'; - - auto code = getCurrentExceptionCode(); - /// If exception code isn't zero, we should return non-zero return code anyway. - return code ? code : -1; + std::cerr << getCurrentExceptionMessage(false) << std::endl; + return getCurrentExceptionCode(); } @@ -497,6 +503,7 @@ void LocalServer::processConfig() ignore_error = config().getBool("ignore-error", false); is_multiquery = true; } + print_stack_trace = config().getBool("stacktrace", false); shared_context = Context::createShared(); global_context = Context::createGlobal(shared_context.get()); diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index e030d8994d5..4daaa5a1f4d 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1331,9 +1331,7 @@ void ClientBase::runInteractive() catch (const Exception & e) { /// We don't need to handle the test hints in the interactive mode. - bool print_stack_trace = config().getBool("stacktrace", false); std::cerr << "Exception on client:" << std::endl << getExceptionMessage(e, print_stack_trace, true) << std::endl << std::endl; - client_exception = std::make_unique(e); } diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index d9034534797..5b1162fd050 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -200,6 +200,7 @@ protected: bool written_first_block = false; size_t processed_rows = 0; /// How many rows have been read or written. + bool print_stack_trace = false; /// The last exception that was received from the server. Is used for the /// return code in batch mode. std::unique_ptr server_exception; From e127fa69762805a7b736d3c72fa5b7f3c5376cbc Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 16 Oct 2021 14:45:22 +0300 Subject: [PATCH 173/919] Add test --- .../02050_clickhouse_client_local_exception.reference | 2 ++ .../02050_clickhouse_client_local_exception.sh | 9 +++++++++ 2 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/02050_clickhouse_client_local_exception.reference create mode 100755 tests/queries/0_stateless/02050_clickhouse_client_local_exception.sh diff --git a/tests/queries/0_stateless/02050_clickhouse_client_local_exception.reference b/tests/queries/0_stateless/02050_clickhouse_client_local_exception.reference new file mode 100644 index 00000000000..2c94e483710 --- /dev/null +++ b/tests/queries/0_stateless/02050_clickhouse_client_local_exception.reference @@ -0,0 +1,2 @@ +OK +OK diff --git a/tests/queries/0_stateless/02050_clickhouse_client_local_exception.sh b/tests/queries/0_stateless/02050_clickhouse_client_local_exception.sh new file mode 100755 index 00000000000..771070bb4bc --- /dev/null +++ b/tests/queries/0_stateless/02050_clickhouse_client_local_exception.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="SELECT 1 INTO OUTFILE 'test.native.zst' FORMAT Native" 2>&1 | grep -q "Code: 76. DB::ErrnoException: Cannot open file test.native.zst, errno: 17, strerror: File exists." && echo 'OK' || echo 'FAIL' ||: +$CLICKHOUSE_LOCAL --query="SELECT 1 INTO OUTFILE 'test.native.zst' FORMAT Native" 2>&1 | grep -q "Code: 76. DB::ErrnoException: Cannot open file test.native.zst, errno: 17, strerror: File exists." && echo 'OK' || echo 'FAIL' ||: + From 1a947d30bf4f074292f8ca15a066fc599d1b5d90 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 16 Oct 2021 16:41:54 +0300 Subject: [PATCH 174/919] add perf test --- .../performance/read_in_order_many_parts.xml | 26 ++++++++++++------- 1 file changed, 17 insertions(+), 9 deletions(-) diff --git a/tests/performance/read_in_order_many_parts.xml b/tests/performance/read_in_order_many_parts.xml index ca713935426..cc69bfeb163 100644 --- a/tests/performance/read_in_order_many_parts.xml +++ b/tests/performance/read_in_order_many_parts.xml @@ -15,17 +15,25 @@ 1000 + + uniqs + + 100 + 10000 + 1000000 + + - CREATE TABLE mt_{parts}_parts(id UInt32, val1 UInt32, val2 UInt32) ENGINE = MergeTree ORDER BY val1 PARTITION BY id % {parts} SETTINGS parts_to_throw_insert=10000, parts_to_delay_insert=10000 - SYSTEM STOP MERGES mt_{parts}_parts - INSERT INTO mt_{parts}_parts SELECT number, rand() % 10000, rand() FROM numbers_mt(100000000) SETTINGS max_block_size=10000000 - SYSTEM START MERGES mt_{parts}_parts - OPTIMIZE TABLE mt_{parts}_parts FINAL + CREATE TABLE mt_{parts}_parts_{uniqs}_uniqs(id UInt32, val1 UInt32, val2 UInt32) ENGINE = MergeTree ORDER BY val1 PARTITION BY id % {parts} SETTINGS parts_to_throw_insert=10000, parts_to_delay_insert=10000 + SYSTEM STOP MERGES mt_{parts}_parts_{uniqs}_uniqs + INSERT INTO mt_{parts}_parts_{uniqs}_uniqs SELECT number, rand() % {uniqs}, rand() FROM numbers_mt(100000000) SETTINGS max_block_size=10000000 + SYSTEM START MERGES mt_{parts}_parts_{uniqs}_uniqs + OPTIMIZE TABLE mt_{parts}_parts_{uniqs}_uniqs FINAL - SELECT val2 FROM mt_{parts}_parts ORDER BY val1 LIMIT 1000000 FORMAT Null - SELECT val2 FROM mt_{parts}_parts ORDER BY val1 LIMIT 10000000 FORMAT Null - SELECT sum(val2) FROM mt_{parts}_parts GROUP BY val1 FORMAT Null + SELECT val2 FROM mt_{parts}_parts_10000_uniqs ORDER BY val1 LIMIT 1000000 FORMAT Null + SELECT val2 FROM mt_{parts}_parts_10000_uniqs ORDER BY val1 LIMIT 10000000 FORMAT Null + SELECT sum(val2) FROM mt_{parts}_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null - DROP TABLE IF EXISTS mt_{parts}_parts + DROP TABLE IF EXISTS mt_{parts}_parts_{uniqs}_uniqs From 91e9ab7760226e37fccba32fc217eafd26109c89 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 16 Oct 2021 19:30:35 +0000 Subject: [PATCH 175/919] Fix tests --- programs/client/Client.cpp | 3 --- programs/local/LocalServer.cpp | 4 ++-- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 30a1b28f669..52457a7f182 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -231,9 +231,6 @@ bool Client::executeMultiQuery(const String & all_queries_text) } catch (...) { - if (!is_interactive) - throw; - // Surprisingly, this is a client error. A server error would // have been reported w/o throwing (see onReceiveSeverException()). client_exception = std::make_unique(getCurrentExceptionMessage(print_stack_trace), getCurrentExceptionCode()); diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 35397968879..03594ab7d90 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -142,7 +142,7 @@ bool LocalServer::executeMultiQuery(const String & all_queries_text) } catch (...) { - if (!is_interactive) + if (!is_interactive && !ignore_error) throw; // Surprisingly, this is a client error. A server error would @@ -472,7 +472,7 @@ catch (const DB::Exception & e) cleanup(); bool print_stack_trace = config().getBool("stacktrace", false); - std::cerr << getExceptionMessage(e, print_stack_trace, true) << std::endl << std::endl; + std::cerr << getExceptionMessage(e, print_stack_trace, true) << std::endl; return e.code() ? e.code() : -1; } catch (...) From 09e3aec96e435b176a27e57402c6836dfd233923 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 16 Oct 2021 19:48:51 +0000 Subject: [PATCH 176/919] Fix local break on timeout --- src/Client/ClientBase.cpp | 3 ++- src/Client/Connection.h | 2 ++ src/Client/IServerConnection.h | 8 ++++++++ src/Client/LocalConnection.h | 2 ++ 4 files changed, 14 insertions(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 4daaa5a1f4d..ae88508e2d5 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -517,6 +517,7 @@ void ClientBase::receiveResult(ASTPtr parsed_query) const size_t poll_interval = std::max(min_poll_interval, std::min(receive_timeout.totalMicroseconds(), default_poll_interval)); + bool break_on_timeout = connection->getConnectionType() != IServerConnection::Type::LOCAL; while (true) { Stopwatch receive_watch(CLOCK_MONOTONIC_COARSE); @@ -547,7 +548,7 @@ void ClientBase::receiveResult(ASTPtr parsed_query) else { double elapsed = receive_watch.elapsedSeconds(); - if (elapsed > receive_timeout.totalSeconds()) + if (break_on_timeout && elapsed > receive_timeout.totalSeconds()) { std::cout << "Timeout exceeded while receiving data from server." << " Waited for " << static_cast(elapsed) << " seconds," diff --git a/src/Client/Connection.h b/src/Client/Connection.h index b6054941aeb..848b848a104 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -60,6 +60,8 @@ public: ~Connection() override; + IServerConnection::Type getConnectionType() const override { return IServerConnection::Type::SERVER; } + static ServerConnectionPtr createConnection(const ConnectionParameters & parameters, ContextPtr context); /// Set throttler of network traffic. One throttler could be used for multiple connections to limit total traffic. diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index 42886c72182..5a853126982 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -56,6 +56,14 @@ class IServerConnection : boost::noncopyable public: virtual ~IServerConnection() = default; + enum class Type + { + SERVER, + LOCAL + }; + + virtual Type getConnectionType() const = 0; + virtual void setDefaultDatabase(const String & database) = 0; virtual void getServerVersion( diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index dcea3ed0fc3..0b2efbb9767 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -56,6 +56,8 @@ public: ~LocalConnection() override; + IServerConnection::Type getConnectionType() const override { return IServerConnection::Type::LOCAL; } + static ServerConnectionPtr createConnection(const ConnectionParameters & connection_parameters, ContextPtr current_context, bool send_progress = false); void setDefaultDatabase(const String & database) override; From 9378b93e4154ac417f1f2f35fefec857ebb76891 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 17 Oct 2021 14:49:27 +0000 Subject: [PATCH 177/919] fix --- src/Common/ErrorCodes.cpp | 2 +- src/Common/filesystemHelpers.cpp | 2 +- src/Common/filesystemHelpers.h | 3 +- src/Dictionaries/LibraryDictionarySource.cpp | 2 +- src/Storages/FileLog/DirectoryWatcherBase.cpp | 32 +++- src/Storages/FileLog/DirectoryWatcherBase.h | 6 +- .../FileLog/FileLogDirectoryWatcher.cpp | 30 ++-- .../FileLog/FileLogDirectoryWatcher.h | 16 +- src/Storages/FileLog/FileLogSettings.h | 7 +- src/Storages/FileLog/FileLogSource.cpp | 9 +- src/Storages/FileLog/FileLogSource.h | 2 +- .../FileLog/ReadBufferFromFileLog.cpp | 11 +- src/Storages/FileLog/ReadBufferFromFileLog.h | 3 +- src/Storages/FileLog/StorageFileLog.cpp | 152 ++++++++++-------- src/Storages/FileLog/StorageFileLog.h | 22 ++- src/Storages/IStorage.h | 3 +- 16 files changed, 189 insertions(+), 113 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 05d4a5cbc5a..e62161eca6d 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -589,7 +589,7 @@ M(619, POSTGRESQL_REPLICATION_INTERNAL_ERROR) \ M(620, QUERY_NOT_ALLOWED) \ M(621, CANNOT_NORMALIZE_STRING) \ - M(622, NOT_REGULAR_FILE) \ + M(622, BAD_FILE_TYPE) \ M(623, IO_SETUP_ERROR) \ \ M(999, KEEPER_EXCEPTION) \ diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 89214ad496e..4c9cd37fc27 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -140,7 +140,7 @@ bool pathStartsWith(const String & path, const String & prefix_path) return pathStartsWith(filesystem_path, filesystem_prefix_path); } -bool symlinkStartsWith(const String & path, const String & prefix_path) +bool isPathOrSymlinkStartsWith(const String & path, const String & prefix_path) { auto filesystem_path = std::filesystem::path(path); auto filesystem_prefix_path = std::filesystem::path(prefix_path); diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index de5802cde6d..104d4cd8987 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -36,8 +36,7 @@ bool pathStartsWith(const std::filesystem::path & path, const std::filesystem::p bool pathStartsWith(const String & path, const String & prefix_path); /// Returns true if symlink starts with prefix path -bool symlinkStartsWith(const String & path, const String & prefix_path); - +bool isPathOrSymlinkStartsWith(const String & path, const String & prefix_path); } namespace FS diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index 551bb1ee2dd..57b1ac852b9 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -44,7 +44,7 @@ LibraryDictionarySource::LibraryDictionarySource( auto dictionaries_lib_path = context->getDictionariesLibPath(); bool path_checked = false; if (fs::is_symlink(path)) - path_checked = symlinkStartsWith(path, dictionaries_lib_path); + path_checked = isPathOrSymlinkStartsWith(path, dictionaries_lib_path); else path_checked = pathStartsWith(path, dictionaries_lib_path); diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index 9ea2e7e1824..cad2072a8e6 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -12,7 +12,7 @@ namespace DB namespace ErrorCodes { extern const int FILE_DOESNT_EXIST; - extern const int DIRECTORY_DOESNT_EXIST; + extern const int BAD_FILE_TYPE; extern const int IO_SETUP_ERROR; } @@ -20,19 +20,19 @@ static constexpr int buffer_size = 4096; DirectoryWatcherBase::DirectoryWatcherBase( FileLogDirectoryWatcher & owner_, const std::string & path_, ContextPtr context_, int event_mask_) - : WithContext(context_->getGlobalContext()), owner(owner_), path(path_), event_mask(event_mask_) + : WithContext(context_), owner(owner_), path(path_), event_mask(event_mask_) { if (!std::filesystem::exists(path)) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "The path {} does not exist", path); + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Path {} does not exist", path); if (!std::filesystem::is_directory(path)) - throw Exception(ErrorCodes::DIRECTORY_DOESNT_EXIST, "The path {} does not a directory", path); + throw Exception(ErrorCodes::BAD_FILE_TYPE, "Path {} is not a directory", path); fd = inotify_init(); if (fd == -1) throw Exception("Cannot initialize inotify", ErrorCodes::IO_SETUP_ERROR); - watch_task = getContext()->getMessageBrokerSchedulePool().createTask("directory_watch", [this] { watchFunc(); }); + watch_task = getContext()->getSchedulePool().createTask("directory_watch", [this] { watchFunc(); }); start(); } @@ -63,6 +63,7 @@ void DirectoryWatcherBase::watchFunc() pfd.events = POLLIN; while (!stopped) { + const auto & settings = owner.storage.getFileLogSettings(); if (poll(&pfd, 1, 500) > 0 && pfd.revents & POLLIN) { int n = read(fd, buffer.data(), buffer.size()); @@ -106,8 +107,25 @@ void DirectoryWatcherBase::watchFunc() n -= sizeof(inotify_event) + p_event->len; } } + + /// Wake up reader thread + auto & mutex = owner.storage.getMutex(); + auto & cv = owner.storage.getConditionVariable(); + std::unique_lock lock(mutex); + owner.storage.setNewEvents(); + lock.unlock(); + cv.notify_one(); + } + else + { + if (milliseconds_to_wait < static_cast(settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) + milliseconds_to_wait *= settings->poll_directory_watch_events_backoff_factor.value; + break; } } + + if (!stopped) + watch_task->scheduleAfter(milliseconds_to_wait); } @@ -115,6 +133,9 @@ DirectoryWatcherBase::~DirectoryWatcherBase() { stop(); close(fd); + + if (watch_task) + watch_task->deactivate(); } void DirectoryWatcherBase::start() @@ -123,7 +144,6 @@ void DirectoryWatcherBase::start() watch_task->activateAndSchedule(); } - void DirectoryWatcherBase::stop() { stopped = true; diff --git a/src/Storages/FileLog/DirectoryWatcherBase.h b/src/Storages/FileLog/DirectoryWatcherBase.h index db89746358b..88f864ac17d 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.h +++ b/src/Storages/FileLog/DirectoryWatcherBase.h @@ -12,8 +12,10 @@ class FileLogDirectoryWatcher; class DirectoryWatcherBase : WithContext { + /// Most of code in this class is copy from the Poco project: + /// https://github.com/ClickHouse-Extras/poco/blob/clickhouse/Foundation/src/DirectoryWatcher.cpp /// This class is used to get notifications about changes - /// to the filesystem, more specifically, to a specific + /// to the filesystem, more precisely, to a specific /// directory. Changes to a directory are reported via /// events. /// @@ -95,6 +97,8 @@ private: std::atomic stopped{false}; + uint64_t milliseconds_to_wait; + const std::string path; int event_mask; int fd; diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index 4dadbed3f09..c4a153dbfe7 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -2,8 +2,9 @@ namespace DB { -FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_, ContextPtr context_) +FileLogDirectoryWatcher::FileLogDirectoryWatcher(const std::string & path_, StorageFileLog & storage_, ContextPtr context_) : path(path_) + , storage(storage_) , log(&Poco::Logger::get("FileLogDirectoryWatcher(" + path + ")")) , dw(std::make_unique(*this, path, context_)) { @@ -39,11 +40,11 @@ void FileLogDirectoryWatcher::onItemAdded(DirectoryWatcherBase::DirectoryEvent e if (auto it = events.find(event_path); it != events.end()) { - it->second.emplace_back(info); + it->second.file_events.emplace_back(info); } else { - events.emplace(event_path, std::vector{info}); + events.emplace(event_path, FileEvents{.file_events = std::vector{info}}); } } @@ -57,11 +58,11 @@ void FileLogDirectoryWatcher::onItemRemoved(DirectoryWatcherBase::DirectoryEvent if (auto it = events.find(event_path); it != events.end()) { - it->second.emplace_back(info); + it->second.file_events.emplace_back(info); } else { - events.emplace(event_path, std::vector{info}); + events.emplace(event_path, FileEvents{.file_events = std::vector{info}}); } } @@ -80,14 +81,17 @@ void FileLogDirectoryWatcher::onItemModified(DirectoryWatcherBase::DirectoryEven /// Already have MODIFY event for this file if (auto it = events.find(event_path); it != events.end()) { - if (it->second.back().type == ev.event) + if (it->second.received_modification_event) return; else - it->second.emplace_back(info); + { + it->second.received_modification_event = true; + it->second.file_events.emplace_back(info); + } } else { - events.emplace(event_path, std::vector{info}); + events.emplace(event_path, FileEvents{.received_modification_event = true, .file_events = std::vector{info}}); } } @@ -100,11 +104,11 @@ void FileLogDirectoryWatcher::onItemMovedFrom(DirectoryWatcherBase::DirectoryEve if (auto it = events.find(event_path); it != events.end()) { - it->second.emplace_back(info); + it->second.file_events.emplace_back(info); } else { - events.emplace(event_path, std::vector{info}); + events.emplace(event_path, FileEvents{.file_events = std::vector{info}}); } } @@ -117,18 +121,18 @@ void FileLogDirectoryWatcher::onItemMovedTo(DirectoryWatcherBase::DirectoryEvent if (auto it = events.find(event_path); it != events.end()) { - it->second.emplace_back(info); + it->second.file_events.emplace_back(info); } else { - events.emplace(event_path, std::vector{info}); + events.emplace(event_path, FileEvents{.file_events = std::vector{info}}); } } void FileLogDirectoryWatcher::onError(Exception e) { std::lock_guard lock(mutex); - LOG_ERROR(log, "Error happened during watching directory {}: {}", path, error.error_msg); + LOG_ERROR(log, "Error happened during watching directory: {}", error.error_msg); error.has_error = true; error.error_msg = e.message(); } diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index 4259cc8722e..8e0ecf5358c 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include @@ -19,7 +20,13 @@ public: std::string callback; }; - using Events = std::unordered_map>; + struct FileEvents + { + bool received_modification_event = false; + std::vector file_events; + }; + + using Events = std::unordered_map; struct Error { @@ -27,7 +34,7 @@ public: std::string error_msg = {}; }; - explicit FileLogDirectoryWatcher(const std::string & path_, ContextPtr context_); + FileLogDirectoryWatcher(const std::string & path_, StorageFileLog & storage_, ContextPtr context_); ~FileLogDirectoryWatcher() = default; Events getEventsAndReset(); @@ -36,6 +43,8 @@ public: const std::string & getPath() const; +private: + friend class DirectoryWatcherBase; /// Here must pass by value, otherwise will lead to stack-use-of-scope void onItemAdded(DirectoryWatcherBase::DirectoryEvent ev); void onItemRemoved(DirectoryWatcherBase::DirectoryEvent ev); @@ -44,9 +53,10 @@ public: void onItemMovedTo(DirectoryWatcherBase::DirectoryEvent ev); void onError(Exception); -private: const std::string path; + StorageFileLog & storage; + /// Note, in order to avoid data race found by fuzzer, put events before dw, /// such that when this class destruction, dw will be destructed before events. /// The data race is because dw create a separate thread to monitor file events diff --git a/src/Storages/FileLog/FileLogSettings.h b/src/Storages/FileLog/FileLogSettings.h index 5ed55d94426..d14120d0ba0 100644 --- a/src/Storages/FileLog/FileLogSettings.h +++ b/src/Storages/FileLog/FileLogSettings.h @@ -14,7 +14,10 @@ class ASTStorage; M(Milliseconds, poll_timeout_ms, 0, "Timeout for single poll from StorageFileLog.", 0) \ M(UInt64, poll_max_batch_size, 0, "Maximum amount of messages to be polled in a single StorageFileLog poll.", 0) \ M(UInt64, max_block_size, 0, "Number of row collected by poll(s) for flushing data from StorageFileLog.", 0) \ - M(UInt64, max_threads, 8, "Number of max threads to parse files, default is 8", 0) + M(UInt64, max_threads, 8, "Number of max threads to parse files, default is 8", 0) \ + M(Milliseconds, poll_directory_watch_events_backoff_init, 500, "The initial sleep value for watch directory thread.", 0) \ + M(Milliseconds, poll_directory_watch_events_backoff_max, 32000, "The max sleep value for watch directory thread.", 0) \ + M(UInt64, poll_directory_watch_events_backoff_factor, 2, "The speed of backoff, exponential by default", 0) #define LIST_OF_FILELOG_SETTINGS(M) \ FILELOG_RELATED_SETTINGS(M) \ @@ -23,7 +26,7 @@ class ASTStorage; DECLARE_SETTINGS_TRAITS(FileLogSettingsTraits, LIST_OF_FILELOG_SETTINGS) -/** Settings for the Kafka engine. +/** Settings for the FileLog engine. * Could be loaded from a CREATE TABLE query (SETTINGS clause). */ struct FileLogSettings : public BaseSettings diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index 7beee42c99c..bb0fcc90016 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -47,16 +47,19 @@ FileLogSource::FileLogSource( void FileLogSource::onFinish() { storage.closeFilesAndStoreMeta(start, end); + finished = true; } Chunk FileLogSource::generate() { + /// Store metas of last written chunk into disk + storage.storeMetas(start, end); + if (!buffer || buffer->noRecords()) { /// There is no onFinish for ISource, we call it /// when no records return to close files onFinish(); - finished = true; return {}; } @@ -104,7 +107,6 @@ Chunk FileLogSource::generate() if (total_rows == 0) { onFinish(); - finished = true; return {}; } @@ -122,9 +124,6 @@ Chunk FileLogSource::generate() auto converting_actions = std::make_shared(std::move(converting_dag)); converting_actions->execute(result_block); - /// After generate each block, store metas into disk - storage.storeMetas(start, end); - return Chunk(result_block.getColumns(), result_block.rows()); } diff --git a/src/Storages/FileLog/FileLogSource.h b/src/Storages/FileLog/FileLogSource.h index cdf60cd4fc1..b8b579beaed 100644 --- a/src/Storages/FileLog/FileLogSource.h +++ b/src/Storages/FileLog/FileLogSource.h @@ -58,7 +58,7 @@ private: Block virtual_header; /// The start pos and end pos of files responsible by this stream, - /// does not includind end + /// does not include end size_t start; size_t end; }; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index f935decd7f8..384de64cd3f 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -47,7 +47,7 @@ bool ReadBufferFromFileLog::poll() if (new_records.empty()) { buffer_status = BufferStatus::NO_RECORD_RETURNED; - LOG_TRACE(log, "No records returned"); + LOG_TRACE(log, "No new records to read"); return false; } else @@ -107,16 +107,19 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ auto & file_meta = StorageFileLog::findInMap(file_infos.meta_by_inode, file_ctx.inode); if (!file_ctx.reader) - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Ifstream for file {} does not initialized", file_meta.file_name); + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Ifstream for file {} is not initialized", file_meta.file_name); auto & reader = file_ctx.reader.value(); StorageFileLog::assertStreamGood(reader); Record record; - while (read_records_size < need_records_size && static_cast(reader.tellg()) < file_meta.last_open_end) + while (read_records_size < need_records_size) { /// Need to get offset before reading record from stream - record.offset = reader.tellg(); + auto offset = reader.tellg(); + if (static_cast(offset) < file_meta.last_open_end) + break; + record.offset = offset; StorageFileLog::assertStreamGood(reader); record.file_name = file_name; diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.h b/src/Storages/FileLog/ReadBufferFromFileLog.h index b4102870c3a..117a858de3b 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.h +++ b/src/Storages/FileLog/ReadBufferFromFileLog.h @@ -71,6 +71,7 @@ private: { RecordData data; std::string file_name; + /// Offset is the start of a row, which is needed for virtual columns. UInt64 offset; }; using Records = std::vector; @@ -78,7 +79,7 @@ private: Records records; Records::const_iterator current; - String current_file = {}; + String current_file; UInt64 current_offset = 0; using TaskThread = BackgroundSchedulePool::TaskHolder; diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 516be811666..66add8128ef 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -37,11 +37,12 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; extern const int CANNOT_STAT; - extern const int NOT_REGULAR_FILE; + extern const int BAD_FILE_TYPE; extern const int CANNOT_READ_ALL_DATA; extern const int LOGICAL_ERROR; extern const int TABLE_METADATA_ALREADY_EXISTS; extern const int CANNOT_SELECT; + extern const int QUERY_NOT_ALLOWED; } namespace @@ -80,44 +81,31 @@ StorageFileLog::StorageFileLog( loadMetaFiles(attach); loadFiles(); -#ifndef NDEBUG assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); -#endif if (path_is_directory) - directory_watch = std::make_unique(root_data_path, context_); + directory_watch = std::make_unique(root_data_path, *this, getContext()); - auto thread = getContext()->getMessageBrokerSchedulePool().createTask(log->name(), [this] { threadFunc(); }); + auto thread = getContext()->getSchedulePool().createTask(log->name(), [this] { threadFunc(); }); task = std::make_shared(std::move(thread)); } catch (...) { + if (!attach) + throw; tryLogCurrentException(__PRETTY_FUNCTION__); } } void StorageFileLog::loadMetaFiles(bool attach) { - /// We just use default storage policy - auto storage_policy = getContext()->getStoragePolicy("default"); - auto data_volume = storage_policy->getVolume(0); - root_meta_path = std::filesystem::path(data_volume->getDisk()->getPath()) / getStorageID().getTableName(); + const auto & storage_id = getStorageID(); + root_meta_path = std::filesystem::path(getContext()->getPath()) / "metadata" / "filelog_storage_metadata" / storage_id.getDatabaseName() + / storage_id.getTableName(); - /// Create table, just create meta data directory - if (!attach) - { - if (std::filesystem::exists(root_meta_path)) - { - throw Exception( - ErrorCodes::TABLE_METADATA_ALREADY_EXISTS, - "Metadata files already exist by path: {}, remove them manually if it is intended", - root_meta_path); - } - std::filesystem::create_directories(root_meta_path); - } /// Attach table - else + if (attach) { /// Meta file may lost, log and create directory if (!std::filesystem::exists(root_meta_path)) @@ -128,14 +116,26 @@ void StorageFileLog::loadMetaFiles(bool attach) /// Load all meta info to file_infos; deserialize(); } + /// Create table, just create meta data directory + else + { + if (std::filesystem::exists(root_meta_path)) + { + throw Exception( + ErrorCodes::TABLE_METADATA_ALREADY_EXISTS, + "Metadata files already exist by path: {}, remove them manually if it is intended", + root_meta_path); + } + std::filesystem::create_directories(root_meta_path); + } } void StorageFileLog::loadFiles() { - if (!symlinkStartsWith(path, getContext()->getUserFilesPath())) + if (!isPathOrSymlinkStartsWith(path, getContext()->getUserFilesPath())) { throw Exception( - ErrorCodes::BAD_ARGUMENTS, "The absolute data path should start with user_files_path {}", getContext()->getUserFilesPath()); + ErrorCodes::BAD_ARGUMENTS, "The absolute data path should be inside `user_files_path`({})", getContext()->getUserFilesPath()); } auto absolute_path = std::filesystem::absolute(path); @@ -181,10 +181,7 @@ void StorageFileLog::loadFiles() if (it->second.file_name != file) { it->second.file_name = file; - if (std::filesystem::exists(getFullMetaPath(it->second.file_name))) - { - std::filesystem::rename(getFullMetaPath(it->second.file_name), getFullMetaPath(file)); - } + std::filesystem::rename(getFullMetaPath(it->second.file_name), getFullMetaPath(file)); } } /// New file @@ -264,12 +261,14 @@ void StorageFileLog::serialize(UInt64 inode, const FileMeta & file_meta) const void StorageFileLog::deserialize() { + /// In case of single file (not a watched directory), + /// iterated directoy always has one file inside. for (const auto & dir_entry : std::filesystem::directory_iterator{root_meta_path}) { if (!dir_entry.is_regular_file()) { throw Exception( - ErrorCodes::NOT_REGULAR_FILE, + ErrorCodes::BAD_FILE_TYPE, "The file {} under {} is not a regular file when deserializing meta files", dir_entry.path().c_str(), root_meta_path); @@ -315,15 +314,14 @@ Pipe StorageFileLog::read( size_t /* max_block_size */, unsigned /* num_streams */) { - auto table_id = getStorageID(); - size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); /// If there are MVs depended on this table, we just forbid reading - if (dependencies_count) + if (has_dependent_mv) { throw Exception( - ErrorCodes::CANNOT_READ_ALL_DATA, - "Can not read from table {}, because it has been depended by other tables", - table_id.getTableName()); + ErrorCodes::QUERY_NOT_ALLOWED, + "Can not make `SELECT` query from table {}, because it has attached dependencies. Remove dependant materialized views if " + "needed", + getStorageID().getTableName()); } if (running_streams.load(std::memory_order_relaxed)) @@ -331,16 +329,12 @@ Pipe StorageFileLog::read( throw Exception("Another select query is running on this table, need to wait it finish.", ErrorCodes::CANNOT_SELECT); } - /// We need this lock, in case read and streamToViews execute at the same time. - /// In case of MV attached during reading - std::lock_guard lock(status_mutex); - updateFileInfos(); /// No files to parse if (file_infos.file_names.empty()) { - LOG_INFO(log, "There is a idle table named {}, no files need to parse.", getName()); + LOG_WARNING(log, "There is a idle table named {}, no files need to parse.", getName()); return Pipe{}; } @@ -371,12 +365,12 @@ Pipe StorageFileLog::read( void StorageFileLog::increaseStreams() { - running_streams.fetch_add(1, std::memory_order_relaxed); + running_streams += 1; } void StorageFileLog::reduceStreams() { - running_streams.fetch_sub(1, std::memory_order_relaxed); + running_streams -= 1; } void StorageFileLog::drop() @@ -457,9 +451,14 @@ void StorageFileLog::openFilesAndSetPos() auto & meta = findInMap(file_infos.meta_by_inode, file_ctx.inode); if (meta.last_writen_position > static_cast(file_end)) { - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "File {} has been broken", file); + throw Exception( + ErrorCodes::CANNOT_READ_ALL_DATA, + "Last saved offsset for File {} is bigger than file size ({} > {})", + file, + meta.last_writen_position, + file_end); } - /// update file end at the monment, used in ReadBuffer and serialize + /// update file end at the moment, used in ReadBuffer and serialize meta.last_open_end = file_end; reader.seekg(meta.last_writen_position); @@ -471,11 +470,9 @@ void StorageFileLog::openFilesAndSetPos() void StorageFileLog::closeFilesAndStoreMeta(size_t start, size_t end) { -#ifndef NDEBUG assert(start >= 0); assert(start < end); assert(end <= file_infos.file_names.size()); -#endif for (size_t i = start; i < end; ++i) { @@ -494,11 +491,9 @@ void StorageFileLog::closeFilesAndStoreMeta(size_t start, size_t end) void StorageFileLog::storeMetas(size_t start, size_t end) { -#ifndef NDEBUG assert(start >= 0); assert(start < end); assert(end <= file_infos.file_names.size()); -#endif for (size_t i = start; i < end; ++i) { @@ -577,9 +572,11 @@ void StorageFileLog::threadFunc() { try { + updateFileInfos(); auto table_id = getStorageID(); // Check if at least one direct dependency is attached size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); + if (dependencies_count) { auto start_time = std::chrono::steady_clock::now(); @@ -595,13 +592,15 @@ void StorageFileLog::threadFunc() if (streamToViews()) { LOG_TRACE(log, "Stream stalled. Reschedule."); - if (milliseconds_to_wait < BACKOFF_TRESHOLD) - milliseconds_to_wait *= 2; + if (!path_is_directory + && milliseconds_to_wait + < static_cast(filelog_settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) + milliseconds_to_wait *= filelog_settings->poll_directory_watch_events_backoff_factor.value; break; } else { - milliseconds_to_wait = RESCHEDULE_MS; + milliseconds_to_wait = filelog_settings->poll_directory_watch_events_backoff_init.totalMilliseconds(); } auto ts = std::chrono::steady_clock::now(); @@ -611,7 +610,6 @@ void StorageFileLog::threadFunc() LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); break; } - updateFileInfos(); } } } @@ -622,12 +620,27 @@ void StorageFileLog::threadFunc() // Wait for attached views if (!task->stream_cancelled) - task->holder->scheduleAfter(milliseconds_to_wait); + { + if (path_is_directory) + { + std::unique_lock lock(mutex); + /// Waiting for watch directory thread to wake up + cv.wait(lock, [this] { return has_new_events; }); + has_new_events = false; + task->holder->schedule(); + } + else + task->holder->scheduleAfter(milliseconds_to_wait); + } } bool StorageFileLog::streamToViews() { - std::lock_guard lock(status_mutex); + if (running_streams.load(std::memory_order_relaxed)) + { + throw Exception("Another select query is running on this table, need to wait it finish.", ErrorCodes::CANNOT_SELECT); + } + has_dependent_mv = true; Stopwatch watch; auto table_id = getStorageID(); @@ -726,6 +739,19 @@ void registerStorageFileLog(StorageFactory & factory) throw Exception("filelog_poll_max_batch_size can not be lower than 1", ErrorCodes::BAD_ARGUMENTS); } + size_t init_sleep_time = filelog_settings->poll_directory_watch_events_backoff_init.totalMilliseconds(); + size_t max_sleep_time = filelog_settings->poll_directory_watch_events_backoff_max.totalMilliseconds(); + if (init_sleep_time > max_sleep_time) + { + throw Exception( + "poll_directory_watch_events_backoff_init can not be greater than poll_directory_watch_events_backoff_max", + ErrorCodes::BAD_ARGUMENTS); + } + + if (filelog_settings->poll_directory_watch_events_backoff_factor.changed + && !filelog_settings->poll_directory_watch_events_backoff_factor.value) + throw Exception("poll_directory_watch_events_backoff_factor can not be 0", ErrorCodes::BAD_ARGUMENTS); + if (args_count != 2) throw Exception( "Arguments size of StorageFileLog should be 2, path and format name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -763,11 +789,9 @@ bool StorageFileLog::updateFileInfos() /// For table just watch one file, we can not use directory monitor to watch it if (!path_is_directory) { -#ifndef NDEBUG assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); assert(file_infos.file_names.size() == 1); -#endif if (auto it = file_infos.context_by_name.find(file_infos.file_names[0]); it != file_infos.context_by_name.end()) { @@ -783,18 +807,16 @@ bool StorageFileLog::updateFileInfos() if (error.has_error) LOG_ERROR(log, "Error happened during watching directory {}: {}", directory_watch->getPath(), error.error_msg); -/// These file infos should always have same size(one for one) before update and after update -#ifndef NDEBUG + /// These file infos should always have same size(one for one) before update and after update assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); -#endif auto events = directory_watch->getEventsAndReset(); - for (const auto & [file_name, event_infos] : events) + for (const auto & [file_name, event_info] : events) { String file_path = getFullDataPath(file_name); - for (const auto & event_info : event_infos) + for(const auto & event_info : event_info.file_events) { switch (event_info.type) { @@ -903,22 +925,20 @@ bool StorageFileLog::updateFileInfos() } file_infos.file_names.swap(valid_files); -/// These file infos should always have same size(one for one) -#ifndef NDEBUG + /// These file infos should always have same size(one for one) assert(file_infos.file_names.size() == file_infos.meta_by_inode.size()); assert(file_infos.file_names.size() == file_infos.context_by_name.size()); -#endif return events.empty() || file_infos.file_names.empty(); } NamesAndTypesList StorageFileLog::getVirtuals() const { - return NamesAndTypesList{{"_file_name", std::make_shared()}, {"_offset", std::make_shared()}}; + return NamesAndTypesList{{"_filename", std::make_shared()}, {"_offset", std::make_shared()}}; } Names StorageFileLog::getVirtualColumnNames() { - return {"_file_name", "_offset"}; + return {"_filename", "_offset"}; } } diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 1b1d5f6671a..04c83d8837d 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include @@ -11,6 +10,7 @@ #include #include +#include #include #include #include @@ -23,6 +23,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +class FileLogDirectoryWatcher; + class StorageFileLog final : public shared_ptr_helper, public IStorage, WithContext { friend struct shared_ptr_helper; @@ -75,8 +77,8 @@ public: struct FileMeta { String file_name; - UInt64 last_writen_position{}; - UInt64 last_open_end{}; + UInt64 last_writen_position = 0; + UInt64 last_open_end = 0; }; using InodeToFileMeta = std::unordered_map; @@ -124,6 +126,12 @@ public: void increaseStreams(); void reduceStreams(); + auto & getConditionVariable() { return cv; } + auto & getMutex() { return mutex; } + void setNewEvents() { has_new_events = true; } + + const auto & getFileLogSettings() const { return filelog_settings; } + protected: StorageFileLog( const StorageID & table_id_, @@ -155,8 +163,6 @@ private: const String format_name; Poco::Logger * log; - std::mutex status_mutex; - std::unique_ptr directory_watch = nullptr; uint64_t milliseconds_to_wait; @@ -177,6 +183,12 @@ private: /// later select should forbid to execute. std::atomic running_streams = 0; + std::atomic has_dependent_mv = false; + + std::mutex mutex; + bool has_new_events = false; + std::condition_variable cv; + void loadFiles(); void loadMetaFiles(bool attach); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 4de7cbc71ff..419c1bf173c 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -586,7 +586,8 @@ public: /// Does not takes underlying Storage (if any) into account. virtual std::optional lifetimeBytes() const { return {}; } - /// Do we should call table->drop immediately when drop table + /// Should table->drop be called at once or with delay (in case of atomic database engine). + /// Needed for integration engines, when there must be no delay for calling drop() method. virtual bool dropTableImmediately() { return false; } private: From 074e02eb1433828138ca06a11c259c6a994841f0 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 18 Oct 2021 06:38:47 +0000 Subject: [PATCH 178/919] fix --- src/Storages/FileLog/DirectoryWatcherBase.cpp | 34 ++--- src/Storages/FileLog/DirectoryWatcherBase.h | 3 +- .../FileLog/FileLogDirectoryWatcher.cpp | 2 +- .../FileLog/FileLogDirectoryWatcher.h | 2 +- .../FileLog/ReadBufferFromFileLog.cpp | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 47 +++--- src/Storages/FileLog/StorageFileLog.h | 33 ++-- .../02023_storage_filelog.reference | 143 ++++++++++++++++++ .../0_stateless/02023_storage_filelog.sh | 19 ++- .../0_stateless/02024_storage_filelog_mv.sh | 9 +- 10 files changed, 228 insertions(+), 66 deletions(-) diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index cad2072a8e6..b5d914f700d 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -1,6 +1,8 @@ #include #include #include +#include +#include #include #include @@ -20,7 +22,11 @@ static constexpr int buffer_size = 4096; DirectoryWatcherBase::DirectoryWatcherBase( FileLogDirectoryWatcher & owner_, const std::string & path_, ContextPtr context_, int event_mask_) - : WithContext(context_), owner(owner_), path(path_), event_mask(event_mask_) + : WithContext(context_) + , owner(owner_) + , path(path_) + , event_mask(event_mask_) + , milliseconds_to_wait(owner.storage.getFileLogSettings()->poll_directory_watch_events_backoff_init.totalMilliseconds()) { if (!std::filesystem::exists(path)) throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Path {} does not exist", path); @@ -64,8 +70,9 @@ void DirectoryWatcherBase::watchFunc() while (!stopped) { const auto & settings = owner.storage.getFileLogSettings(); - if (poll(&pfd, 1, 500) > 0 && pfd.revents & POLLIN) + if (poll(&pfd, 1, milliseconds_to_wait) > 0 && pfd.revents & POLLIN) { + milliseconds_to_wait = settings->poll_directory_watch_events_backoff_init.totalMilliseconds(); int n = read(fd, buffer.data(), buffer.size()); int i = 0; if (n > 0) @@ -109,33 +116,20 @@ void DirectoryWatcherBase::watchFunc() } /// Wake up reader thread - auto & mutex = owner.storage.getMutex(); - auto & cv = owner.storage.getConditionVariable(); - std::unique_lock lock(mutex); - owner.storage.setNewEvents(); - lock.unlock(); - cv.notify_one(); + owner.storage.wakeUp(); } - else - { - if (milliseconds_to_wait < static_cast(settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) - milliseconds_to_wait *= settings->poll_directory_watch_events_backoff_factor.value; - break; + else + { + if (milliseconds_to_wait < static_cast(settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) + milliseconds_to_wait *= settings->poll_directory_watch_events_backoff_factor.value; } } - - if (!stopped) - watch_task->scheduleAfter(milliseconds_to_wait); } - DirectoryWatcherBase::~DirectoryWatcherBase() { stop(); close(fd); - - if (watch_task) - watch_task->deactivate(); } void DirectoryWatcherBase::start() diff --git a/src/Storages/FileLog/DirectoryWatcherBase.h b/src/Storages/FileLog/DirectoryWatcherBase.h index 88f864ac17d..a640f686c8a 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.h +++ b/src/Storages/FileLog/DirectoryWatcherBase.h @@ -97,10 +97,11 @@ private: std::atomic stopped{false}; - uint64_t milliseconds_to_wait; const std::string path; int event_mask; + uint64_t milliseconds_to_wait; + int fd; }; diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp index c4a153dbfe7..192721f9f3c 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.cpp +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.cpp @@ -78,9 +78,9 @@ void FileLogDirectoryWatcher::onItemModified(DirectoryWatcherBase::DirectoryEven auto event_path = ev.path; EventInfo info{ev.event, "onItemModified"}; - /// Already have MODIFY event for this file if (auto it = events.find(event_path); it != events.end()) { + /// Already have MODIFY event for this file if (it->second.received_modification_event) return; else diff --git a/src/Storages/FileLog/FileLogDirectoryWatcher.h b/src/Storages/FileLog/FileLogDirectoryWatcher.h index 8e0ecf5358c..0b0c86397aa 100644 --- a/src/Storages/FileLog/FileLogDirectoryWatcher.h +++ b/src/Storages/FileLog/FileLogDirectoryWatcher.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include @@ -10,6 +9,7 @@ namespace DB { +class StorageFileLog; class FileLogDirectoryWatcher { diff --git a/src/Storages/FileLog/ReadBufferFromFileLog.cpp b/src/Storages/FileLog/ReadBufferFromFileLog.cpp index 384de64cd3f..a55df9fe09e 100644 --- a/src/Storages/FileLog/ReadBufferFromFileLog.cpp +++ b/src/Storages/FileLog/ReadBufferFromFileLog.cpp @@ -117,7 +117,7 @@ void ReadBufferFromFileLog::readNewRecords(ReadBufferFromFileLog::Records & new_ { /// Need to get offset before reading record from stream auto offset = reader.tellg(); - if (static_cast(offset) < file_meta.last_open_end) + if (static_cast(offset) >= file_meta.last_open_end) break; record.offset = offset; StorageFileLog::assertStreamGood(reader); diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 66add8128ef..81303f623aa 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -47,8 +47,6 @@ namespace ErrorCodes namespace { - const auto RESCHEDULE_MS = 500; - const auto BACKOFF_TRESHOLD = 32000; const auto MAX_THREAD_WORK_DURATION_MS = 60000; } @@ -57,7 +55,6 @@ StorageFileLog::StorageFileLog( ContextPtr context_, const ColumnsDescription & columns_, const String & path_, - const String & relative_data_path_, const String & format_name_, std::unique_ptr settings, const String & comment, @@ -66,10 +63,9 @@ StorageFileLog::StorageFileLog( , WithContext(context_->getGlobalContext()) , filelog_settings(std::move(settings)) , path(path_) - , relative_data_path(relative_data_path_) , format_name(format_name_) , log(&Poco::Logger::get("StorageFileLog (" + table_id_.table_name + ")")) - , milliseconds_to_wait(RESCHEDULE_MS) + , milliseconds_to_wait(filelog_settings->poll_directory_watch_events_backoff_init.totalMilliseconds()) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); @@ -100,9 +96,9 @@ StorageFileLog::StorageFileLog( void StorageFileLog::loadMetaFiles(bool attach) { - const auto & storage_id = getStorageID(); - root_meta_path = std::filesystem::path(getContext()->getPath()) / "metadata" / "filelog_storage_metadata" / storage_id.getDatabaseName() - / storage_id.getTableName(); + const auto & storage = getStorageID(); + root_meta_path + = std::filesystem::path(getContext()->getPath()) / ".filelog_storage_metadata" / storage.getDatabaseName() / storage.getTableName(); /// Attach table if (attach) @@ -110,8 +106,8 @@ void StorageFileLog::loadMetaFiles(bool attach) /// Meta file may lost, log and create directory if (!std::filesystem::exists(root_meta_path)) { + /// Create root_meta_path directory when store meta data LOG_ERROR(log, "Metadata files of table {} are lost.", getStorageID().getTableName()); - std::filesystem::create_directories(root_meta_path); } /// Load all meta info to file_infos; deserialize(); @@ -180,8 +176,8 @@ void StorageFileLog::loadFiles() /// data file have been renamed, need update meta file's name if (it->second.file_name != file) { - it->second.file_name = file; std::filesystem::rename(getFullMetaPath(it->second.file_name), getFullMetaPath(file)); + it->second.file_name = file; } } /// New file @@ -261,6 +257,8 @@ void StorageFileLog::serialize(UInt64 inode, const FileMeta & file_meta) const void StorageFileLog::deserialize() { + if (!std::filesystem::exists(root_meta_path)) + return; /// In case of single file (not a watched directory), /// iterated directoy always has one file inside. for (const auto & dir_entry : std::filesystem::directory_iterator{root_meta_path}) @@ -324,7 +322,7 @@ Pipe StorageFileLog::read( getStorageID().getTableName()); } - if (running_streams.load(std::memory_order_relaxed)) + if (running_streams) { throw Exception("Another select query is running on this table, need to wait it finish.", ErrorCodes::CANNOT_SELECT); } @@ -409,6 +407,9 @@ void StorageFileLog::shutdown() { task->stream_cancelled = true; + /// Reader thread may wait for wake up + wakeUp(); + LOG_TRACE(log, "Waiting for cleanup"); task->holder->deactivate(); } @@ -623,10 +624,13 @@ void StorageFileLog::threadFunc() { if (path_is_directory) { - std::unique_lock lock(mutex); - /// Waiting for watch directory thread to wake up + std::unique_lock lock(mutex); + /// Waiting for watch directory thread to wake up cv.wait(lock, [this] { return has_new_events; }); has_new_events = false; + + if (task->stream_cancelled) + return; task->holder->schedule(); } else @@ -636,7 +640,7 @@ void StorageFileLog::threadFunc() bool StorageFileLog::streamToViews() { - if (running_streams.load(std::memory_order_relaxed)) + if (running_streams) { throw Exception("Another select query is running on this table, need to wait it finish.", ErrorCodes::CANNOT_SELECT); } @@ -702,6 +706,14 @@ bool StorageFileLog::streamToViews() return updateFileInfos(); } +void StorageFileLog::wakeUp() +{ + std::unique_lock lock(mutex); + has_new_events = true; + lock.unlock(); + cv.notify_one(); +} + void registerStorageFileLog(StorageFactory & factory) { auto creator_fn = [](const StorageFactory::Arguments & args) @@ -767,7 +779,6 @@ void registerStorageFileLog(StorageFactory & factory) args.getContext(), args.columns, path, - args.relative_data_path, format, std::move(filelog_settings), args.comment, @@ -813,10 +824,10 @@ bool StorageFileLog::updateFileInfos() auto events = directory_watch->getEventsAndReset(); - for (const auto & [file_name, event_info] : events) + for (const auto & [file_name, event_infos] : events) { String file_path = getFullDataPath(file_name); - for(const auto & event_info : event_info.file_events) + for (const auto & event_info : event_infos.file_events) { switch (event_info.type) { @@ -836,7 +847,7 @@ bool StorageFileLog::updateFileInfos() file_infos.meta_by_inode.emplace(inode, FileMeta{.file_name = file_name}); if (auto it = file_infos.context_by_name.find(file_name); it != file_infos.context_by_name.end()) - it->second = FileContext{.inode = inode}; + it->second = FileContext{.status = FileStatus::OPEN, .inode = inode}; else file_infos.context_by_name.emplace(file_name, FileContext{.inode = inode}); } diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index 04c83d8837d..da87cd1be5a 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -126,9 +127,7 @@ public: void increaseStreams(); void reduceStreams(); - auto & getConditionVariable() { return cv; } - auto & getMutex() { return mutex; } - void setNewEvents() { has_new_events = true; } + void wakeUp(); const auto & getFileLogSettings() const { return filelog_settings; } @@ -138,7 +137,6 @@ protected: ContextPtr context_, const ColumnsDescription & columns_, const String & path_, - const String & relative_data_path_, const String & format_name_, std::unique_ptr settings, const String & comment, @@ -148,14 +146,11 @@ private: std::unique_ptr filelog_settings; const String path; - /// For meta file - const String relative_data_path; bool path_is_directory = true; /// If path argument of the table is a regular file, it equals to user_files_path /// otherwise, it equals to user_files_path/ + path_argument/, e.g. path String root_data_path; - /// relative_data_path/ + table_name/ String root_meta_path; FileInfos file_infos; @@ -163,20 +158,8 @@ private: const String format_name; Poco::Logger * log; - std::unique_ptr directory_watch = nullptr; - uint64_t milliseconds_to_wait; - struct TaskContext - { - BackgroundSchedulePool::TaskHolder holder; - std::atomic stream_cancelled {false}; - explicit TaskContext(BackgroundSchedulePool::TaskHolder&& task_) : holder(std::move(task_)) - { - } - }; - std::shared_ptr task; - /// In order to avoid data race, using a naive trick to forbid execute two select /// simultaneously, although read is not useful in this engine. Using an atomic /// variable to records current unfinishing streams, then if have unfinishing streams, @@ -189,6 +172,18 @@ private: bool has_new_events = false; std::condition_variable cv; + struct TaskContext + { + BackgroundSchedulePool::TaskHolder holder; + std::atomic stream_cancelled {false}; + explicit TaskContext(BackgroundSchedulePool::TaskHolder&& task_) : holder(std::move(task_)) + { + } + }; + std::shared_ptr task; + + std::unique_ptr directory_watch = nullptr; + void loadFiles(); void loadMetaFiles(bool attach); diff --git a/tests/queries/0_stateless/02023_storage_filelog.reference b/tests/queries/0_stateless/02023_storage_filelog.reference index 0ab71c65c6b..c787d2047db 100644 --- a/tests/queries/0_stateless/02023_storage_filelog.reference +++ b/tests/queries/0_stateless/02023_storage_filelog.reference @@ -141,4 +141,147 @@ 120 120 120 120 120 120 +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 +100 100 +101 101 +102 102 +103 103 +104 104 +105 105 +106 106 +107 107 +108 108 +109 109 +110 110 +111 111 +112 112 +113 113 +114 114 +115 115 +116 116 +117 117 +118 118 +119 119 +120 120 +150 150 +151 151 +152 152 +153 153 +154 154 +155 155 +156 156 +157 157 +158 158 +159 159 +160 160 +161 161 +162 162 +163 163 +164 164 +165 165 +166 166 +167 167 +168 168 +169 169 +170 170 +171 171 +172 172 +173 173 +174 174 +175 175 +176 176 +177 177 +178 178 +179 179 +180 180 +181 181 +182 182 +183 183 +184 184 +185 185 +186 186 +187 187 +188 188 +189 189 +190 190 +191 191 +192 192 +193 193 +194 194 +195 195 +196 196 +197 197 +198 198 +199 199 +200 200 +200 200 +201 201 +202 202 +203 203 +204 204 +205 205 +206 206 +207 207 +208 208 +209 209 +210 210 +211 211 +212 212 +213 213 +214 214 +215 215 +216 216 +217 217 +218 218 +219 219 +220 220 +221 221 +222 222 +223 223 +224 224 +225 225 +226 226 +227 227 +228 228 +229 229 +230 230 +231 231 +232 232 +233 233 +234 234 +235 235 +236 236 +237 237 +238 238 +239 239 +240 240 +241 241 +242 242 +243 243 +244 244 +245 245 +246 246 +247 247 +248 248 +249 249 +250 250 OK diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index b695c270835..b13bb4da065 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # 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')" -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}') +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 ${user_files_path}/logs/ @@ -41,11 +41,28 @@ touch ${user_files_path}/logs/a.txt cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/c.txt cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/d.txt cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/e.txt +mv ${user_files_path}/logs/b.txt ${user_files_path}/logs/j.txt rm ${user_files_path}/logs/d.txt ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" +${CLICKHOUSE_CLIENT} --query "detach table file_log;" +cp ${user_files_path}/logs/e.txt ${user_files_path}/logs/f.txt +mv ${user_files_path}/logs/e.txt ${user_files_path}/logs/g.txt +mv ${user_files_path}/logs/c.txt ${user_files_path}/logs/h.txt +for i in {150..200} +do + echo $i, $i >> ${user_files_path}/logs/h.txt +done +for i in {200..250} +do + echo $i, $i >> ${user_files_path}/logs/i.txt +done +${CLICKHOUSE_CLIENT} --query "attach table file_log;" + +${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" + ${CLICKHOUSE_CLIENT} --query "detach table file_log;" ${CLICKHOUSE_CLIENT} --query "attach table file_log;" diff --git a/tests/queries/0_stateless/02024_storage_filelog_mv.sh b/tests/queries/0_stateless/02024_storage_filelog_mv.sh index 0503552b8dd..95dfb74ecf6 100755 --- a/tests/queries/0_stateless/02024_storage_filelog_mv.sh +++ b/tests/queries/0_stateless/02024_storage_filelog_mv.sh @@ -14,10 +14,6 @@ user_files_path=$(clickhouse-client --query "select _path,_file from file('nonex mkdir -p ${user_files_path}/logs/ rm -rf ${user_files_path}/logs/* -for i in {1..20} -do - echo $i, $i >> ${user_files_path}/logs/a.txt -done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" ${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/logs/', 'CSV');" @@ -25,6 +21,11 @@ ${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=Fil ${CLICKHOUSE_CLIENT} --query "drop table if exists mv;" ${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order by k as select * from file_log;" +for i in {1..20} +do + echo $i, $i >> ${user_files_path}/logs/a.txt +done + for i in {1..200} do sleep 0.1 From a32abda322c6cf3728d9529721f8ccd9067843a6 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 18 Oct 2021 07:08:37 +0000 Subject: [PATCH 179/919] fix --- src/Storages/FileLog/StorageFileLog.cpp | 21 ++++++++++++------- .../0_stateless/02024_storage_filelog_mv.sh | 10 ++++----- 2 files changed, 18 insertions(+), 13 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 81303f623aa..f807663dbec 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -624,14 +624,19 @@ void StorageFileLog::threadFunc() { if (path_is_directory) { - std::unique_lock lock(mutex); - /// Waiting for watch directory thread to wake up - cv.wait(lock, [this] { return has_new_events; }); - has_new_events = false; + if (milliseconds_to_wait < static_cast(filelog_settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) + task->holder->scheduleAfter(milliseconds_to_wait); + else + { + std::unique_lock lock(mutex); + /// Waiting for watch directory thread to wake up + cv.wait(lock, [this] { return has_new_events; }); + has_new_events = false; - if (task->stream_cancelled) - return; - task->holder->schedule(); + if (task->stream_cancelled) + return; + task->holder->schedule(); + } } else task->holder->scheduleAfter(milliseconds_to_wait); @@ -711,7 +716,7 @@ void StorageFileLog::wakeUp() std::unique_lock lock(mutex); has_new_events = true; lock.unlock(); - cv.notify_one(); + cv.notify_one(); } void registerStorageFileLog(StorageFactory & factory) diff --git a/tests/queries/0_stateless/02024_storage_filelog_mv.sh b/tests/queries/0_stateless/02024_storage_filelog_mv.sh index 95dfb74ecf6..5170906b8d9 100755 --- a/tests/queries/0_stateless/02024_storage_filelog_mv.sh +++ b/tests/queries/0_stateless/02024_storage_filelog_mv.sh @@ -15,17 +15,17 @@ user_files_path=$(clickhouse-client --query "select _path,_file from file('nonex mkdir -p ${user_files_path}/logs/ rm -rf ${user_files_path}/logs/* +for i in {1..20} +do + echo $i, $i >> ${user_files_path}/logs/a.txt +done + ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" ${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/logs/', 'CSV');" ${CLICKHOUSE_CLIENT} --query "drop table if exists mv;" ${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order by k as select * from file_log;" -for i in {1..20} -do - echo $i, $i >> ${user_files_path}/logs/a.txt -done - for i in {1..200} do sleep 0.1 From 6cf82f859cd77377dfdc737b480000350066fee8 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 18 Oct 2021 07:27:37 +0000 Subject: [PATCH 180/919] fix --- src/Storages/FileLog/StorageFileLog.cpp | 19 +++++++++++++------ src/Storages/FileLog/StorageFileLog.h | 2 ++ .../0_stateless/02023_storage_filelog.sh | 2 +- 3 files changed, 16 insertions(+), 7 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index f807663dbec..1323331639d 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -569,14 +569,22 @@ bool StorageFileLog::checkDependencies(const StorageID & table_id) return true; } +size_t StorageFileLog::getTableDependentCount() const +{ + auto table_id = getStorageID(); + // Check if at least one direct dependency is attached + return DatabaseCatalog::instance().getDependencies(table_id).size(); +} + void StorageFileLog::threadFunc() { try { updateFileInfos(); + auto table_id = getStorageID(); - // Check if at least one direct dependency is attached - size_t dependencies_count = DatabaseCatalog::instance().getDependencies(table_id).size(); + + auto dependencies_count = getTableDependentCount(); if (dependencies_count) { @@ -593,9 +601,8 @@ void StorageFileLog::threadFunc() if (streamToViews()) { LOG_TRACE(log, "Stream stalled. Reschedule."); - if (!path_is_directory - && milliseconds_to_wait - < static_cast(filelog_settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) + if (milliseconds_to_wait + < static_cast(filelog_settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) milliseconds_to_wait *= filelog_settings->poll_directory_watch_events_backoff_factor.value; break; } @@ -624,7 +631,7 @@ void StorageFileLog::threadFunc() { if (path_is_directory) { - if (milliseconds_to_wait < static_cast(filelog_settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) + if (!getTableDependentCount()) task->holder->scheduleAfter(milliseconds_to_wait); else { diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index da87cd1be5a..f0aac1a7985 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -199,6 +199,8 @@ private: bool updateFileInfos(); + size_t getTableDependentCount() const; + /// Used in shutdown() void serialize() const; /// Used in FileSource closeFileAndStoreMeta(file_name); diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index b13bb4da065..8279e09ff65 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # 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')" -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}') +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 ${user_files_path}/logs/ From 0c628726af1894c1f78ba3db4961a86c53ef3adb Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 18 Oct 2021 07:48:58 +0000 Subject: [PATCH 181/919] fix build --- src/Storages/FileLog/FileLogSource.cpp | 1 - src/Storages/FileLog/StorageFileLog.cpp | 4 ++-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index bb0fcc90016..f046e745046 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 1323331639d..2a6e124e6e8 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -13,19 +13,19 @@ #include #include #include -#include +#include #include #include #include #include #include +#include #include #include #include #include #include #include -#include #include From 3d39ea8e37758c033f8c1e7e6abd24fee0662930 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 18 Oct 2021 11:06:30 +0300 Subject: [PATCH 182/919] Fix WHERE --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 31e8fe6454f..6909b3d2575 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -821,6 +821,16 @@ Columns MergeTreeRangeReader::continueReadingChain(ReadResult & result, size_t & const auto & rows_per_granule = result.rowsPerGranule(); const auto & started_ranges = result.startedRanges(); + auto current_task_last_mark_range = std::max_element(started_ranges.begin(), started_ranges.end(), + [&](const ReadResult::RangeInfo & lhs, const ReadResult::RangeInfo & rhs) + { + return lhs.range.end < rhs.range.end; + }); + + size_t current_task_last_mark = 0; + if (current_task_last_mark_range != started_ranges.end()) + current_task_last_mark = current_task_last_mark_range->range.end; + size_t next_range_to_start = 0; auto size = rows_per_granule.size(); @@ -832,7 +842,7 @@ Columns MergeTreeRangeReader::continueReadingChain(ReadResult & result, size_t & num_rows += stream.finalize(columns); const auto & range = started_ranges[next_range_to_start].range; ++next_range_to_start; - stream = Stream(range.begin, range.end, 0, merge_tree_reader); + stream = Stream(range.begin, range.end, current_task_last_mark, merge_tree_reader); } bool last = i + 1 == size; From 81fb4bcf76f995e70cdb0f062ffc6e4366210930 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 18 Oct 2021 08:14:54 +0000 Subject: [PATCH 183/919] fix style --- src/Storages/FileLog/DirectoryWatcherBase.cpp | 8 ++++---- src/Storages/FileLog/StorageFileLog.cpp | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index b5d914f700d..c459079ec06 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -118,10 +118,10 @@ void DirectoryWatcherBase::watchFunc() /// Wake up reader thread owner.storage.wakeUp(); } - else - { - if (milliseconds_to_wait < static_cast(settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) - milliseconds_to_wait *= settings->poll_directory_watch_events_backoff_factor.value; + else + { + if (milliseconds_to_wait < static_cast(settings->poll_directory_watch_events_backoff_max.totalMilliseconds())) + milliseconds_to_wait *= settings->poll_directory_watch_events_backoff_factor.value; } } } diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 2a6e124e6e8..92c30a86b49 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -260,7 +260,7 @@ void StorageFileLog::deserialize() if (!std::filesystem::exists(root_meta_path)) return; /// In case of single file (not a watched directory), - /// iterated directoy always has one file inside. + /// iterated directory always has one file inside. for (const auto & dir_entry : std::filesystem::directory_iterator{root_meta_path}) { if (!dir_entry.is_regular_file()) @@ -317,7 +317,7 @@ Pipe StorageFileLog::read( { throw Exception( ErrorCodes::QUERY_NOT_ALLOWED, - "Can not make `SELECT` query from table {}, because it has attached dependencies. Remove dependant materialized views if " + "Can not make `SELECT` query from table {}, because it has attached dependencies. Remove dependent materialized views if " "needed", getStorageID().getTableName()); } From 1131a3ebe8056071b6a3aad19f88e53d90cc5ab9 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 18 Oct 2021 12:15:08 +0300 Subject: [PATCH 184/919] fix test --- .../0_stateless/01604_explain_ast_of_nonselect_query.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01604_explain_ast_of_nonselect_query.reference b/tests/queries/0_stateless/01604_explain_ast_of_nonselect_query.reference index 3545278ec1a..3c92ffb0a8c 100644 --- a/tests/queries/0_stateless/01604_explain_ast_of_nonselect_query.reference +++ b/tests/queries/0_stateless/01604_explain_ast_of_nonselect_query.reference @@ -1,6 +1,6 @@ AlterQuery t1 (children 1) ExpressionList (children 1) - AlterCommand 32 (children 1) + AlterCommand 33 (children 1) Function equals (children 1) ExpressionList (children 2) Identifier date From e543f720c696cc22169256492807aee9dd8d287f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 18 Oct 2021 12:29:34 +0300 Subject: [PATCH 185/919] fix clang-tidy --- .../Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h index 29209f462e4..8b83e4e215b 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h @@ -40,7 +40,7 @@ public: AggregatingTransformParamsPtr params_, SortDescription description_, size_t max_block_size_, - size_t max_threads_); + size_t merge_threads_); void initialize(Inputs inputs) override; void consume(Input & input, size_t source_num) override; From b2976fadb065571124ca675c04314d98e51b7933 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 18 Oct 2021 11:57:03 +0000 Subject: [PATCH 186/919] fix test --- .../0_stateless/02025_storage_filelog_virtual_col.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index 781253c2184..a92f93991ef 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -24,11 +24,11 @@ done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" ${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/logs/', 'CSV');" -${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" +${CLICKHOUSE_CLIENT} --query "select *, _filename, _offset from file_log order by _filename, _offset;" cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/b.txt -${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" +${CLICKHOUSE_CLIENT} --query "select *, _filename, _offset from file_log order by _filename, _offset;" for i in {100..120} do @@ -44,13 +44,13 @@ cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/e.txt rm ${user_files_path}/logs/d.txt -${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" +${CLICKHOUSE_CLIENT} --query "select *, _filename, _offset from file_log order by _filename, _offset;" ${CLICKHOUSE_CLIENT} --query "detach table file_log;" ${CLICKHOUSE_CLIENT} --query "attach table file_log;" # should no records return -${CLICKHOUSE_CLIENT} --query "select *, _file_name, _offset from file_log order by _file_name, _offset;" +${CLICKHOUSE_CLIENT} --query "select *, _filename, _offset from file_log order by _filename, _offset;" truncate ${user_files_path}/logs/a.txt --size 0 From 2a780bf96825571b4e668cfca9dbd1f78ea4a9d9 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 18 Oct 2021 15:48:43 +0300 Subject: [PATCH 187/919] Fix build --- src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index e3d3c11c33a..ae7ecb955b0 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include From 8cae60c0f658dd5c44c1b3bc588993356411f682 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 18 Oct 2021 15:51:40 +0300 Subject: [PATCH 188/919] Fix resetParser --- src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index bb1d95c0634..ee616cb0630 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -189,6 +189,7 @@ void RowInputFormatWithNamesAndTypes::resetParser() RowInputFormatWithDiagnosticInfo::resetParser(); column_mapping->column_indexes_for_input_fields.clear(); column_mapping->not_presented_columns.clear(); + column_mapping->names_of_columns.clear(); } void RowInputFormatWithNamesAndTypes::tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) From f861da2dd10d894d12c2d2e6f2b6316b0813889f Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 15 Oct 2021 19:59:28 +0300 Subject: [PATCH 189/919] Fix [I]LIKE function --- src/Functions/MatchImpl.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/MatchImpl.h b/src/Functions/MatchImpl.h index d71a5a1ffe3..7dc0712023f 100644 --- a/src/Functions/MatchImpl.h +++ b/src/Functions/MatchImpl.h @@ -200,7 +200,7 @@ struct MatchImpl } /// We check that the entry does not pass through the boundaries of strings. - if (pos + strstr_pattern.size() < begin + offsets[i]) + if (pos + required_substring.size() < begin + offsets[i]) { /// And if it does not, if necessary, we check the regexp. @@ -344,7 +344,7 @@ struct MatchImpl const UInt8 * next_pos = begin; /// If required substring is larger than string size - it cannot be found. - if (strstr_pattern.size() <= n) + if (required_substring.size() <= n) { Searcher searcher(required_substring.data(), required_substring.size(), end - pos); @@ -360,7 +360,7 @@ struct MatchImpl } next_pos += n; - if (pos + strstr_pattern.size() <= next_pos) + if (pos + required_substring.size() <= next_pos) { /// And if it does not, if necessary, we check the regexp. From f18dca38369c17e5d55ce78b090671be0637a370 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 15 Oct 2021 20:00:08 +0300 Subject: [PATCH 190/919] Add test --- .../queries/0_stateless/02045_like_function.reference | 2 ++ tests/queries/0_stateless/02045_like_function.sql | 10 ++++++++++ 2 files changed, 12 insertions(+) create mode 100644 tests/queries/0_stateless/02045_like_function.reference create mode 100644 tests/queries/0_stateless/02045_like_function.sql diff --git a/tests/queries/0_stateless/02045_like_function.reference b/tests/queries/0_stateless/02045_like_function.reference new file mode 100644 index 00000000000..0633853274a --- /dev/null +++ b/tests/queries/0_stateless/02045_like_function.reference @@ -0,0 +1,2 @@ +1 +1 1 1 1 1 1 diff --git a/tests/queries/0_stateless/02045_like_function.sql b/tests/queries/0_stateless/02045_like_function.sql new file mode 100644 index 00000000000..d395e8d4579 --- /dev/null +++ b/tests/queries/0_stateless/02045_like_function.sql @@ -0,0 +1,10 @@ +SELECT 'r\\a1bbb' LIKE '%r\\\\a1%bbb%' AS res; + +WITH lower('\RealVNC\WinVNC4 /v password') as CommandLine +SELECT + CommandLine LIKE '%\\\\realvnc\\\\winvnc4%password%' as t1, + CommandLine LIKE '%\\\\realvnc\\\\winvnc4 %password%' as t2, + CommandLine LIKE '%\\\\realvnc\\\\winvnc4%password' as t3, + CommandLine LIKE '%\\\\realvnc\\\\winvnc4 %password' as t4, + CommandLine LIKE '%realvnc%winvnc4%password%' as t5, + CommandLine LIKE '%\\\\winvnc4%password%' as t6; From 5a4a752cfbf3de81f17e407a3dfccaffa6b27e13 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 18 Oct 2021 17:37:46 +0300 Subject: [PATCH 191/919] Unite sorting steps. --- src/Interpreters/InterpreterSelectQuery.cpp | 50 +--- src/Processors/QueryPlan/MergeSortingStep.cpp | 96 ------- src/Processors/QueryPlan/MergeSortingStep.h | 47 ---- .../QueryPlan/MergingSortedStep.cpp | 85 ------ src/Processors/QueryPlan/MergingSortedStep.h | 36 --- .../QueryPlan/PartialSortingStep.cpp | 93 ------- src/Processors/QueryPlan/PartialSortingStep.h | 35 --- src/Processors/QueryPlan/SortingStep.cpp | 241 ++++++++++++++++++ src/Processors/QueryPlan/SortingStep.h | 76 ++++++ 9 files changed, 327 insertions(+), 432 deletions(-) delete mode 100644 src/Processors/QueryPlan/MergeSortingStep.cpp delete mode 100644 src/Processors/QueryPlan/MergeSortingStep.h delete mode 100644 src/Processors/QueryPlan/MergingSortedStep.cpp delete mode 100644 src/Processors/QueryPlan/MergingSortedStep.h delete mode 100644 src/Processors/QueryPlan/PartialSortingStep.cpp delete mode 100644 src/Processors/QueryPlan/PartialSortingStep.h create mode 100644 src/Processors/QueryPlan/SortingStep.cpp create mode 100644 src/Processors/QueryPlan/SortingStep.h diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 35c8c32c65b..c6ef18d7660 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -43,15 +43,12 @@ #include #include #include -#include #include #include #include -#include +#include #include -#include #include -#include #include #include #include @@ -2260,35 +2257,20 @@ void InterpreterSelectQuery::executeWindow(QueryPlan & query_plan) // happens in case of `over ()`. if (!w.full_sort_description.empty() && (i == 0 || !sortIsPrefix(w, *windows_sorted[i - 1]))) { - auto partial_sorting = std::make_unique( - query_plan.getCurrentDataStream(), - w.full_sort_description, - 0 /* LIMIT */, - SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode)); - partial_sorting->setStepDescription("Sort each block for window '" + w.window_name + "'"); - query_plan.addStep(std::move(partial_sorting)); - auto merge_sorting_step = std::make_unique( + auto sorting_step = std::make_unique( query_plan.getCurrentDataStream(), w.full_sort_description, 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, context->getTemporaryVolume(), settings.min_free_disk_space_for_temporary_data); - merge_sorting_step->setStepDescription("Merge sorted blocks for window '" + w.window_name + "'"); - query_plan.addStep(std::move(merge_sorting_step)); - - // First MergeSorted, now MergingSorted. - auto merging_sorted = std::make_unique( - query_plan.getCurrentDataStream(), - w.full_sort_description, - settings.max_block_size, - 0 /* LIMIT */); - merging_sorted->setStepDescription("Merge sorted streams for window '" + w.window_name + "'"); - query_plan.addStep(std::move(merging_sorted)); + sorting_step->setStepDescription("Sorting for window '" + w.window_name + "'"); + query_plan.addStep(std::move(sorting_step)); } auto window_step = std::make_unique(query_plan.getCurrentDataStream(), w, w.window_functions); @@ -2303,7 +2285,7 @@ void InterpreterSelectQuery::executeOrderOptimized(QueryPlan & query_plan, Input { const Settings & settings = context->getSettingsRef(); - auto finish_sorting_step = std::make_unique( + auto finish_sorting_step = std::make_unique( query_plan.getCurrentDataStream(), input_sorting_info->order_key_prefix_descr, output_order_descr, @@ -2333,32 +2315,20 @@ void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfo const Settings & settings = context->getSettingsRef(); - auto partial_sorting = std::make_unique( - query_plan.getCurrentDataStream(), - output_order_descr, - limit, - SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode)); - - partial_sorting->setStepDescription("Sort each block for ORDER BY"); - query_plan.addStep(std::move(partial_sorting)); - /// Merge the sorted blocks. - auto merge_sorting_step = std::make_unique( + auto sorting_step = std::make_unique( query_plan.getCurrentDataStream(), output_order_descr, settings.max_block_size, 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, context->getTemporaryVolume(), settings.min_free_disk_space_for_temporary_data); - merge_sorting_step->setStepDescription("Merge sorted blocks for ORDER BY"); - query_plan.addStep(std::move(merge_sorting_step)); - - /// If there are several streams, we merge them into one - executeMergeSorted(query_plan, output_order_descr, limit, "for ORDER BY"); + sorting_step->setStepDescription("Sorting for ORDER BY"); } @@ -2376,7 +2346,7 @@ void InterpreterSelectQuery::executeMergeSorted(QueryPlan & query_plan, const So const Settings & settings = context->getSettingsRef(); auto merging_sorted - = std::make_unique(query_plan.getCurrentDataStream(), sort_description, settings.max_block_size, limit); + = std::make_unique(query_plan.getCurrentDataStream(), sort_description, settings.max_block_size, limit); merging_sorted->setStepDescription("Merge sorted streams " + description); query_plan.addStep(std::move(merging_sorted)); diff --git a/src/Processors/QueryPlan/MergeSortingStep.cpp b/src/Processors/QueryPlan/MergeSortingStep.cpp deleted file mode 100644 index 534f05a4d6e..00000000000 --- a/src/Processors/QueryPlan/MergeSortingStep.cpp +++ /dev/null @@ -1,96 +0,0 @@ -#include -#include -#include -#include -#include - -namespace DB -{ - -static ITransformingStep::Traits getTraits(size_t limit) -{ - return ITransformingStep::Traits - { - { - .preserves_distinct_columns = true, - .returns_single_stream = false, - .preserves_number_of_streams = true, - .preserves_sorting = false, - }, - { - .preserves_number_of_rows = limit == 0, - } - }; -} - -MergeSortingStep::MergeSortingStep( - const DataStream & input_stream, - const SortDescription & description_, - size_t max_merged_block_size_, - UInt64 limit_, - size_t max_bytes_before_remerge_, - double remerge_lowered_memory_bytes_ratio_, - size_t max_bytes_before_external_sort_, - VolumePtr tmp_volume_, - size_t min_free_disk_space_) - : ITransformingStep(input_stream, input_stream.header, getTraits(limit_)) - , description(description_) - , max_merged_block_size(max_merged_block_size_) - , limit(limit_) - , max_bytes_before_remerge(max_bytes_before_remerge_) - , remerge_lowered_memory_bytes_ratio(remerge_lowered_memory_bytes_ratio_) - , max_bytes_before_external_sort(max_bytes_before_external_sort_), tmp_volume(tmp_volume_) - , min_free_disk_space(min_free_disk_space_) -{ - /// TODO: check input_stream is partially sorted by the same description. - output_stream->sort_description = description; - output_stream->sort_mode = input_stream.has_single_port ? DataStream::SortMode::Stream - : DataStream::SortMode::Port; -} - -void MergeSortingStep::updateLimit(size_t limit_) -{ - if (limit_ && (limit == 0 || limit_ < limit)) - { - limit = limit_; - transform_traits.preserves_number_of_rows = false; - } -} - -void MergeSortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr - { - if (stream_type == QueryPipelineBuilder::StreamType::Totals) - return nullptr; - - return std::make_shared( - header, description, max_merged_block_size, limit, - max_bytes_before_remerge / pipeline.getNumStreams(), - remerge_lowered_memory_bytes_ratio, - max_bytes_before_external_sort, - tmp_volume, - min_free_disk_space); - }); -} - -void MergeSortingStep::describeActions(FormatSettings & settings) const -{ - String prefix(settings.offset, ' '); - settings.out << prefix << "Sort description: "; - dumpSortDescription(description, input_streams.front().header, settings.out); - settings.out << '\n'; - - if (limit) - settings.out << prefix << "Limit " << limit << '\n'; -} - -void MergeSortingStep::describeActions(JSONBuilder::JSONMap & map) const -{ - map.add("Sort Description", explainSortDescription(description, input_streams.front().header)); - - if (limit) - map.add("Limit", limit); -} - -} diff --git a/src/Processors/QueryPlan/MergeSortingStep.h b/src/Processors/QueryPlan/MergeSortingStep.h deleted file mode 100644 index 947ced829c6..00000000000 --- a/src/Processors/QueryPlan/MergeSortingStep.h +++ /dev/null @@ -1,47 +0,0 @@ -#pragma once -#include -#include -#include -#include - -namespace DB -{ - -/// Sorts stream of data. See MergeSortingTransform. -class MergeSortingStep : public ITransformingStep -{ -public: - explicit MergeSortingStep( - const DataStream & input_stream, - const SortDescription & description_, - size_t max_merged_block_size_, - UInt64 limit_, - size_t max_bytes_before_remerge_, - double remerge_lowered_memory_bytes_ratio_, - size_t max_bytes_before_external_sort_, - VolumePtr tmp_volume_, - size_t min_free_disk_space_); - - String getName() const override { return "MergeSorting"; } - - void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - - void describeActions(JSONBuilder::JSONMap & map) const override; - void describeActions(FormatSettings & settings) const override; - - /// Add limit or change it to lower value. - void updateLimit(size_t limit_); - -private: - SortDescription description; - size_t max_merged_block_size; - UInt64 limit; - - size_t max_bytes_before_remerge; - double remerge_lowered_memory_bytes_ratio; - size_t max_bytes_before_external_sort; - VolumePtr tmp_volume; - size_t min_free_disk_space; -}; - -} diff --git a/src/Processors/QueryPlan/MergingSortedStep.cpp b/src/Processors/QueryPlan/MergingSortedStep.cpp deleted file mode 100644 index ed1f24db55b..00000000000 --- a/src/Processors/QueryPlan/MergingSortedStep.cpp +++ /dev/null @@ -1,85 +0,0 @@ -#include -#include -#include -#include -#include - -namespace DB -{ - -static ITransformingStep::Traits getTraits(size_t limit) -{ - return ITransformingStep::Traits - { - { - .preserves_distinct_columns = true, - .returns_single_stream = true, - .preserves_number_of_streams = false, - .preserves_sorting = false, - }, - { - .preserves_number_of_rows = limit == 0, - } - }; -} - -MergingSortedStep::MergingSortedStep( - const DataStream & input_stream, - SortDescription sort_description_, - size_t max_block_size_, - UInt64 limit_) - : ITransformingStep(input_stream, input_stream.header, getTraits(limit_)) - , sort_description(std::move(sort_description_)) - , max_block_size(max_block_size_) - , limit(limit_) -{ - /// TODO: check input_stream is partially sorted (each port) by the same description. - output_stream->sort_description = sort_description; - output_stream->sort_mode = DataStream::SortMode::Stream; -} - -void MergingSortedStep::updateLimit(size_t limit_) -{ - if (limit_ && (limit == 0 || limit_ < limit)) - { - limit = limit_; - transform_traits.preserves_number_of_rows = false; - } -} - -void MergingSortedStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - /// If there are several streams, then we merge them into one - if (pipeline.getNumStreams() > 1) - { - - auto transform = std::make_shared( - pipeline.getHeader(), - pipeline.getNumStreams(), - sort_description, - max_block_size, limit); - - pipeline.addTransform(std::move(transform)); - } -} - -void MergingSortedStep::describeActions(FormatSettings & settings) const -{ - String prefix(settings.offset, ' '); - settings.out << prefix << "Sort description: "; - dumpSortDescription(sort_description, input_streams.front().header, settings.out); - settings.out << '\n'; - - if (limit) - settings.out << prefix << "Limit " << limit << '\n'; -} - -void MergingSortedStep::describeActions(JSONBuilder::JSONMap & map) const -{ - map.add("Sort Description", explainSortDescription(sort_description, input_streams.front().header)); - - if (limit) - map.add("Limit", limit); -} - -} diff --git a/src/Processors/QueryPlan/MergingSortedStep.h b/src/Processors/QueryPlan/MergingSortedStep.h deleted file mode 100644 index e886de42ca8..00000000000 --- a/src/Processors/QueryPlan/MergingSortedStep.h +++ /dev/null @@ -1,36 +0,0 @@ -#pragma once -#include -#include -#include -#include - -namespace DB -{ - -/// Merge streams of data into single sorted stream. -class MergingSortedStep : public ITransformingStep -{ -public: - explicit MergingSortedStep( - const DataStream & input_stream, - SortDescription sort_description_, - size_t max_block_size_, - UInt64 limit_ = 0); - - String getName() const override { return "MergingSorted"; } - - void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - - void describeActions(JSONBuilder::JSONMap & map) const override; - void describeActions(FormatSettings & settings) const override; - - /// Add limit or change it to lower value. - void updateLimit(size_t limit_); - -private: - SortDescription sort_description; - size_t max_block_size; - UInt64 limit; -}; - -} diff --git a/src/Processors/QueryPlan/PartialSortingStep.cpp b/src/Processors/QueryPlan/PartialSortingStep.cpp deleted file mode 100644 index 420e7b583ca..00000000000 --- a/src/Processors/QueryPlan/PartialSortingStep.cpp +++ /dev/null @@ -1,93 +0,0 @@ -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -static ITransformingStep::Traits getTraits(size_t limit) -{ - return ITransformingStep::Traits - { - { - .preserves_distinct_columns = true, - .returns_single_stream = false, - .preserves_number_of_streams = true, - .preserves_sorting = false, - }, - { - .preserves_number_of_rows = limit == 0, - } - }; -} - -PartialSortingStep::PartialSortingStep( - const DataStream & input_stream, - SortDescription sort_description_, - UInt64 limit_, - SizeLimits size_limits_) - : ITransformingStep(input_stream, input_stream.header, getTraits(limit_)) - , sort_description(std::move(sort_description_)) - , limit(limit_) - , size_limits(size_limits_) -{ - output_stream->sort_description = sort_description; - output_stream->sort_mode = DataStream::SortMode::Chunk; -} - -void PartialSortingStep::updateLimit(size_t limit_) -{ - if (limit_ && (limit == 0 || limit_ < limit)) - { - limit = limit_; - transform_traits.preserves_number_of_rows = false; - } -} - -void PartialSortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr - { - if (stream_type != QueryPipelineBuilder::StreamType::Main) - return nullptr; - - return std::make_shared(header, sort_description, limit); - }); - - StreamLocalLimits limits; - limits.mode = LimitsMode::LIMITS_CURRENT; //-V1048 - limits.size_limits = size_limits; - - pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr - { - if (stream_type != QueryPipelineBuilder::StreamType::Main) - return nullptr; - - auto transform = std::make_shared(header, limits); - return transform; - }); -} - -void PartialSortingStep::describeActions(FormatSettings & settings) const -{ - String prefix(settings.offset, ' '); - settings.out << prefix << "Sort description: "; - dumpSortDescription(sort_description, input_streams.front().header, settings.out); - settings.out << '\n'; - - if (limit) - settings.out << prefix << "Limit " << limit << '\n'; -} - -void PartialSortingStep::describeActions(JSONBuilder::JSONMap & map) const -{ - map.add("Sort Description", explainSortDescription(sort_description, input_streams.front().header)); - - if (limit) - map.add("Limit", limit); -} - -} diff --git a/src/Processors/QueryPlan/PartialSortingStep.h b/src/Processors/QueryPlan/PartialSortingStep.h deleted file mode 100644 index 9b7b8e8baa5..00000000000 --- a/src/Processors/QueryPlan/PartialSortingStep.h +++ /dev/null @@ -1,35 +0,0 @@ -#pragma once -#include -#include -#include - -namespace DB -{ - -/// Sort separate chunks of data. -class PartialSortingStep : public ITransformingStep -{ -public: - explicit PartialSortingStep( - const DataStream & input_stream, - SortDescription sort_description_, - UInt64 limit_, - SizeLimits size_limits_); - - String getName() const override { return "PartialSorting"; } - - void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - - void describeActions(JSONBuilder::JSONMap & map) const override; - void describeActions(FormatSettings & settings) const override; - - /// Add limit or change it to lower value. - void updateLimit(size_t limit_); - -private: - SortDescription sort_description; - UInt64 limit; - SizeLimits size_limits; -}; - -} diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp new file mode 100644 index 00000000000..32b314b1c50 --- /dev/null +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -0,0 +1,241 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +static ITransformingStep::Traits getTraits(size_t limit) +{ + return ITransformingStep::Traits + { + { + .preserves_distinct_columns = true, + .returns_single_stream = true, + .preserves_number_of_streams = false, + .preserves_sorting = false, + }, + { + .preserves_number_of_rows = limit == 0, + } + }; +} + +SortingStep::SortingStep( + const DataStream & input_stream, + const SortDescription & description_, + size_t max_block_size_, + UInt64 limit_, + SizeLimits size_limits_, + size_t max_bytes_before_remerge_, + double remerge_lowered_memory_bytes_ratio_, + size_t max_bytes_before_external_sort_, + VolumePtr tmp_volume_, + size_t min_free_disk_space_) + : ITransformingStep(input_stream, input_stream.header, getTraits(limit_)) + , type(Type::Full) + , result_description(description_) + , max_block_size(max_block_size_) + , limit(limit_) + , size_limits(size_limits_) + , max_bytes_before_remerge(max_bytes_before_remerge_) + , remerge_lowered_memory_bytes_ratio(remerge_lowered_memory_bytes_ratio_) + , max_bytes_before_external_sort(max_bytes_before_external_sort_), tmp_volume(tmp_volume_) + , min_free_disk_space(min_free_disk_space_) +{ + /// TODO: check input_stream is partially sorted by the same description. + output_stream->sort_description = result_description; + output_stream->sort_mode = DataStream::SortMode::Stream; +} + +SortingStep::SortingStep( + const DataStream & input_stream_, + SortDescription prefix_description_, + SortDescription result_description_, + size_t max_block_size_, + UInt64 limit_) + : ITransformingStep(input_stream_, input_stream_.header, getTraits(limit_)) + , type(Type::FinishSorting) + , prefix_description(std::move(prefix_description_)) + , result_description(std::move(result_description_)) + , max_block_size(max_block_size_) + , limit(limit_) +{ + /// TODO: check input_stream is sorted by prefix_description. + output_stream->sort_description = result_description; + output_stream->sort_mode = DataStream::SortMode::Stream; +} + +SortingStep::SortingStep( + const DataStream & input_stream, + SortDescription sort_description_, + size_t max_block_size_, + UInt64 limit_) + : ITransformingStep(input_stream, input_stream.header, getTraits(limit_)) + , type(Type::MergingSorted) + , result_description(std::move(sort_description_)) + , max_block_size(max_block_size_) + , limit(limit_) +{ + /// TODO: check input_stream is partially sorted (each port) by the same description. + output_stream->sort_description = result_description; + output_stream->sort_mode = DataStream::SortMode::Stream; +} + +void SortingStep::updateLimit(size_t limit_) +{ + if (limit_ && (limit == 0 || limit_ < limit)) + { + limit = limit_; + transform_traits.preserves_number_of_rows = false; + } +} + +void SortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + if (type == Type::FinishSorting) + { + bool need_finish_sorting = (prefix_description.size() < result_description.size()); + if (pipeline.getNumStreams() > 1) + { + UInt64 limit_for_merging = (need_finish_sorting ? 0 : limit); + auto transform = std::make_shared( + pipeline.getHeader(), + pipeline.getNumStreams(), + prefix_description, + max_block_size, + limit_for_merging); + + pipeline.addTransform(std::move(transform)); + } + + if (need_finish_sorting) + { + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + { + if (stream_type != QueryPipelineBuilder::StreamType::Main) + return nullptr; + + return std::make_shared(header, result_description, limit); + }); + + /// NOTE limits are not applied to the size of temporary sets in FinishSortingTransform + pipeline.addSimpleTransform([&](const Block & header) -> ProcessorPtr + { + return std::make_shared( + header, prefix_description, result_description, max_block_size, limit); + }); + } + } + else if (type == Type::Full) + { + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + { + if (stream_type != QueryPipelineBuilder::StreamType::Main) + return nullptr; + + return std::make_shared(header, result_description, limit); + }); + + StreamLocalLimits limits; + limits.mode = LimitsMode::LIMITS_CURRENT; //-V1048 + limits.size_limits = size_limits; + + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + { + if (stream_type != QueryPipelineBuilder::StreamType::Main) + return nullptr; + + auto transform = std::make_shared(header, limits); + return transform; + }); + + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + { + if (stream_type == QueryPipelineBuilder::StreamType::Totals) + return nullptr; + + return std::make_shared( + header, result_description, max_block_size, limit, + max_bytes_before_remerge / pipeline.getNumStreams(), + remerge_lowered_memory_bytes_ratio, + max_bytes_before_external_sort, + tmp_volume, + min_free_disk_space); + }); + + /// If there are several streams, then we merge them into one + if (pipeline.getNumStreams() > 1) + { + + auto transform = std::make_shared( + pipeline.getHeader(), + pipeline.getNumStreams(), + result_description, + max_block_size, limit); + + pipeline.addTransform(std::move(transform)); + } + } + else if (type == Type::MergingSorted) + { /// If there are several streams, then we merge them into one + if (pipeline.getNumStreams() > 1) + { + + auto transform = std::make_shared( + pipeline.getHeader(), + pipeline.getNumStreams(), + result_description, + max_block_size, limit); + + pipeline.addTransform(std::move(transform)); + } + } +} + +void SortingStep::describeActions(FormatSettings & settings) const +{ + String prefix(settings.offset, ' '); + + if (!prefix_description.empty()) + { + settings.out << prefix << "Prefix sort description: "; + dumpSortDescription(prefix_description, input_streams.front().header, settings.out); + settings.out << '\n'; + + settings.out << prefix << "Result sort description: "; + dumpSortDescription(result_description, input_streams.front().header, settings.out); + settings.out << '\n'; + } + else + { + settings.out << prefix << "Sort description: "; + dumpSortDescription(result_description, input_streams.front().header, settings.out); + settings.out << '\n'; + } + + if (limit) + settings.out << prefix << "Limit " << limit << '\n'; +} + +void SortingStep::describeActions(JSONBuilder::JSONMap & map) const +{ + if (!prefix_description.empty()) + { + map.add("Prefix Sort Description", explainSortDescription(prefix_description, input_streams.front().header)); + map.add("Result Sort Description", explainSortDescription(result_description, input_streams.front().header)); + } + else + map.add("Sort Description", explainSortDescription(result_description, input_streams.front().header)); + + if (limit) + map.add("Limit", limit); +} + +} diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h new file mode 100644 index 00000000000..e34ee2842da --- /dev/null +++ b/src/Processors/QueryPlan/SortingStep.h @@ -0,0 +1,76 @@ +#pragma once +#include +#include +#include +#include + +namespace DB +{ + +/// Sort data stream +class SortingStep : public ITransformingStep +{ +public: + /// Full + SortingStep( + const DataStream & input_stream, + const SortDescription & description_, + size_t max_block_size_, + UInt64 limit_, + SizeLimits size_limits_, + size_t max_bytes_before_remerge_, + double remerge_lowered_memory_bytes_ratio_, + size_t max_bytes_before_external_sort_, + VolumePtr tmp_volume_, + size_t min_free_disk_space_); + + /// FinishSorting + SortingStep( + const DataStream & input_stream_, + SortDescription prefix_description_, + SortDescription result_description_, + size_t max_block_size_, + UInt64 limit_); + + /// MergingSorted + SortingStep( + const DataStream & input_stream, + SortDescription sort_description_, + size_t max_block_size_, + UInt64 limit_ = 0); + + String getName() const override { return "Sorting"; } + + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + void describeActions(JSONBuilder::JSONMap & map) const override; + void describeActions(FormatSettings & settings) const override; + + /// Add limit or change it to lower value. + void updateLimit(size_t limit_); + +private: + + enum class Type + { + Full, + FinishSorting, + MergingSorted, + }; + + Type type; + + SortDescription prefix_description; + SortDescription result_description; + size_t max_block_size; + UInt64 limit; + SizeLimits size_limits; + + size_t max_bytes_before_remerge; + double remerge_lowered_memory_bytes_ratio; + size_t max_bytes_before_external_sort; + VolumePtr tmp_volume; + size_t min_free_disk_space; +}; + +} From e9f3cf665199b358e79c6a3b938a5cb25a33fe46 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 18 Oct 2021 17:49:26 +0300 Subject: [PATCH 192/919] Fix used memory calculation --- src/Common/ProgressIndication.cpp | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index bf3397f50e1..5a3f8cfc350 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -121,11 +121,9 @@ ProgressIndication::MemoryUsage ProgressIndication::getMemoryUsage() const return std::accumulate(thread_data.cbegin(), thread_data.cend(), MemoryUsage{}, [](MemoryUsage const & acc, auto const & host_data) { - auto host_usage = std::accumulate(host_data.second.cbegin(), host_data.second.cend(), ZERO, - [](UInt64 memory, auto const & data) - { - return memory + data.second.memory_usage; - }); + UInt64 host_usage = 0; + if (auto it = host_data.second.find(ZERO); it != host_data.second.end()) + host_usage = it->second.memory_usage; return MemoryUsage{.total = acc.total + host_usage, .max = std::max(acc.max, host_usage)}; }); } From bfe2a937eb16d67fef754bbf3b17ef043bba8776 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 18 Oct 2021 18:27:51 +0300 Subject: [PATCH 193/919] At least able to start --- programs/server/Server.cpp | 3 + src/Coordination/Changelog.cpp | 8 ++ src/Coordination/Changelog.h | 2 + src/Coordination/KeeperDispatcher.cpp | 15 ++ src/Coordination/KeeperDispatcher.h | 3 +- src/Coordination/KeeperLogStore.cpp | 6 + src/Coordination/KeeperLogStore.h | 2 + src/Coordination/KeeperServer.cpp | 51 +++++++ src/Coordination/KeeperServer.h | 2 + src/Coordination/KeeperSnapshotManager.cpp | 45 ++++-- src/Coordination/KeeperSnapshotManager.h | 22 ++- src/Coordination/KeeperStateMachine.cpp | 34 ++++- src/Coordination/KeeperStateMachine.h | 8 ++ src/Coordination/KeeperStateManager.cpp | 143 ++++++++++++++------ src/Coordination/KeeperStateManager.h | 57 ++++++-- src/Coordination/fuzzers/CMakeLists.txt | 2 + src/Coordination/fuzzers/request_fuzzer.cpp | 36 +++++ src/Interpreters/Context.cpp | 38 ++++-- src/Interpreters/Context.h | 2 + 19 files changed, 393 insertions(+), 86 deletions(-) create mode 100644 src/Coordination/fuzzers/CMakeLists.txt create mode 100644 src/Coordination/fuzzers/request_fuzzer.cpp diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 2b526608715..48ba22e08d0 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -863,6 +863,9 @@ if (ThreadFuzzer::instance().isEffective()) if (config->has("max_concurrent_queries")) global_context->getProcessList().setMaxSize(config->getInt("max_concurrent_queries", 0)); + if (config->has("keeper_server")) + global_context->updateKeeperConfiguration(*config); + if (!initial_loading) { /// We do not load ZooKeeper configuration on the first config loading diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 14df3aab03b..eabb1f2c5a6 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -631,6 +631,14 @@ LogEntryPtr Changelog::entryAt(uint64_t index) return src; } +LogEntryPtr Changelog::getLatestConfigChange() const +{ + for (const auto & [_, entry] : logs) + if (entry->get_val_type() == nuraft::conf) + return entry; + return nullptr; +} + nuraft::ptr Changelog::serializeEntriesToBuffer(uint64_t index, int32_t count) { std::vector> returned_logs; diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 87ec359cd3c..196c7c4a1e5 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -102,6 +102,8 @@ public: /// Last entry in log, or fake entry with term 0 if log is empty LogEntryPtr getLastEntry() const; + LogEntryPtr getLatestConfigChange() const; + /// Return log entries between [start, end) LogEntriesPtr getLogEntriesBetween(uint64_t start_index, uint64_t end_index); diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index a28e8d96915..3b8681aabb7 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -273,6 +273,8 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf server->waitInit(); LOG_DEBUG(log, "Quorum initialized"); + + updateConfiguration(config); } catch (...) { @@ -497,4 +499,17 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms) return future.get(); } +void KeeperDispatcher::updateConfiguration(const Poco::Util::AbstractConfiguration & config) +{ + if (isLeader()) + { + server->updateConfiguration(config); + } + else + { + LOG_INFO(log, "Configuration changed, but we are not leader, so we will wait update from leader"); + } + +} + } diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index f49063f8dea..30993037a5b 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -62,7 +62,6 @@ private: ThreadFromGlobalPool session_cleaner_thread; /// Dumping new snapshots to disk ThreadFromGlobalPool snapshot_thread; - /// RAFT wrapper. std::unique_ptr server; @@ -102,6 +101,8 @@ public: /// standalone_keeper -- we are standalone keeper application (not inside clickhouse server) void initialize(const Poco::Util::AbstractConfiguration & config, bool standalone_keeper); + void updateConfiguration(const Poco::Util::AbstractConfiguration & config); + /// Shutdown internal keeper parts (server, state machine, log storage, etc) void shutdown(); diff --git a/src/Coordination/KeeperLogStore.cpp b/src/Coordination/KeeperLogStore.cpp index 7e08449306e..2736deaf2af 100644 --- a/src/Coordination/KeeperLogStore.cpp +++ b/src/Coordination/KeeperLogStore.cpp @@ -112,4 +112,10 @@ void KeeperLogStore::end_of_append_batch(uint64_t /*start_index*/, uint64_t /*co changelog.flush(); } +nuraft::ptr KeeperLogStore::getLatestConfigChange() const +{ + std::lock_guard lock(changelog_lock); + return changelog.getLatestConfigChange(); +} + } diff --git a/src/Coordination/KeeperLogStore.h b/src/Coordination/KeeperLogStore.h index bd0ecb78130..9e840e9699e 100644 --- a/src/Coordination/KeeperLogStore.h +++ b/src/Coordination/KeeperLogStore.h @@ -58,6 +58,8 @@ public: /// Flush batch of appended entries void end_of_append_batch(uint64_t start_index, uint64_t count) override; + nuraft::ptr getLatestConfigChange() const; + private: mutable std::mutex changelog_lock; Poco::Logger * log; diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index b27170c8ba1..6838c383be1 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -121,6 +121,22 @@ void KeeperServer::startup() state_manager->loadLogStore(state_machine->last_commit_index() + 1, coordination_settings->reserved_log_items); + auto latest_snapshot_config = state_machine->getClusterConfig(); + auto latest_log_store_config = state_manager->getLatestConfigFromLogStore(); + + if (latest_snapshot_config && latest_log_store_config) + { + if (latest_snapshot_config->get_log_idx() > latest_log_store_config->get_log_idx()) + state_manager->setClusterConfig(latest_snapshot_config); + else + state_manager->setClusterConfig(latest_log_store_config); + } + else if (latest_snapshot_config) + state_manager->setClusterConfig(latest_snapshot_config); + else if (latest_log_store_config) + state_manager->setClusterConfig(latest_log_store_config); +/// else use parsed config in state_manager constructor (first start) + bool single_server = state_manager->getTotalServers() == 1; nuraft::raft_params params; @@ -363,4 +379,39 @@ std::vector KeeperServer::getDeadSessions() return state_machine->getDeadSessions(); } +void KeeperServer::updateConfiguration(const Poco::Util::AbstractConfiguration & config) +{ + auto diff = state_manager->getConfigurationDiff(config); + if (diff.empty()) + { + LOG_TRACE(log, "Configuration update triggered, but nothing changed for RAFT"); + return; + } + else if (diff.size() > 1) + { + LOG_WARNING(log, "Configuration changed for more than one server ({}) from cluster, it's strictly not recommended", diff.size()); + } + + for (auto & task : diff) + { + if (task.action_type == ConfigUpdateActionType::AddServer) + { + auto result = raft_instance->add_srv(*task.server); + if (!result->get_accepted()) + throw Exception(ErrorCodes::RAFT_ERROR, "Configuration change to add server (id {}) was not accepted by RAFT", task.server->get_id()); + } + else if (task.action_type == ConfigUpdateActionType::RemoveServer) + { + auto result = raft_instance->remove_srv(task.server->get_id()); + if (!result->get_accepted()) + throw Exception(ErrorCodes::RAFT_ERROR, "Configuration change to remove server (id {}) was not accepted by RAFT", task.server->get_id()); + } + else if (task.action_type == ConfigUpdateActionType::UpdatePriority) + raft_instance->set_priority(task.server->get_id(), task.server->get_priority()); + else + LOG_WARNING(log, "Unknown configuration update type {}", static_cast(task.action_type)); + } + +} + } diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index a7e96156dc1..696053726cd 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -83,6 +83,8 @@ public: void shutdown(); int getServerID() const { return server_id; } + + void updateConfiguration(const Poco::Util::AbstractConfiguration & config); }; } diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 0a31878879c..d7ca7b82d78 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -205,9 +205,16 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr writeBinary(id, out); } } + + if (snapshot.cluster_config) + { + auto buffer = snapshot.cluster_config->serialize(); + writeVarUInt(buffer->size(), out); + out.write(reinterpret_cast(buffer->data_begin()), buffer->size()); + } } -SnapshotMetadataPtr KeeperStorageSnapshot::deserialize(KeeperStorage & storage, ReadBuffer & in) +void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in) { uint8_t version; readBinary(version, in); @@ -215,11 +222,13 @@ SnapshotMetadataPtr KeeperStorageSnapshot::deserialize(KeeperStorage & storage, if (current_version > CURRENT_SNAPSHOT_VERSION) throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unsupported snapshot version {}", version); - SnapshotMetadataPtr result = deserializeSnapshotMetadata(in); + deserialization_result.snapshot_meta = deserializeSnapshotMetadata(in); + KeeperStorage & storage = *deserialization_result.storage; + int64_t session_id; readBinary(session_id, in); - storage.zxid = result->get_last_log_idx(); + storage.zxid = deserialization_result.snapshot_meta->get_last_log_idx(); storage.session_id_counter = session_id; /// Before V1 we serialized ACL without acl_map @@ -309,13 +318,24 @@ SnapshotMetadataPtr KeeperStorageSnapshot::deserialize(KeeperStorage & storage, current_session_size++; } - return result; + /// Optional cluster config + ClusterConfigPtr cluster_config = nullptr; + if (!in.eof()) + { + size_t data_size; + readVarUInt(data_size, in); + auto buffer = nuraft::buffer::alloc(data_size); + in.readStrict(reinterpret_cast(buffer->data_begin()), data_size); + buffer->pos(0); + deserialization_result.cluster_config = ClusterConfig::deserialize(*buffer); + } } -KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t up_to_log_idx_) +KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_) : storage(storage_) , snapshot_meta(std::make_shared(up_to_log_idx_, 0, std::make_shared())) , session_id(storage->session_id_counter) + , cluster_config(cluster_config_) { storage->enableSnapshotMode(); snapshot_container_size = storage->container.snapshotSize(); @@ -325,10 +345,11 @@ KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t session_and_auth = storage->session_and_auth; } -KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_) +KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_) : storage(storage_) , snapshot_meta(snapshot_meta_) , session_id(storage->session_id_counter) + , cluster_config(cluster_config_) { storage->enableSnapshotMode(); snapshot_container_size = storage->container.snapshotSize(); @@ -461,7 +482,7 @@ bool KeeperSnapshotManager::isZstdCompressed(nuraft::ptr buffer) return magic_from_buffer == ZSTD_COMPRESSED_MAGIC; } -SnapshotMetaAndStorage KeeperSnapshotManager::deserializeSnapshotFromBuffer(nuraft::ptr buffer) const +SnapshotDeserializationResult KeeperSnapshotManager::deserializeSnapshotFromBuffer(nuraft::ptr buffer) const { bool is_zstd_compressed = isZstdCompressed(buffer); @@ -473,12 +494,13 @@ SnapshotMetaAndStorage KeeperSnapshotManager::deserializeSnapshotFromBuffer(nura else compressed_reader = std::make_unique(*reader); - auto storage = std::make_unique(storage_tick_time, superdigest); - auto snapshot_metadata = KeeperStorageSnapshot::deserialize(*storage, *compressed_reader); - return std::make_pair(snapshot_metadata, std::move(storage)); + SnapshotDeserializationResult result; + result.storage = std::make_unique(storage_tick_time, superdigest); + KeeperStorageSnapshot::deserialize(result, *compressed_reader); + return result; } -SnapshotMetaAndStorage KeeperSnapshotManager::restoreFromLatestSnapshot() +SnapshotDeserializationResult KeeperSnapshotManager::restoreFromLatestSnapshot() { if (existing_snapshots.empty()) return {}; @@ -502,7 +524,6 @@ void KeeperSnapshotManager::removeSnapshot(uint64_t log_idx) throw Exception(ErrorCodes::UNKNOWN_SNAPSHOT, "Unknown snapshot with log index {}", log_idx); std::filesystem::remove(itr->second); existing_snapshots.erase(itr); - } diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 1d14409a574..8dd86f2f94e 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -9,6 +9,8 @@ namespace DB using SnapshotMetadata = nuraft::snapshot; using SnapshotMetadataPtr = std::shared_ptr; +using ClusterConfig = nuraft::cluster_config; +using ClusterConfigPtr = nuraft::ptr; enum SnapshotVersion : uint8_t { @@ -20,6 +22,13 @@ enum SnapshotVersion : uint8_t static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V3; +struct SnapshotDeserializationResult +{ + KeeperStoragePtr storage; + SnapshotMetadataPtr snapshot_meta; + ClusterConfigPtr cluster_config; +}; + /// In memory keeper snapshot. Keeper Storage based on a hash map which can be /// turned into snapshot mode. This operation is fast and KeeperStorageSnapshot /// class do it in constructor. It also copies iterators from storage hash table @@ -31,14 +40,15 @@ static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V3; struct KeeperStorageSnapshot { public: - KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t up_to_log_idx_); + KeeperStorageSnapshot(KeeperStorage * storage_, uint64_t up_to_log_idx_, const ClusterConfigPtr & cluster_config_ = nullptr); + + KeeperStorageSnapshot(KeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_, const ClusterConfigPtr & cluster_config_ = nullptr); - KeeperStorageSnapshot(KeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_); ~KeeperStorageSnapshot(); static void serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out); - static SnapshotMetadataPtr deserialize(KeeperStorage & storage, ReadBuffer & in); + static void deserialize(SnapshotDeserializationResult & deserialization_result, ReadBuffer & in); KeeperStorage * storage; @@ -58,6 +68,8 @@ public: KeeperStorage::SessionAndAuth session_and_auth; /// ACLs cache for better performance. Without we cannot deserialize storage. std::unordered_map acl_map; + /// Cluster config from snapshot, can be empty + ClusterConfigPtr cluster_config; }; using KeeperStorageSnapshotPtr = std::shared_ptr; @@ -76,7 +88,7 @@ public: bool compress_snapshots_zstd_ = true, const std::string & superdigest_ = "", size_t storage_tick_time_ = 500); /// Restore storage from latest available snapshot - SnapshotMetaAndStorage restoreFromLatestSnapshot(); + SnapshotDeserializationResult restoreFromLatestSnapshot(); /// Compress snapshot and serialize it to buffer nuraft::ptr serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const; @@ -84,7 +96,7 @@ public: /// Serialize already compressed snapshot to disk (return path) std::string serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx); - SnapshotMetaAndStorage deserializeSnapshotFromBuffer(nuraft::ptr buffer) const; + SnapshotDeserializationResult deserializeSnapshotFromBuffer(nuraft::ptr buffer) const; /// Deserialize snapshot with log index up_to_log_idx from disk into compressed nuraft buffer. nuraft::ptr deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const; diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index fe64f8afde7..12ec2fb1470 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -74,7 +74,10 @@ void KeeperStateMachine::init() try { latest_snapshot_buf = snapshot_manager.deserializeSnapshotBufferFromDisk(latest_log_index); - std::tie(latest_snapshot_meta, storage) = snapshot_manager.deserializeSnapshotFromBuffer(latest_snapshot_buf); + auto snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(latest_snapshot_buf); + storage = std::move(snapshot_deserialization_result.storage); + latest_snapshot_meta = snapshot_deserialization_result.snapshot_meta; + cluster_config = snapshot_deserialization_result.cluster_config; last_committed_idx = latest_snapshot_meta->get_last_log_idx(); loaded = true; break; @@ -152,13 +155,24 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) { /// deserialize and apply snapshot to storage std::lock_guard lock(storage_and_responses_lock); - std::tie(latest_snapshot_meta, storage) = snapshot_manager.deserializeSnapshotFromBuffer(latest_snapshot_ptr); + auto snapshot_deserialization_result = snapshot_manager.deserializeSnapshotFromBuffer(latest_snapshot_buf); + storage = std::move(snapshot_deserialization_result.storage); + latest_snapshot_meta = snapshot_deserialization_result.snapshot_meta; + cluster_config = snapshot_deserialization_result.cluster_config; } last_committed_idx = s.get_last_log_idx(); return true; } + +void KeeperStateMachine::commit_config(const ulong /*log_idx*/, nuraft::ptr & new_conf) +{ + std::lock_guard lock(cluster_config_lock); + auto tmp = new_conf->serialize(); + cluster_config = ClusterConfig::deserialize(*tmp); +} + nuraft::ptr KeeperStateMachine::last_snapshot() { /// Just return the latest snapshot. @@ -177,7 +191,7 @@ void KeeperStateMachine::create_snapshot( CreateSnapshotTask snapshot_task; { /// lock storage for a short period time to turn on "snapshot mode". After that we can read consistent storage state without locking. std::lock_guard lock(storage_and_responses_lock); - snapshot_task.snapshot = std::make_shared(storage.get(), snapshot_meta_copy); + snapshot_task.snapshot = std::make_shared(storage.get(), snapshot_meta_copy, getClusterConfig()); } /// create snapshot task for background execution (in snapshot thread) @@ -239,7 +253,7 @@ void KeeperStateMachine::save_logical_snp_obj( if (obj_id == 0) /// Fake snapshot required by NuRaft at startup { std::lock_guard lock(storage_and_responses_lock); - KeeperStorageSnapshot snapshot(storage.get(), s.get_last_log_idx()); + KeeperStorageSnapshot snapshot(storage.get(), s.get_last_log_idx(), getClusterConfig()); cloned_buffer = snapshot_manager.serializeSnapshotToBuffer(snapshot); } else @@ -324,4 +338,16 @@ void KeeperStateMachine::shutdownStorage() storage->finalize(); } +ClusterConfigPtr KeeperStateMachine::getClusterConfig() const +{ + std::lock_guard lock(cluster_config_lock); + if (cluster_config) + { + /// dumb way to return copy... + auto tmp = cluster_config->serialize(); + return ClusterConfig::deserialize(*tmp); + } + return nullptr; +} + } diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 983692f7b7f..1afc99aabc1 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -32,6 +32,8 @@ public: nuraft::ptr commit(const uint64_t log_idx, nuraft::buffer & data) override; + void commit_config(const uint64_t log_idx, nuraft::ptr & new_conf) override; + /// Currently not supported void rollback(const uint64_t /*log_idx*/, nuraft::buffer & /*data*/) override {} @@ -76,6 +78,8 @@ public: void shutdownStorage(); + ClusterConfigPtr getClusterConfig() const; + private: /// In our state machine we always have a single snapshot which is stored @@ -109,8 +113,12 @@ private: /// Last committed Raft log number. std::atomic last_committed_idx; + Poco::Logger * log; + mutable std::mutex cluster_config_lock; + ClusterConfigPtr cluster_config; + /// Special part of ACL system -- superdigest specified in server config. const std::string superdigest; }; diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 2e12b7488c0..cca9f503294 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -31,42 +31,22 @@ namespace } -KeeperStateManager::KeeperStateManager(int server_id_, const std::string & host, int port, const std::string & logs_path) - : my_server_id(server_id_) - , my_port(port) - , secure(false) - , log_store(nuraft::cs_new(logs_path, 5000, false, false)) - , cluster_config(nuraft::cs_new()) -{ - auto peer_config = nuraft::cs_new(my_server_id, host + ":" + std::to_string(port)); - cluster_config->get_servers().push_back(peer_config); -} -KeeperStateManager::KeeperStateManager( - int my_server_id_, - const std::string & config_prefix, - const Poco::Util::AbstractConfiguration & config, - const CoordinationSettingsPtr & coordination_settings, - bool standalone_keeper) - : my_server_id(my_server_id_) - , secure(config.getBool(config_prefix + ".raft_configuration.secure", false)) - , log_store(nuraft::cs_new( - getLogsPathFromConfig(config_prefix, config, standalone_keeper), - coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync, coordination_settings->compress_logs)) - , cluster_config(nuraft::cs_new()) +KeeperServersConfiguration KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfiguration & config) const { - + KeeperServersConfiguration result; + result.cluster_config = std::make_shared(); Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix + ".raft_configuration", keys); - total_servers = keys.size(); + size_t total_servers = 0; for (const auto & server_key : keys) { if (!startsWith(server_key, "server")) continue; std::string full_prefix = config_prefix + ".raft_configuration." + server_key; - int server_id = config.getInt(full_prefix + ".id"); + int new_server_id = config.getInt(full_prefix + ".id"); std::string hostname = config.getString(full_prefix + ".hostname"); int port = config.getInt(full_prefix + ".port"); bool can_become_leader = config.getBool(full_prefix + ".can_become_leader", true); @@ -74,24 +54,57 @@ KeeperStateManager::KeeperStateManager( bool start_as_follower = config.getBool(full_prefix + ".start_as_follower", false); if (start_as_follower) - start_as_follower_servers.insert(server_id); + result.servers_start_as_followers.insert(new_server_id); auto endpoint = hostname + ":" + std::to_string(port); - auto peer_config = nuraft::cs_new(server_id, 0, endpoint, "", !can_become_leader, priority); - if (server_id == my_server_id) + auto peer_config = nuraft::cs_new(new_server_id, 0, endpoint, "", !can_become_leader, priority); + if (my_server_id == new_server_id) { - my_server_config = peer_config; - my_port = port; + result.config = peer_config; + result.port = port; } - cluster_config->get_servers().push_back(peer_config); + result.cluster_config->get_servers().push_back(peer_config); + total_servers++; } - if (!my_server_config) + if (!result.config) throw Exception(ErrorCodes::RAFT_ERROR, "Our server id {} not found in raft_configuration section", my_server_id); - if (start_as_follower_servers.size() == cluster_config->get_servers().size()) + if (result.servers_start_as_followers.size() == total_servers) throw Exception(ErrorCodes::RAFT_ERROR, "At least one of servers should be able to start as leader (without )"); + + return result; +} + +KeeperStateManager::KeeperStateManager(int server_id_, const std::string & host, int port, const std::string & logs_path) + : my_server_id(server_id_) + , secure(false) + , log_store(nuraft::cs_new(logs_path, 5000, false, false)) + , log(&Poco::Logger::get("KeeperStateManager")) +{ + auto peer_config = nuraft::cs_new(my_server_id, host + ":" + std::to_string(port)); + servers_configuration.cluster_config = nuraft::cs_new(); + servers_configuration.port = port; + servers_configuration.config = peer_config; + servers_configuration.cluster_config->get_servers().push_back(peer_config); +} + +KeeperStateManager::KeeperStateManager( + int server_id_, + const std::string & config_prefix_, + const Poco::Util::AbstractConfiguration & config, + const CoordinationSettingsPtr & coordination_settings, + bool standalone_keeper) + : my_server_id(server_id_) + , secure(config.getBool(config_prefix_ + ".raft_configuration.secure", false)) + , config_prefix(config_prefix_) + , servers_configuration(parseServersConfiguration(config)) + , log_store(nuraft::cs_new( + getLogsPathFromConfig(config_prefix_, config, standalone_keeper), + coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync, coordination_settings->compress_logs)) + , log(&Poco::Logger::get("KeeperStateManager")) +{ } void KeeperStateManager::loadLogStore(uint64_t last_commited_index, uint64_t logs_to_keep) @@ -99,6 +112,19 @@ void KeeperStateManager::loadLogStore(uint64_t last_commited_index, uint64_t log log_store->init(last_commited_index, logs_to_keep); } +ClusterConfigPtr KeeperStateManager::getLatestConfigFromLogStore() const +{ + auto entry_with_change = log_store->getLatestConfigChange(); + if (entry_with_change) + return ClusterConfig::deserialize(entry_with_change->get_buf()); + return nullptr; +} + +void KeeperStateManager::setClusterConfig(const ClusterConfigPtr & cluster_config) +{ + servers_configuration.cluster_config = cluster_config; +} + void KeeperStateManager::flushLogStore() { log_store->flush(); @@ -106,18 +132,57 @@ void KeeperStateManager::flushLogStore() void KeeperStateManager::save_config(const nuraft::cluster_config & config) { - // Just keep in memory in this example. - // Need to write to disk here, if want to make it durable. nuraft::ptr buf = config.serialize(); - cluster_config = nuraft::cluster_config::deserialize(*buf); + servers_configuration.cluster_config = nuraft::cluster_config::deserialize(*buf); } void KeeperStateManager::save_state(const nuraft::srv_state & state) { - // Just keep in memory in this example. - // Need to write to disk here, if want to make it durable. nuraft::ptr buf = state.serialize(); server_state = nuraft::srv_state::deserialize(*buf); - } +} + +ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const +{ + auto new_servers_configuration = parseServersConfiguration(config); + if (new_servers_configuration.port != servers_configuration.port) + throw Exception(ErrorCodes::RAFT_ERROR, "Cannot change port of already running RAFT server"); + + std::unordered_map new_ids, old_ids; + for (auto new_server : new_servers_configuration.cluster_config->get_servers()) + new_ids[new_server->get_id()] = new_server; + + for (auto old_server : servers_configuration.cluster_config->get_servers()) + old_ids[old_server->get_id()] = old_server; + + std::unordered_map servers_to_remove, servers_to_add; + + auto comp = [] (auto & a, auto & b) { return a.first < b.first; }; + std::set_difference(old_ids.begin(), old_ids.end(), new_ids.begin(), new_ids.end(), std::inserter(servers_to_remove, servers_to_remove.begin()), comp); + std::set_difference(new_ids.begin(), new_ids.end(), old_ids.begin(), old_ids.end(), std::inserter(servers_to_add, servers_to_add.begin()), comp); + + ConfigUpdateActions result; + + for (auto & [_, server_config] : servers_to_remove) + result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::RemoveServer, server_config}); + + for (auto & [_, server_config] : servers_to_add) + result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::AddServer, server_config}); + + for (const auto & old_server : servers_configuration.cluster_config->get_servers()) + { + for (const auto & new_server : new_servers_configuration.cluster_config->get_servers()) + { + if (old_server->get_id() == new_server->get_id()) + { + if (old_server->get_priority() != new_server->get_priority()) + result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::UpdatePriority, new_server}); + break; + } + } + } + + return result; +} } diff --git a/src/Coordination/KeeperStateManager.h b/src/Coordination/KeeperStateManager.h index 2a93a1dc62b..ccef1fc68c3 100644 --- a/src/Coordination/KeeperStateManager.h +++ b/src/Coordination/KeeperStateManager.h @@ -6,16 +6,42 @@ #include #include // Y_IGNORE #include +#include namespace DB { +using KeeperServerConfigPtr = nuraft::ptr; + +struct KeeperServersConfiguration +{ + int port; + KeeperServerConfigPtr config; + std::unordered_set servers_start_as_followers; + ClusterConfigPtr cluster_config; +}; + +enum class ConfigUpdateActionType +{ + RemoveServer, + AddServer, + UpdatePriority, +}; + +struct ConfigUpdateAction +{ + ConfigUpdateActionType action_type; + KeeperServerConfigPtr server; +}; + +using ConfigUpdateActions = std::vector; + class KeeperStateManager : public nuraft::state_mgr { public: KeeperStateManager( int server_id_, - const std::string & config_prefix, + const std::string & config_prefix_, const Poco::Util::AbstractConfiguration & config, const CoordinationSettingsPtr & coordination_settings, bool standalone_keeper); @@ -30,7 +56,7 @@ public: void flushLogStore(); - nuraft::ptr load_config() override { return cluster_config; } + nuraft::ptr load_config() override { return servers_configuration.cluster_config; } void save_config(const nuraft::cluster_config & config) override; @@ -40,17 +66,17 @@ public: nuraft::ptr load_log_store() override { return log_store; } - Int32 server_id() override { return my_server_id; } + int32_t server_id() override { return my_server_id; } - nuraft::ptr get_srv_config() const { return my_server_config; } + nuraft::ptr get_srv_config() const { return servers_configuration.config; } void system_exit(const int /* exit_code */) override {} - int getPort() const { return my_port; } + int getPort() const { return servers_configuration.port; } bool shouldStartAsFollower() const { - return start_as_follower_servers.count(my_server_id); + return servers_configuration.servers_start_as_followers.count(my_server_id); } bool isSecure() const @@ -60,18 +86,25 @@ public: nuraft::ptr getLogStore() const { return log_store; } - uint64_t getTotalServers() const { return total_servers; } + uint64_t getTotalServers() const { return servers_configuration.cluster_config->get_servers().size(); } + + ClusterConfigPtr getLatestConfigFromLogStore() const; + + void setClusterConfig(const ClusterConfigPtr & cluster_config); + + ConfigUpdateActions getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const; private: int my_server_id; - int my_port; bool secure; - uint64_t total_servers{0}; - std::unordered_set start_as_follower_servers; + std::string config_prefix; + KeeperServersConfiguration servers_configuration; nuraft::ptr log_store; - nuraft::ptr my_server_config; - nuraft::ptr cluster_config; nuraft::ptr server_state; + + Poco::Logger * log; + + KeeperServersConfiguration parseServersConfiguration(const Poco::Util::AbstractConfiguration & config) const; }; } diff --git a/src/Coordination/fuzzers/CMakeLists.txt b/src/Coordination/fuzzers/CMakeLists.txt new file mode 100644 index 00000000000..926162c2fe3 --- /dev/null +++ b/src/Coordination/fuzzers/CMakeLists.txt @@ -0,0 +1,2 @@ +add_executable (request_fuzzer request_fuzzer.cpp) +target_link_libraries (request_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) diff --git a/src/Coordination/fuzzers/request_fuzzer.cpp b/src/Coordination/fuzzers/request_fuzzer.cpp new file mode 100644 index 00000000000..1a7e2902832 --- /dev/null +++ b/src/Coordination/fuzzers/request_fuzzer.cpp @@ -0,0 +1,36 @@ +#include +#include +#include +#include + + +using namespace DB; + +extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) +try +{ + try { + ResponsesQueue queue; + SnapshotsQueue snapshots_queue{1}; + + CoordinationSettingsPtr settings = std::make_shared(); + auto state_machine = std::make_shared(queue, snapshots_queue, "./snapshots", settings); + state_machine->init(); + DB::KeeperLogStore changelog("./logs", settings->rotate_log_storage_interval, true); + changelog.init(state_machine->last_commit_index() + 1, settings->reserved_log_items); + + nuraft::ptr ret = nuraft::buffer::alloc(sizeof(size)); + nuraft::buffer_serializer bs(ret); + bs.put_raw(data, size); + + state_machine->commit(1, *ret); + + } catch (...) { + return 0; + } + return 0; +} +catch (...) +{ + return 1; +} diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 4140194b873..cc0dfdfdea8 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -161,8 +161,8 @@ struct ContextSharedPart ConfigurationPtr zookeeper_config; /// Stores zookeeper configs #if USE_NURAFT - mutable std::mutex keeper_storage_dispatcher_mutex; - mutable std::shared_ptr keeper_storage_dispatcher; + mutable std::mutex keeper_dispatcher_mutex; + mutable std::shared_ptr keeper_dispatcher; #endif mutable std::mutex auxiliary_zookeepers_mutex; mutable std::map auxiliary_zookeepers; /// Map for auxiliary ZooKeeper clients. @@ -1836,16 +1836,16 @@ void Context::setSystemZooKeeperLogAfterInitializationIfNeeded() void Context::initializeKeeperDispatcher() const { #if USE_NURAFT - std::lock_guard lock(shared->keeper_storage_dispatcher_mutex); + std::lock_guard lock(shared->keeper_dispatcher_mutex); - if (shared->keeper_storage_dispatcher) + if (shared->keeper_dispatcher) throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize Keeper multiple times"); const auto & config = getConfigRef(); if (config.has("keeper_server")) { - shared->keeper_storage_dispatcher = std::make_shared(); - shared->keeper_storage_dispatcher->initialize(config, getApplicationType() == ApplicationType::KEEPER); + shared->keeper_dispatcher = std::make_shared(); + shared->keeper_dispatcher->initialize(config, getApplicationType() == ApplicationType::KEEPER); } #endif } @@ -1853,27 +1853,39 @@ void Context::initializeKeeperDispatcher() const #if USE_NURAFT std::shared_ptr & Context::getKeeperDispatcher() const { - std::lock_guard lock(shared->keeper_storage_dispatcher_mutex); - if (!shared->keeper_storage_dispatcher) + 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_storage_dispatcher; + return shared->keeper_dispatcher; } #endif void Context::shutdownKeeperDispatcher() const { #if USE_NURAFT - std::lock_guard lock(shared->keeper_storage_dispatcher_mutex); - if (shared->keeper_storage_dispatcher) + std::lock_guard lock(shared->keeper_dispatcher_mutex); + if (shared->keeper_dispatcher) { - shared->keeper_storage_dispatcher->shutdown(); - shared->keeper_storage_dispatcher.reset(); + shared->keeper_dispatcher->shutdown(); + shared->keeper_dispatcher.reset(); } #endif } +void Context::updateKeeperConfiguration(const Poco::Util::AbstractConfiguration & config) +{ +#if USE_NURAFT + std::lock_guard lock(shared->keeper_dispatcher_mutex); + if (!shared->keeper_dispatcher) + return; + + shared->keeper_dispatcher->updateConfiguration(config); +#endif +} + + zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const { std::lock_guard lock(shared->auxiliary_zookeepers_mutex); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 023151e4768..fc00f910eac 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -671,6 +671,7 @@ public: #endif void initializeKeeperDispatcher() const; void shutdownKeeperDispatcher() const; + void updateKeeperConfiguration(const Poco::Util::AbstractConfiguration & config); /// Set auxiliary zookeepers configuration at server starting or configuration reloading. void reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config); @@ -792,6 +793,7 @@ public: DisksMap getDisksMap() const; void updateStorageConfiguration(const Poco::Util::AbstractConfiguration & config); + /// Provides storage politics schemes StoragePolicyPtr getStoragePolicy(const String & name) const; From 0a838d5926529a46fdcdee12f63278e6cbd7dee9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 18 Oct 2021 18:28:17 +0300 Subject: [PATCH 194/919] Remove fuzzers --- src/Coordination/fuzzers/CMakeLists.txt | 2 -- src/Coordination/fuzzers/request_fuzzer.cpp | 36 --------------------- 2 files changed, 38 deletions(-) delete mode 100644 src/Coordination/fuzzers/CMakeLists.txt delete mode 100644 src/Coordination/fuzzers/request_fuzzer.cpp diff --git a/src/Coordination/fuzzers/CMakeLists.txt b/src/Coordination/fuzzers/CMakeLists.txt deleted file mode 100644 index 926162c2fe3..00000000000 --- a/src/Coordination/fuzzers/CMakeLists.txt +++ /dev/null @@ -1,2 +0,0 @@ -add_executable (request_fuzzer request_fuzzer.cpp) -target_link_libraries (request_fuzzer PRIVATE dbms ${LIB_FUZZING_ENGINE}) diff --git a/src/Coordination/fuzzers/request_fuzzer.cpp b/src/Coordination/fuzzers/request_fuzzer.cpp deleted file mode 100644 index 1a7e2902832..00000000000 --- a/src/Coordination/fuzzers/request_fuzzer.cpp +++ /dev/null @@ -1,36 +0,0 @@ -#include -#include -#include -#include - - -using namespace DB; - -extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) -try -{ - try { - ResponsesQueue queue; - SnapshotsQueue snapshots_queue{1}; - - CoordinationSettingsPtr settings = std::make_shared(); - auto state_machine = std::make_shared(queue, snapshots_queue, "./snapshots", settings); - state_machine->init(); - DB::KeeperLogStore changelog("./logs", settings->rotate_log_storage_interval, true); - changelog.init(state_machine->last_commit_index() + 1, settings->reserved_log_items); - - nuraft::ptr ret = nuraft::buffer::alloc(sizeof(size)); - nuraft::buffer_serializer bs(ret); - bs.put_raw(data, size); - - state_machine->commit(1, *ret); - - } catch (...) { - return 0; - } - return 0; -} -catch (...) -{ - return 1; -} From 1d9cfc04ef8873cbd16a1fe040e2e7f13640777d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 18 Oct 2021 18:27:07 +0300 Subject: [PATCH 195/919] improve performance of aggregation in order of primary key --- src/Processors/Chunk.h | 1 + .../FinishAggregatingInOrderAlgorithm.cpp | 60 ++++++------------- .../FinishAggregatingInOrderAlgorithm.h | 15 ++--- .../FinishAggregatingInOrderTransform.h | 8 +-- src/Processors/Merges/IMergingTransform.cpp | 2 +- src/Processors/Merges/IMergingTransform.h | 2 +- src/Processors/QueryPlan/AggregatingStep.cpp | 30 +++++----- .../AggregatingInOrderTransform.cpp | 2 +- .../Transforms/AggregatingInOrderTransform.h | 6 +- ...gingAggregatedMemoryEfficientTransform.cpp | 7 --- ...ergingAggregatedMemoryEfficientTransform.h | 7 +++ ...1_mergetree_read_in_order_spread.reference | 31 +++++----- 12 files changed, 70 insertions(+), 101 deletions(-) diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index bb8266d6e27..011d719d094 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -82,6 +82,7 @@ public: MutableColumns cloneEmptyColumns() const; const ChunkInfoPtr & getChunkInfo() const { return chunk_info; } + bool hasChunkInfo() const { return chunk_info != nullptr; } void setChunkInfo(ChunkInfoPtr chunk_info_) { chunk_info = std::move(chunk_info_); } UInt64 getNumRows() const { return num_rows; } diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp index 99e3adf893c..f54331d5550 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -25,14 +26,12 @@ FinishAggregatingInOrderAlgorithm::FinishAggregatingInOrderAlgorithm( size_t num_inputs_, AggregatingTransformParamsPtr params_, SortDescription description_, - size_t max_block_size_, - size_t merge_threads_) + size_t max_block_size_) : header(header_) , num_inputs(num_inputs_) , params(params_) , description(std::move(description_)) , max_block_size(max_block_size_) - , pool(merge_threads_) { /// Replace column names in description to positions. for (auto & column_description : description) @@ -60,12 +59,6 @@ void FinishAggregatingInOrderAlgorithm::consume(Input & input, size_t source_num IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge() { - if (finished) - { - auto res = popResult(); - return Status(std::move(res), results.empty()); - } - if (!inputs_to_update.empty()) { Status status(inputs_to_update.back()); @@ -89,13 +82,7 @@ IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge() } if (!best_input) - { - aggregate(); - pool.wait(); - finished = true; - auto res = popResult(); - return Status(std::move(res), results.empty()); - } + return Status(prepareToMerge(), true); /// Chunk at best_input will be aggregated entirely. auto & best_state = states[*best_input]; @@ -126,34 +113,20 @@ IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge() /// Do not merge blocks, if there are too few rows. if (accumulated_rows >= max_block_size) - aggregate(); + status.chunk = prepareToMerge(); - status.chunk = popResult(); return status; } -Chunk FinishAggregatingInOrderAlgorithm::popResult() -{ - std::lock_guard lock(results_mutex); - - if (results.empty()) - return {}; - - auto res = std::move(results.back()); - results.pop_back(); - return res; -} - -void FinishAggregatingInOrderAlgorithm::aggregate() +Chunk FinishAggregatingInOrderAlgorithm::prepareToMerge() { accumulated_rows = 0; - pool.scheduleOrThrowOnError([this, blocks_list = std::move(blocks)]() mutable - { - auto aggregated = params->aggregator.mergeBlocks(blocks_list, false); + auto info = std::make_shared(); + info->chunks = std::make_unique(std::move(chunks)); - std::lock_guard lock(results_mutex); - results.emplace_back(aggregated.getColumns(), aggregated.rows()); - }); + Chunk chunk; + chunk.setChunkInfo(std::move(info)); + return chunk; } void FinishAggregatingInOrderAlgorithm::addToAggregation() @@ -164,22 +137,25 @@ void FinishAggregatingInOrderAlgorithm::addToAggregation() if (!state.isValid() || state.current_row == state.to_row) continue; - if (state.to_row - state.current_row == state.num_rows) + size_t current_rows = state.to_row - state.current_row; + if (current_rows == state.num_rows) { - blocks.emplace_back(header.cloneWithColumns(state.all_columns)); + chunks.emplace_back(state.all_columns, current_rows); } else { Columns new_columns; new_columns.reserve(state.all_columns.size()); for (const auto & column : state.all_columns) - new_columns.emplace_back(column->cut(state.current_row, state.to_row - state.current_row)); + new_columns.emplace_back(column->cut(state.current_row, current_rows)); - blocks.emplace_back(header.cloneWithColumns(new_columns)); + chunks.emplace_back(std::move(new_columns), current_rows); } + chunks.back().setChunkInfo(std::make_shared()); + states[i].current_row = states[i].to_row; - accumulated_rows += blocks.back().rows(); + accumulated_rows += current_rows; if (!states[i].isValid()) inputs_to_update.push_back(i); diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h index 8b83e4e215b..2591e11a211 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h @@ -39,17 +39,15 @@ public: size_t num_inputs_, AggregatingTransformParamsPtr params_, SortDescription description_, - size_t max_block_size_, - size_t merge_threads_); + size_t max_block_size_); void initialize(Inputs inputs) override; void consume(Input & input, size_t source_num) override; Status merge() override; private: - void aggregate(); + Chunk prepareToMerge(); void addToAggregation(); - Chunk popResult(); struct State { @@ -72,17 +70,14 @@ private: AggregatingTransformParamsPtr params; SortDescription description; size_t max_block_size; - ThreadPool pool; - - std::mutex results_mutex; - std::vector results; Inputs current_inputs; + std::vector states; std::vector inputs_to_update; - BlocksList blocks; + + std::vector chunks; size_t accumulated_rows = 0; - bool finished = false; }; } diff --git a/src/Processors/Merges/FinishAggregatingInOrderTransform.h b/src/Processors/Merges/FinishAggregatingInOrderTransform.h index 3a388a36727..5605cea8e39 100644 --- a/src/Processors/Merges/FinishAggregatingInOrderTransform.h +++ b/src/Processors/Merges/FinishAggregatingInOrderTransform.h @@ -17,16 +17,14 @@ public: size_t num_inputs, AggregatingTransformParamsPtr params, SortDescription description, - size_t max_block_size, - size_t merge_threads) + size_t max_block_size) : IMergingTransform( - num_inputs, header, header, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false, + num_inputs, header, {}, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false, header, num_inputs, params, std::move(description), - max_block_size, - merge_threads) + max_block_size) { } diff --git a/src/Processors/Merges/IMergingTransform.cpp b/src/Processors/Merges/IMergingTransform.cpp index cba78390c97..85416673d4f 100644 --- a/src/Processors/Merges/IMergingTransform.cpp +++ b/src/Processors/Merges/IMergingTransform.cpp @@ -132,7 +132,7 @@ IProcessor::Status IMergingTransformBase::prepare() bool is_port_full = !output.canPush(); /// Push if has data. - if (state.output_chunk && !is_port_full) + if ((state.output_chunk || state.output_chunk.hasChunkInfo()) && !is_port_full) output.push(std::move(state.output_chunk)); if (!is_initialized) diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index 8b0a44ae025..1acd4916260 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -107,7 +107,7 @@ public: IMergingAlgorithm::Status status = algorithm.merge(); - if (status.chunk && status.chunk.hasRows()) + if ((status.chunk && status.chunk.hasRows()) || status.chunk.hasChunkInfo()) { // std::cerr << "Got chunk with " << status.chunk.getNumRows() << " rows" << std::endl; state.output_chunk = std::move(status.chunk); diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 4942757cbd8..0a362c4750a 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include @@ -87,24 +88,21 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B aggregating_in_order = collector.detachProcessors(0); - for (auto & column_description : group_by_sort_description) - { - if (!column_description.column_name.empty()) - { - column_description.column_number = pipeline.getHeader().getPositionByName(column_description.column_name); - column_description.column_name.clear(); - } - } - auto transform = std::make_shared( pipeline.getHeader(), pipeline.getNumStreams(), transform_params, group_by_sort_description, - max_block_size, - merge_threads); + max_block_size); pipeline.addTransform(std::move(transform)); + pipeline.resize(merge_threads); + + pipeline.addSimpleTransform([&](const Block &) + { + return std::make_shared(transform_params); + }); + aggregating_sorted = collector.detachProcessors(1); } else @@ -114,14 +112,14 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B return std::make_shared(header, transform_params, group_by_sort_description, max_block_size); }); + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, transform_params); + }); + aggregating_in_order = collector.detachProcessors(0); } - pipeline.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, transform_params); - }); - finalizing = collector.detachProcessors(2); return; } diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 33bc5280422..d6c5491f45c 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -133,7 +133,7 @@ void AggregatingInOrderTransform::consume(Chunk chunk) variants.without_key = nullptr; - /// Arenas cannot be destroyed here, since later, in FinalizingSimpleTransform + /// Arenas cannot be destroyed here, since later, in FinalizeAggregatedTransform /// there will be finalizeChunk(), but even after /// finalizeChunk() we cannot destroy arena, since some memory /// from Arena still in use, so we attach it to the Chunk to diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.h b/src/Processors/Transforms/AggregatingInOrderTransform.h index 10793e885ce..e0cb9b80967 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.h +++ b/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -64,10 +64,10 @@ private: }; -class FinalizingSimpleTransform : public ISimpleTransform +class FinalizeAggregatedTransform : public ISimpleTransform { public: - FinalizingSimpleTransform(Block header, AggregatingTransformParamsPtr params_) + FinalizeAggregatedTransform(Block header, AggregatingTransformParamsPtr params_) : ISimpleTransform({std::move(header)}, {params_->getHeader()}, true) , params(params_) {} @@ -82,7 +82,7 @@ public: } } - String getName() const override { return "FinalizingSimpleTransform"; } + String getName() const override { return "FinalizeAggregatedTransform"; } private: AggregatingTransformParamsPtr params; diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp index df2ea4b03f0..0eec02809e4 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp @@ -12,13 +12,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -struct ChunksToMerge : public ChunkInfo -{ - std::unique_ptr chunks; - Int32 bucket_num = -1; - bool is_overflows = false; -}; - GroupingAggregatedTransform::GroupingAggregatedTransform( const Block & header_, size_t num_inputs_, AggregatingTransformParamsPtr params_) : IProcessor(InputPorts(num_inputs_, header_), { Block() }) diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h index 4367f6fec32..b4a62f8a13e 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h @@ -137,6 +137,13 @@ private: void addChunk(Chunk chunk, size_t from_input); }; +struct ChunksToMerge : public ChunkInfo +{ + std::unique_ptr chunks; + Int32 bucket_num = -1; + bool is_overflows = false; +}; + class Pipe; /// Adds processors to pipe which performs memory efficient merging of partially aggregated data from several sources. diff --git a/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference index cdc595a3c57..b6b8b04907c 100644 --- a/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference +++ b/tests/queries/0_stateless/01551_mergetree_read_in_order_spread.reference @@ -1,17 +1,18 @@ (Expression) -ExpressionTransform +ExpressionTransform × 3 (Aggregating) - FinalizingSimpleTransform - FinishAggregatingInOrderTransform 3 → 1 - AggregatingInOrderTransform × 3 - (Expression) - ExpressionTransform × 3 - (SettingQuotaAndLimits) - (ReadFromMergeTree) - ExpressionTransform × 4 - MergeTreeInOrder 0 → 1 - MergingSortedTransform 2 → 1 - ExpressionTransform × 2 - MergeTreeInOrder × 2 0 → 1 - ExpressionTransform - MergeTreeInOrder 0 → 1 + MergingAggregatedBucketTransform × 3 + Resize 1 → 3 + FinishAggregatingInOrderTransform 3 → 1 + AggregatingInOrderTransform × 3 + (Expression) + ExpressionTransform × 3 + (SettingQuotaAndLimits) + (ReadFromMergeTree) + ExpressionTransform × 4 + MergeTreeInOrder 0 → 1 + MergingSortedTransform 2 → 1 + ExpressionTransform × 2 + MergeTreeInOrder × 2 0 → 1 + ExpressionTransform + MergeTreeInOrder 0 → 1 From d5d481735067173b03ba5e5d3207b49d289034e7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 18 Oct 2021 18:35:11 +0300 Subject: [PATCH 196/919] Add disk proxies back, add setReadUntilPosition to ReadBuffer --- src/Compression/CachedCompressedReadBuffer.h | 2 ++ src/Compression/CompressedReadBufferBase.cpp | 8 +---- src/Compression/CompressedReadBufferBase.h | 6 ---- .../CompressedReadBufferFromFile.h | 1 + src/Disks/DiskRestartProxy.cpp | 15 +-------- src/Disks/ReadBufferFromRemoteFSGather.cpp | 2 +- src/Disks/ReadBufferFromRemoteFSGather.h | 2 +- src/Disks/S3/registerDiskS3.cpp | 32 +++++++++---------- ...chronousReadIndirectBufferFromRemoteFS.cpp | 14 ++++---- ...ynchronousReadIndirectBufferFromRemoteFS.h | 2 +- src/IO/ReadBuffer.h | 2 ++ src/IO/RestartAwareReadBuffer.h | 24 ++++++++++++++ .../MergeTree/MergeTreeReaderStream.cpp | 4 +-- 13 files changed, 60 insertions(+), 54 deletions(-) create mode 100644 src/IO/RestartAwareReadBuffer.h diff --git a/src/Compression/CachedCompressedReadBuffer.h b/src/Compression/CachedCompressedReadBuffer.h index 6eedf66a487..eb45667dc25 100644 --- a/src/Compression/CachedCompressedReadBuffer.h +++ b/src/Compression/CachedCompressedReadBuffer.h @@ -58,6 +58,8 @@ public: profile_callback = profile_callback_; clock_type = clock_type_; } + + void setReadUntilPosition(size_t position) override { file_in->setReadUntilPosition(position); } }; } diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index b0609b9ba26..9e3b22fac74 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -14,6 +14,7 @@ #include #include #include +#include namespace ProfileEvents @@ -108,13 +109,6 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c } -void CompressedReadBufferBase::setRightOffset(size_t offset) -{ - if (auto * async_in = dynamic_cast(compressed_in)) - async_in->setRightOffset(offset); -} - - /// Read compressed data into compressed_buffer. Get size of decompressed data from block header. Checksum if need. /// Returns number of compressed bytes read. size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, size_t & size_compressed_without_checksum, bool always_copy) diff --git a/src/Compression/CompressedReadBufferBase.h b/src/Compression/CompressedReadBufferBase.h index 0f788ec445d..c32a169aecc 100644 --- a/src/Compression/CompressedReadBufferBase.h +++ b/src/Compression/CompressedReadBufferBase.h @@ -60,12 +60,6 @@ public: disable_checksum = true; } - /** - * For asynchronous range reading from remote fs need to update last offset for current task, - * when newer tasks read behind previous task last mark. - */ - void setRightOffset(size_t offset); - public: CompressionCodecPtr codec; }; diff --git a/src/Compression/CompressedReadBufferFromFile.h b/src/Compression/CompressedReadBufferFromFile.h index 125e80a0078..8d9abbb3c6c 100644 --- a/src/Compression/CompressedReadBufferFromFile.h +++ b/src/Compression/CompressedReadBufferFromFile.h @@ -63,6 +63,7 @@ public: file_in.setProfileCallback(profile_callback_, clock_type_); } + void setReadUntilPosition(size_t position) override { file_in.setReadUntilPosition(position); } }; } diff --git a/src/Disks/DiskRestartProxy.cpp b/src/Disks/DiskRestartProxy.cpp index 0d762f6fa9b..a1b2ef4e9f7 100644 --- a/src/Disks/DiskRestartProxy.cpp +++ b/src/Disks/DiskRestartProxy.cpp @@ -1,7 +1,7 @@ #include "DiskRestartProxy.h" -#include #include +#include namespace DB { @@ -13,19 +13,6 @@ namespace ErrorCodes using Millis = std::chrono::milliseconds; using Seconds = std::chrono::seconds; -/// Holds restart read lock till buffer destruction. -class RestartAwareReadBuffer : public ReadBufferFromFileDecorator -{ -public: - RestartAwareReadBuffer(const DiskRestartProxy & disk, std::unique_ptr impl_) - : ReadBufferFromFileDecorator(std::move(impl_)), lock(disk.mutex) { } - - void prefetch() override { impl->prefetch(); } - -private: - ReadLock lock; -}; - /// Holds restart read lock till buffer finalize. class RestartAwareWriteBuffer : public WriteBufferFromFileDecorator { diff --git a/src/Disks/ReadBufferFromRemoteFSGather.cpp b/src/Disks/ReadBufferFromRemoteFSGather.cpp index 500d6b651b1..f8246f41340 100644 --- a/src/Disks/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/ReadBufferFromRemoteFSGather.cpp @@ -166,7 +166,7 @@ void ReadBufferFromRemoteFSGather::seek(off_t offset) } -void ReadBufferFromRemoteFSGather::setRightOffset(size_t offset) +void ReadBufferFromRemoteFSGather::setReadUntilPosition(size_t offset) { assert(last_offset < offset); current_buf.reset(); diff --git a/src/Disks/ReadBufferFromRemoteFSGather.h b/src/Disks/ReadBufferFromRemoteFSGather.h index 3285d7190d2..70d3adc8fb1 100644 --- a/src/Disks/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/ReadBufferFromRemoteFSGather.h @@ -32,7 +32,7 @@ public: void seek(off_t offset); /// SEEK_SET only. - void setRightOffset(size_t offset); + void setReadUntilPosition(size_t position) override; size_t readInto(char * data, size_t size, size_t offset, size_t ignore = 0); diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index 1db9c3938c6..5eabbfff5f8 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -200,27 +200,27 @@ void registerDiskS3(DiskFactory & factory) s3disk->startup(); - // bool cache_enabled = config.getBool(config_prefix + ".cache_enabled", true); + bool cache_enabled = config.getBool(config_prefix + ".cache_enabled", true); - // if (cache_enabled) - // { - // String cache_path = config.getString(config_prefix + ".cache_path", context->getPath() + "disks/" + name + "/cache/"); + if (cache_enabled) + { + String cache_path = config.getString(config_prefix + ".cache_path", context->getPath() + "disks/" + name + "/cache/"); - // if (metadata_path == cache_path) - // throw Exception("Metadata and cache path should be different: " + metadata_path, ErrorCodes::BAD_ARGUMENTS); + if (metadata_path == cache_path) + throw Exception("Metadata and cache path should be different: " + metadata_path, ErrorCodes::BAD_ARGUMENTS); - // auto cache_disk = std::make_shared("s3-cache", cache_path, 0); - // auto cache_file_predicate = [] (const String & path) - // { - // return path.ends_with("idx") // index files. - // || path.ends_with("mrk") || path.ends_with("mrk2") || path.ends_with("mrk3") // mark files. - // || path.ends_with("txt") || path.ends_with("dat"); - // }; + auto cache_disk = std::make_shared("s3-cache", cache_path, 0); + auto cache_file_predicate = [] (const String & path) + { + return path.ends_with("idx") // index files. + || path.ends_with("mrk") || path.ends_with("mrk2") || path.ends_with("mrk3") // mark files. + || path.ends_with("txt") || path.ends_with("dat"); + }; - // s3disk = std::make_shared(s3disk, cache_disk, cache_file_predicate); - // } + s3disk = std::make_shared(s3disk, cache_disk, cache_file_predicate); + } - return s3disk; + return std::make_shared(s3disk); }; factory.registerDiskType("s3", creator); } diff --git a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp index ce836082130..08beed079e8 100644 --- a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -78,8 +78,10 @@ void AsynchronousReadIndirectBufferFromRemoteFS::prefetch() return; if (absolute_position > last_offset) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Read beyond last offset ({} > {})", - absolute_position, last_offset); + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Read beyond last offset ({} > {}) {}", + absolute_position, last_offset, buffer_events); + } /// Prefetch even in case hasPendingData() == true. prefetch_future = readInto(prefetch_buffer.data(), prefetch_buffer.size()); @@ -90,12 +92,12 @@ void AsynchronousReadIndirectBufferFromRemoteFS::prefetch() } -void AsynchronousReadIndirectBufferFromRemoteFS::setRightOffset(size_t offset) +void AsynchronousReadIndirectBufferFromRemoteFS::setReadUntilPosition(size_t offset) { buffer_events += "-- Set last offset " + toString(offset) + "--"; if (prefetch_future.valid()) { - std::cerr << buffer_events << std::endl; + LOG_DEBUG(&Poco::Logger::get("kssenii"), buffer_events); /// TODO: Planning to put logical error here after more testing, // because seems like future is never supposed to be valid at this point. std::terminate(); @@ -107,7 +109,7 @@ void AsynchronousReadIndirectBufferFromRemoteFS::setRightOffset(size_t offset) } last_offset = offset; - impl->setRightOffset(offset); + impl->setReadUntilPosition(offset); } @@ -250,7 +252,7 @@ void AsynchronousReadIndirectBufferFromRemoteFS::finalize() prefetch_future.wait(); prefetch_future = {}; } - std::cerr << "Buffer events: " << buffer_events << std::endl; + LOG_DEBUG(&Poco::Logger::get("kssenii"), buffer_events); } diff --git a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h index 2dcd89016ed..f1cfbfde8f6 100644 --- a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -47,7 +47,7 @@ public: void prefetch() override; - void setRightOffset(size_t offset); + void setReadUntilPosition(size_t position) override; private: bool nextImpl() override; diff --git a/src/IO/ReadBuffer.h b/src/IO/ReadBuffer.h index 609065feb7e..29c1c4ce43a 100644 --- a/src/IO/ReadBuffer.h +++ b/src/IO/ReadBuffer.h @@ -202,6 +202,8 @@ public: */ virtual void prefetch() {} + virtual void setReadUntilPosition(size_t /* position */) {} + protected: /// The number of bytes to ignore from the initial position of `working_buffer` /// buffer. Apparently this is an additional out-parameter for nextImpl(), diff --git a/src/IO/RestartAwareReadBuffer.h b/src/IO/RestartAwareReadBuffer.h new file mode 100644 index 00000000000..c8e871c8460 --- /dev/null +++ b/src/IO/RestartAwareReadBuffer.h @@ -0,0 +1,24 @@ +#include +#include + +namespace DB +{ + +using ReadLock = std::shared_lock; + +/// Holds restart read lock till buffer destruction. +class RestartAwareReadBuffer : public ReadBufferFromFileDecorator +{ +public: + RestartAwareReadBuffer(const DiskRestartProxy & disk, std::unique_ptr impl_) + : ReadBufferFromFileDecorator(std::move(impl_)), lock(disk.mutex) { } + + void prefetch() override { impl->prefetch(); } + + void setReadUntilPosition(size_t position) override { impl->setReadUntilPosition(position); } + +private: + ReadLock lock; +}; + +} diff --git a/src/Storages/MergeTree/MergeTreeReaderStream.cpp b/src/Storages/MergeTree/MergeTreeReaderStream.cpp index 707a8c85c73..0c4e14d0ae8 100644 --- a/src/Storages/MergeTree/MergeTreeReaderStream.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderStream.cpp @@ -187,9 +187,9 @@ void MergeTreeReaderStream::adjustForRange(size_t left_mark, size_t right_mark) { last_right_offset = right_offset; if (cached_buffer) - cached_buffer->setRightOffset(last_right_offset); + cached_buffer->setReadUntilPosition(last_right_offset); if (non_cached_buffer) - non_cached_buffer->setRightOffset(last_right_offset); + non_cached_buffer->setReadUntilPosition(last_right_offset); } } From ab0d2504e57682711ae22ec83d3df642b1f0455a Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 18 Oct 2021 15:44:54 +0000 Subject: [PATCH 197/919] fix --- src/Storages/FileLog/StorageFileLog.cpp | 15 +++++++++------ src/Storages/FileLog/StorageFileLog.h | 6 ++++-- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 92c30a86b49..7f4522841de 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -327,6 +327,7 @@ Pipe StorageFileLog::read( throw Exception("Another select query is running on this table, need to wait it finish.", ErrorCodes::CANNOT_SELECT); } + std::lock_guard lock(file_infos_mutex); updateFileInfos(); /// No files to parse @@ -578,16 +579,17 @@ size_t StorageFileLog::getTableDependentCount() const void StorageFileLog::threadFunc() { + bool reschedule = false; try { - updateFileInfos(); - + std::lock_guard lock(file_infos_mutex); auto table_id = getStorageID(); auto dependencies_count = getTableDependentCount(); if (dependencies_count) { + has_dependent_mv = true; auto start_time = std::chrono::steady_clock::now(); // Keep streaming as long as there are attached views and streaming is not cancelled @@ -616,6 +618,7 @@ void StorageFileLog::threadFunc() if (duration.count() > MAX_THREAD_WORK_DURATION_MS) { LOG_TRACE(log, "Thread work duration limit exceeded. Reschedule."); + reschedule = true; break; } } @@ -631,7 +634,7 @@ void StorageFileLog::threadFunc() { if (path_is_directory) { - if (!getTableDependentCount()) + if (!getTableDependentCount() || reschedule) task->holder->scheduleAfter(milliseconds_to_wait); else { @@ -654,9 +657,9 @@ bool StorageFileLog::streamToViews() { if (running_streams) { - throw Exception("Another select query is running on this table, need to wait it finish.", ErrorCodes::CANNOT_SELECT); + LOG_INFO(log, "Another select query is running on this table, need to wait it finish."); + return true; } - has_dependent_mv = true; Stopwatch watch; auto table_id = getStorageID(); @@ -670,7 +673,7 @@ bool StorageFileLog::streamToViews() if (max_streams_number == 0) { LOG_INFO(log, "There is a idle table named {}, no files need to parse.", getName()); - return false; + return updateFileInfos(); } // Create an INSERT query for streaming data diff --git a/src/Storages/FileLog/StorageFileLog.h b/src/Storages/FileLog/StorageFileLog.h index f0aac1a7985..e1d95ae736f 100644 --- a/src/Storages/FileLog/StorageFileLog.h +++ b/src/Storages/FileLog/StorageFileLog.h @@ -166,12 +166,14 @@ private: /// later select should forbid to execute. std::atomic running_streams = 0; - std::atomic has_dependent_mv = false; - std::mutex mutex; bool has_new_events = false; std::condition_variable cv; + bool has_dependent_mv = false; + + std::mutex file_infos_mutex; + struct TaskContext { BackgroundSchedulePool::TaskHolder holder; From deb696fd27e8a9f7e3f931bf7c0e76abacd410f9 Mon Sep 17 00:00:00 2001 From: Alexey Date: Mon, 18 Oct 2021 16:07:19 +0000 Subject: [PATCH 198/919] en draft --- docs/en/operations/settings/settings.md | 38 +++++++++++++++++++ .../statements/describe-table.md | 4 ++ 2 files changed, 42 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2b088bf45bf..2d3f0b812b4 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3814,3 +3814,41 @@ Default value: `0`. **See Also** - [optimize_move_to_prewhere](#optimize_move_to_prewhere) setting + +## describe_include_subcolumns {#describe_include_subcolumns} + +Enables describing subcolumns for a [DESCRIBE](../../sql-reference/statements/describe-table.md) query. Subcolumns appear if [Tuple](../../sql-reference/data-types/tuple.md) or [Map](../../sql-reference/data-types/map.md) data types are used. + +Possible values: + +- 0 — Disabled. +- 1 — Enabled. + +Default value: `0`. + +**EXAMPLE** + +Query: + +``` sql +CREATE TABLE describe_example ( + id UInt64, user Tuple (name String, age UInt8), dict Map(String, UInt32) +) ENGINE = MergeTree() ORDER BY id; + +DESCRIBE TABLE describe_example SETTINGS describe_include_subcolumns=1; +``` + +Result: + +``` text +┌─name────────┬─type──────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┐ +│ id │ UInt64 │ │ │ │ │ │ 0 │ +│ user │ Tuple(name String, age UInt8) │ │ │ │ │ │ 0 │ +│ dict │ Map(String, UInt32) │ │ │ │ │ │ 0 │ +│ user.name │ String │ │ │ │ │ │ 1 │ +│ user.age │ UInt8 │ │ │ │ │ │ 1 │ +│ dict.size0 │ UInt64 │ │ │ │ │ │ 1 │ +│ dict.keys │ Array(String) │ │ │ │ │ │ 1 │ +│ dict.values │ Array(UInt32) │ │ │ │ │ │ 1 │ +└─────────────┴───────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┴──────────────┘ +``` diff --git a/docs/en/sql-reference/statements/describe-table.md b/docs/en/sql-reference/statements/describe-table.md index bc197bf0f72..3d71f8e875b 100644 --- a/docs/en/sql-reference/statements/describe-table.md +++ b/docs/en/sql-reference/statements/describe-table.md @@ -18,3 +18,7 @@ Returns the following `String` type columns: - `comment_expression` — Comment text. Nested data structures are output in “expanded” format. Each column is shown separately, with the name after a dot. + +**See Also** + +- [describe_include_subcolumns](../../operations/settings/settings.md#describe_include_subcolumns) setting. \ No newline at end of file From ef3597f0543b8ed1b02285934654a6bcc4daf635 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 18 Oct 2021 19:21:49 +0300 Subject: [PATCH 199/919] Fix build. --- .../Optimizations/filterPushDown.cpp | 10 ++----- .../QueryPlan/Optimizations/limitPushDown.cpp | 28 +++---------------- 2 files changed, 6 insertions(+), 32 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 687946659b6..e81cec723a1 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -7,10 +7,7 @@ #include #include #include -#include -#include -#include -#include +#include #include #include #include @@ -237,10 +234,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes // { // } - if (typeid_cast(child.get()) - || typeid_cast(child.get()) - || typeid_cast(child.get()) - || typeid_cast(child.get())) + if (typeid_cast(child.get())) { Names allowed_inputs = child->getOutputStream().header.getNames(); if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, allowed_inputs)) diff --git a/src/Processors/QueryPlan/Optimizations/limitPushDown.cpp b/src/Processors/QueryPlan/Optimizations/limitPushDown.cpp index 01af6a2bbde..eb65f49103b 100644 --- a/src/Processors/QueryPlan/Optimizations/limitPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/limitPushDown.cpp @@ -2,10 +2,7 @@ #include #include #include -#include -#include -#include -#include +#include #include namespace DB::QueryPlanOptimizations @@ -21,32 +18,15 @@ static bool tryUpdateLimitForSortingSteps(QueryPlan::Node * node, size_t limit) QueryPlan::Node * child = nullptr; bool updated = false; - if (auto * merging_sorted = typeid_cast(step.get())) + if (auto * sorting = typeid_cast(step.get())) { /// TODO: remove LimitStep here. - merging_sorted->updateLimit(limit); + sorting->updateLimit(limit); updated = true; child = node->children.front(); } - else if (auto * finish_sorting = typeid_cast(step.get())) - { - /// TODO: remove LimitStep here. - finish_sorting->updateLimit(limit); - updated = true; - } - else if (auto * merge_sorting = typeid_cast(step.get())) - { - merge_sorting->updateLimit(limit); - updated = true; - child = node->children.front(); - } - else if (auto * partial_sorting = typeid_cast(step.get())) - { - partial_sorting->updateLimit(limit); - updated = true; - } - /// We often have chain PartialSorting -> MergeSorting -> MergingSorted + /// In case we have several sorting steps. /// Try update limit for them also if possible. if (child) tryUpdateLimitForSortingSteps(child, limit); From 3e742524b45204a812a9fe76d3ad5afbd681b9fd Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 18 Oct 2021 19:15:46 +0300 Subject: [PATCH 200/919] update comments --- .../FinishAggregatingInOrderAlgorithm.h | 17 ++++++++++------- src/Processors/QueryPlan/AggregatingStep.cpp | 9 +++++++++ 2 files changed, 19 insertions(+), 7 deletions(-) diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h index 2591e11a211..209bb9e4045 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h @@ -16,20 +16,23 @@ using AggregatingTransformParamsPtr = std::shared_ptr 1) { + /** The pipeline is the following: + * + * --> AggregatingInOrder --> MergingAggregatedBucket + * --> AggregatingInOrder --> FinishAggregatingInOrder --> ResizeProcessor --> MergingAggregatedBucket + * --> AggregatingInOrder --> MergingAggregatedBucket + */ + auto many_data = std::make_shared(pipeline.getNumStreams()); size_t counter = 0; pipeline.addSimpleTransform([&](const Block & header) @@ -96,6 +103,8 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B max_block_size); pipeline.addTransform(std::move(transform)); + + /// Do merge of aggregated data in parallel. pipeline.resize(merge_threads); pipeline.addSimpleTransform([&](const Block &) From 34d46245c3987a726c37bd3ed13c54f06ae99c83 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 18 Oct 2021 20:53:30 +0300 Subject: [PATCH 201/919] Try fix some tests. --- src/Interpreters/InterpreterSelectQuery.cpp | 1 + .../QueryPlan/FinishSortingStep.cpp | 115 ------------------ src/Processors/QueryPlan/FinishSortingStep.h | 36 ------ 3 files changed, 1 insertion(+), 151 deletions(-) delete mode 100644 src/Processors/QueryPlan/FinishSortingStep.cpp delete mode 100644 src/Processors/QueryPlan/FinishSortingStep.h diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c6ef18d7660..494497fce9a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2329,6 +2329,7 @@ void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfo settings.min_free_disk_space_for_temporary_data); sorting_step->setStepDescription("Sorting for ORDER BY"); + query_plan.addStep(std::move(sorting_step)); } diff --git a/src/Processors/QueryPlan/FinishSortingStep.cpp b/src/Processors/QueryPlan/FinishSortingStep.cpp deleted file mode 100644 index 9002c804e7b..00000000000 --- a/src/Processors/QueryPlan/FinishSortingStep.cpp +++ /dev/null @@ -1,115 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -static ITransformingStep::Traits getTraits(size_t limit) -{ - return ITransformingStep::Traits - { - { - .preserves_distinct_columns = true, - .returns_single_stream = true, - .preserves_number_of_streams = false, - .preserves_sorting = false, - }, - { - .preserves_number_of_rows = limit == 0, - } - }; -} - -FinishSortingStep::FinishSortingStep( - const DataStream & input_stream_, - SortDescription prefix_description_, - SortDescription result_description_, - size_t max_block_size_, - UInt64 limit_) - : ITransformingStep(input_stream_, input_stream_.header, getTraits(limit_)) - , prefix_description(std::move(prefix_description_)) - , result_description(std::move(result_description_)) - , max_block_size(max_block_size_) - , limit(limit_) -{ - /// TODO: check input_stream is sorted by prefix_description. - output_stream->sort_description = result_description; - output_stream->sort_mode = DataStream::SortMode::Stream; -} - -void FinishSortingStep::updateLimit(size_t limit_) -{ - if (limit_ && (limit == 0 || limit_ < limit)) - { - limit = limit_; - transform_traits.preserves_number_of_rows = false; - } -} - -void FinishSortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - bool need_finish_sorting = (prefix_description.size() < result_description.size()); - if (pipeline.getNumStreams() > 1) - { - UInt64 limit_for_merging = (need_finish_sorting ? 0 : limit); - auto transform = std::make_shared( - pipeline.getHeader(), - pipeline.getNumStreams(), - prefix_description, - max_block_size, - limit_for_merging); - - pipeline.addTransform(std::move(transform)); - } - - if (need_finish_sorting) - { - pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr - { - if (stream_type != QueryPipelineBuilder::StreamType::Main) - return nullptr; - - return std::make_shared(header, result_description, limit); - }); - - /// NOTE limits are not applied to the size of temporary sets in FinishSortingTransform - pipeline.addSimpleTransform([&](const Block & header) -> ProcessorPtr - { - return std::make_shared( - header, prefix_description, result_description, max_block_size, limit); - }); - } -} - -void FinishSortingStep::describeActions(FormatSettings & settings) const -{ - String prefix(settings.offset, ' '); - - settings.out << prefix << "Prefix sort description: "; - dumpSortDescription(prefix_description, input_streams.front().header, settings.out); - settings.out << '\n'; - - settings.out << prefix << "Result sort description: "; - dumpSortDescription(result_description, input_streams.front().header, settings.out); - settings.out << '\n'; - - if (limit) - settings.out << prefix << "Limit " << limit << '\n'; -} - -void FinishSortingStep::describeActions(JSONBuilder::JSONMap & map) const -{ - map.add("Prefix Sort Description", explainSortDescription(prefix_description, input_streams.front().header)); - map.add("Result Sort Description", explainSortDescription(result_description, input_streams.front().header)); - - if (limit) - map.add("Limit", limit); -} - -} diff --git a/src/Processors/QueryPlan/FinishSortingStep.h b/src/Processors/QueryPlan/FinishSortingStep.h deleted file mode 100644 index fd56c4353e7..00000000000 --- a/src/Processors/QueryPlan/FinishSortingStep.h +++ /dev/null @@ -1,36 +0,0 @@ -#pragma once -#include -#include - -namespace DB -{ - -/// Finish sorting of pre-sorted data. See FinishSortingTransform. -class FinishSortingStep : public ITransformingStep -{ -public: - FinishSortingStep( - const DataStream & input_stream_, - SortDescription prefix_description_, - SortDescription result_description_, - size_t max_block_size_, - UInt64 limit_); - - String getName() const override { return "FinishSorting"; } - - void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - - void describeActions(JSONBuilder::JSONMap & map) const override; - void describeActions(FormatSettings & settings) const override; - - /// Add limit or change it to lower value. - void updateLimit(size_t limit_); - -private: - SortDescription prefix_description; - SortDescription result_description; - size_t max_block_size; - UInt64 limit; -}; - -} From 89fe606d15f94b95b9633525a9dc0e4d052610ff Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 18 Oct 2021 23:16:02 +0300 Subject: [PATCH 202/919] try fix 'some fetches may stuck' --- src/Storages/MergeTree/MergeTreePartInfo.h | 3 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 58 +++-- .../MergeTree/ReplicatedMergeTreeQueue.h | 14 +- .../ReplicatedMergeTreeRestartingThread.cpp | 2 + src/Storages/StorageReplicatedMergeTree.cpp | 216 +++++++++++++----- 5 files changed, 215 insertions(+), 78 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreePartInfo.h b/src/Storages/MergeTree/MergeTreePartInfo.h index 82fe0b860c8..b4c5687209b 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/src/Storages/MergeTree/MergeTreePartInfo.h @@ -139,7 +139,8 @@ struct DetachedPartInfo : public MergeTreePartInfo "clone", "attaching", "deleting", - "tmp-fetch" + "tmp-fetch", + "covered-by-broken", }); /// NOTE: It may parse part info incorrectly. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index f397e73aa31..6c1cd04d4ae 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -55,6 +55,15 @@ void ReplicatedMergeTreeQueue::clear() mutation_pointer.clear(); } +void ReplicatedMergeTreeQueue::setBrokenPartsToEnqueueFetchesOnLoading(Strings && parts_to_fetch) +{ + std::lock_guard lock(state_mutex); + /// Can be called only before queue initialization + assert(broken_parts_to_enqueue_fetches_on_loading.empty()); + assert(virtual_parts.size() == 0); + broken_parts_to_enqueue_fetches_on_loading = std::move(parts_to_fetch); +} + void ReplicatedMergeTreeQueue::initialize(zkutil::ZooKeeperPtr zookeeper) { std::lock_guard lock(state_mutex); @@ -176,6 +185,19 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper) } +void ReplicatedMergeTreeQueue::createLogEntriesToFetchBrokenParts() +{ + std::lock_guard lock(state_mutex); + if (broken_parts_to_enqueue_fetches_on_loading.empty()) + return; + + for (const auto & broken_part_name : broken_parts_to_enqueue_fetches_on_loading) + storage.removePartAndEnqueueFetch(broken_part_name); + + broken_parts_to_enqueue_fetches_on_loading.clear(); +} + + void ReplicatedMergeTreeQueue::insertUnlocked( const LogEntryPtr & entry, std::optional & min_unprocessed_insert_time_changed, std::lock_guard & state_lock) @@ -982,7 +1004,7 @@ bool ReplicatedMergeTreeQueue::checkReplaceRangeCanBeRemoved(const MergeTreePart void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( zkutil::ZooKeeperPtr zookeeper, const MergeTreePartInfo & part_info, - const ReplicatedMergeTreeLogEntryData & current) + const std::optional & current) { /// TODO is it possible to simplify it? Queue to_wait; @@ -993,19 +1015,21 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( /// Remove operations with parts, contained in the range to be deleted, from the queue. std::unique_lock lock(state_mutex); - [[maybe_unused]] bool called_from_alter_query_directly = current.replace_range_entry && current.replace_range_entry->columns_version < 0; - assert(currently_executing_drop_or_replace_range || called_from_alter_query_directly); + [[maybe_unused]] bool called_from_alter_query_directly = current && current->replace_range_entry && current->replace_range_entry->columns_version < 0; + [[maybe_unused]] bool called_for_broken_part = !current; + assert(currently_executing_drop_or_replace_range || called_from_alter_query_directly || called_for_broken_part); for (Queue::iterator it = queue.begin(); it != queue.end();) { auto type = (*it)->type; - bool is_simple_producing_op = type == LogEntry::GET_PART || type == LogEntry::ATTACH_PART || type == LogEntry::MERGE_PARTS || type == LogEntry::MUTATE_PART; + bool simple_op_covered = is_simple_producing_op && part_info.contains(MergeTreePartInfo::fromPartName((*it)->new_part_name, format_version)); - if (simple_op_covered || checkReplaceRangeCanBeRemoved(part_info, *it, current)) + bool replace_range_covered = current && checkReplaceRangeCanBeRemoved(part_info, *it, *current); + if (simple_op_covered || replace_range_covered) { if ((*it)->currently_executing) to_wait.push_back(*it); @@ -1035,16 +1059,20 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( } -bool ReplicatedMergeTreeQueue::isNotCoveredByFuturePartsImpl(const String & log_entry_name, const String & new_part_name, +bool ReplicatedMergeTreeQueue::isNotCoveredByFuturePartsImpl(const LogEntry & entry, const String & new_part_name, String & out_reason, std::lock_guard & /* queue_lock */) const { /// Let's check if the same part is now being created by another action. - if (future_parts.count(new_part_name)) + auto entry_for_same_part_it = future_parts.find(new_part_name); + if (entry_for_same_part_it != future_parts.end()) { - const char * format_str = "Not executing log entry {} for part {} " - "because another log entry for the same part is being processed. This shouldn't happen often."; - LOG_INFO(log, format_str, log_entry_name, new_part_name); - out_reason = fmt::format(format_str, log_entry_name, new_part_name); + const LogEntry & another_entry = *entry_for_same_part_it->second; + const char * format_str = "Not executing log entry {} of type {} for part {} " + "because another log entry {} of type {} for the same part ({}) is being processed. This shouldn't happen often."; + LOG_INFO(log, format_str, entry.znode_name, entry.type, entry.new_part_name, + another_entry.znode_name, another_entry.type, another_entry.new_part_name); + out_reason = fmt::format(format_str, entry.znode_name, entry.type, entry.new_part_name, + another_entry.znode_name, another_entry.type, another_entry.new_part_name); return false; /** When the corresponding action is completed, then `isNotCoveredByFuturePart` next time, will succeed, @@ -1067,8 +1095,8 @@ bool ReplicatedMergeTreeQueue::isNotCoveredByFuturePartsImpl(const String & log_ { const char * format_str = "Not executing log entry {} for part {} " "because it is covered by part {} that is currently executing."; - LOG_TRACE(log, format_str, log_entry_name, new_part_name, future_part_elem.first); - out_reason = fmt::format(format_str, log_entry_name, new_part_name, future_part_elem.first); + LOG_TRACE(log, format_str, entry.znode_name, new_part_name, future_part_elem.first); + out_reason = fmt::format(format_str, entry.znode_name, new_part_name, future_part_elem.first); return false; } } @@ -1101,7 +1129,7 @@ bool ReplicatedMergeTreeQueue::addFuturePartIfNotCoveredByThem(const String & pa if (drop_ranges.isAffectedByDropRange(part_name, reject_reason)) return false; - if (isNotCoveredByFuturePartsImpl(entry.znode_name, part_name, reject_reason, lock)) + if (isNotCoveredByFuturePartsImpl(entry, part_name, reject_reason, lock)) { CurrentlyExecuting::setActualPartName(entry, part_name, *this); return true; @@ -1122,7 +1150,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( /// some other entry which is currently executing, then we can postpone this entry. for (const String & new_part_name : entry.getVirtualPartNames(format_version)) { - if (!isNotCoveredByFuturePartsImpl(entry.znode_name, new_part_name, out_postpone_reason, state_lock)) + if (!isNotCoveredByFuturePartsImpl(entry, new_part_name, out_postpone_reason, state_lock)) return false; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 30c8bafcf6e..c059ff7a072 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -156,6 +156,8 @@ private: /// We need it because alters have to be executed sequentially (one by one). ReplicatedMergeTreeAltersSequence alter_sequence; + Strings broken_parts_to_enqueue_fetches_on_loading; + /// List of subscribers /// A subscriber callback is called when an entry queue is deleted mutable std::mutex subscribers_mutex; @@ -208,7 +210,7 @@ private: * Should be called under state_mutex. */ bool isNotCoveredByFuturePartsImpl( - const String & log_entry_name, + const LogEntry & entry, const String & new_part_name, String & out_reason, std::lock_guard & state_lock) const; @@ -324,7 +326,8 @@ public: /** Remove the action from the queue with the parts covered by part_name (from ZK and from the RAM). * And also wait for the completion of their execution, if they are now being executed. */ - void removePartProducingOpsInRange(zkutil::ZooKeeperPtr zookeeper, const MergeTreePartInfo & part_info, const ReplicatedMergeTreeLogEntryData & current); + void removePartProducingOpsInRange(zkutil::ZooKeeperPtr zookeeper, const MergeTreePartInfo & part_info, + const std::optional & current); /** In the case where there are not enough parts to perform the merge in part_name * - move actions with merged parts to the end of the queue @@ -455,6 +458,13 @@ public: /// It's needed because queue itself can trigger it's task handler and in /// this case race condition is possible. QueueLocks lockQueue(); + + /// Can be called only on data parts loading. + /// We need loaded queue to create GET_PART entry for broken (or missing) part, + /// but queue is not loaded yet on data parts loading. + void setBrokenPartsToEnqueueFetchesOnLoading(Strings && parts_to_fetch); + /// Must be called right after queue loading. + void createLogEntriesToFetchBrokenParts(); }; class ReplicatedMergeTreeMergePredicate diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 16597fc4be9..c29ae873c5c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -178,6 +178,8 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup() storage.queue.load(zookeeper); + storage.queue.createLogEntriesToFetchBrokenParts(); + /// pullLogsToQueue() after we mark replica 'is_active' (and after we repair if it was lost); /// because cleanup_thread doesn't delete log_pointer of active replicas. storage.queue.pullLogsToQueue(zookeeper, {}, ReplicatedMergeTreeQueue::LOAD); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 416d37cd351..6980d523ed5 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1211,44 +1211,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) } /// Add to the queue jobs to pick up the missing parts from other replicas and remove from ZK the information that we have them. - std::vector> exists_futures; - exists_futures.reserve(parts_to_fetch.size()); - for (const String & part_name : parts_to_fetch) - { - String part_path = fs::path(replica_path) / "parts" / part_name; - exists_futures.emplace_back(zookeeper->asyncExists(part_path)); - } - - std::vector> enqueue_futures; - enqueue_futures.reserve(parts_to_fetch.size()); - for (size_t i = 0; i < parts_to_fetch.size(); ++i) - { - const String & part_name = parts_to_fetch[i]; - - Coordination::Requests ops; - - LOG_ERROR(log, "Removing locally missing part from ZooKeeper and queueing a fetch: {}", part_name); - time_t part_create_time = 0; - Coordination::ExistsResponse exists_resp = exists_futures[i].get(); - if (exists_resp.error == Coordination::Error::ZOK) - { - part_create_time = exists_resp.stat.ctime / 1000; - removePartFromZooKeeper(part_name, ops, exists_resp.stat.numChildren > 0); - } - LogEntry log_entry; - log_entry.type = LogEntry::GET_PART; - log_entry.source_replica = ""; - log_entry.new_part_name = part_name; - log_entry.create_time = part_create_time; - - /// We assume that this occurs before the queue is loaded (queue.initialize). - ops.emplace_back(zkutil::makeCreateRequest( - fs::path(replica_path) / "queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); - enqueue_futures.emplace_back(zookeeper->asyncMulti(ops)); - } - - for (auto & future : enqueue_futures) - future.get(); + queue.setBrokenPartsToEnqueueFetchesOnLoading(std::move(parts_to_fetch)); /// Remove extra local parts. for (const DataPartPtr & part : unexpected_parts) @@ -2343,13 +2306,57 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo std::sort(source_queue_names.begin(), source_queue_names.end()); - Strings source_queue; - for (const String & entry_name : source_queue_names) + struct QueueEntryInfo { - String entry; - if (!zookeeper->tryGet(fs::path(source_path) / "queue" / entry_name, entry)) - continue; - source_queue.push_back(entry); + String data; + Coordination::Stat stat; + LogEntryPtr parsed_entry; + bool need_copy = true; + }; + + std::vector source_queue; + ActiveDataPartSet get_part_set{format_version}; + ActiveDataPartSet drop_range_set{format_version}; + + { + std::vector queue_get_futures; + source_queue.resize(source_queue_names.size()); + queue_get_futures.reserve(source_queue_names.size()); + + for (const String & entry_name : source_queue_names) + queue_get_futures.push_back(zookeeper->asyncTryGet(fs::path(source_path) / "queue" / entry_name)); + for (size_t i = 0; i < source_queue_names.size(); ++i) + { + auto res = queue_get_futures[i].get(); + /// It's ok if entry is already executed and removed: we also will get source parts set. + if (res.error == Coordination::Error::ZNONODE) + continue; + + assert(res.error == Coordination::Error::ZOK); + auto & info = source_queue[i]; + info.data = std::move(res.data); + info.stat = std::move(res.stat); + try + { + info.parsed_entry = LogEntry::parse(info.data, info.stat); + } + catch (...) + { + tryLogCurrentException(log, "Cannot parse source queue entry " + source_queue_names[i]); + } + + /// It may be ok if source replica has newer version. We will copy entry as is. + if (!info.parsed_entry) + continue; + + info.parsed_entry->znode_name = source_queue_names[i]; + + if (info.parsed_entry->type == LogEntry::GET_PART) + get_part_set.add(info.parsed_entry->new_part_name); + + if (info.parsed_entry->type == LogEntry::DROP_RANGE) + drop_range_set.add(info.parsed_entry->new_part_name); + } } /// We should do it after copying queue, because some ALTER_METADATA entries can be lost otherwise. @@ -2357,9 +2364,10 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo /// Add to the queue jobs to receive all the active parts that the reference/master replica has. Strings source_replica_parts = zookeeper->getChildren(fs::path(source_path) / "parts"); - ActiveDataPartSet active_parts_set(format_version, source_replica_parts); + for (const auto & active_part : source_replica_parts) + get_part_set.add(active_part); - Strings active_parts = active_parts_set.getParts(); + Strings active_parts = get_part_set.getParts(); /// Remove local parts if source replica does not have them, because such parts will never be fetched by other replicas. Strings local_parts_in_zk = zookeeper->getChildren(fs::path(replica_path) / "parts"); @@ -2367,7 +2375,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo for (const auto & part : local_parts_in_zk) { - if (active_parts_set.getContainingPart(part).empty()) + if (get_part_set.getContainingPart(part).empty()) { parts_to_remove_from_zk.emplace_back(part); LOG_WARNING(log, "Source replica does not have part {}. Removing it from ZooKeeper.", part); @@ -2392,7 +2400,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo for (const auto & part : local_active_parts) { - if (active_parts_set.getContainingPart(part->name).empty()) + if (get_part_set.getContainingPart(part->name).empty()) { parts_to_remove_from_working_set.emplace_back(part); LOG_WARNING(log, "Source replica does not have part {}. Removing it from working set.", part->name); @@ -2412,13 +2420,54 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo removePartsFromWorkingSet(parts_to_remove_from_working_set, true); + std::unordered_set created_get_parts; + + /// Avoid creation of GET_PART entries which covered by another GET_PART or DROP_RANGE + /// and creation of multiple entries with the same new_part_name. + auto should_ignore_log_entry = [&] (const String & part_name, const String & log_msg_context) -> bool + { + /// We should not create entries covered by DROP_RANGE, because we will remove them anyway (kind of optimization). + String covering_drop_range = drop_range_set.getContainingPart(part_name); + if (!covering_drop_range.empty()) + { + LOG_TRACE(log, "{} {}: it's covered by DROP_RANGE {}", log_msg_context, part_name, covering_drop_range); + return true; + } + + /// We should not create entries covered by GET_PART, + /// because GET_PART entry has no source parts and we can execute it only by fetching. + /// Parts covered by GET_PART are useless and may cause replication to stuck if covered part is lost. + String covering_get_part_entry = get_part_set.getContainingPart(part_name); + + if (covering_get_part_entry.empty()) + return false; + + if (covering_get_part_entry != part_name) + { + LOG_TRACE(log, "{} {}: it's covered by GET_PART {}", log_msg_context, part_name, covering_get_part_entry); + return true; + } + + if (created_get_parts.count(part_name)) + { + /// NOTE It would be better to copy log entry instead of creating GET_PART + /// if there are GET_PART and log entry of other type with the same new_part_name. + /// But it's a bit harder to implement, because it requires full-fledged virtual_parts set. + LOG_TRACE(log, "{} {}: GET_PART for it is already created", log_msg_context, part_name); + return true; + } + + return false; + }; + for (const String & name : active_parts) { + if (should_ignore_log_entry(name, "Not fetching")) + continue; + LogEntry log_entry; - if (!are_restoring_replica) - log_entry.type = LogEntry::GET_PART; - else + if (are_restoring_replica) { LOG_DEBUG(log, "Obtaining checksum for path {}", name); @@ -2443,23 +2492,46 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo const auto [lo, hi] = desired_checksums.hash_of_all_files; log_entry.part_checksum = getHexUIntUppercase(hi) + getHexUIntUppercase(lo); } + else + { + log_entry.type = LogEntry::GET_PART; + } log_entry.source_replica = ""; log_entry.new_part_name = name; log_entry.create_time = tryGetPartCreateTime(zookeeper, source_path, name); + LOG_TEST(log, "Enqueueing {} for fetch", name); zookeeper->create(fs::path(replica_path) / "queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential); + created_get_parts.insert(name); } - LOG_DEBUG(log, "Queued {} parts to be fetched", active_parts.size()); + size_t total_parts_to_fetch = created_get_parts.size(); + LOG_DEBUG(log, "Queued {} parts to be fetched, {} parts ignored", total_parts_to_fetch, active_parts.size() - total_parts_to_fetch); /// Add content of the reference/master replica queue to the queue. - for (const String & entry : source_queue) + size_t total_entries_to_copy = 0; + for (const auto & entry_info : source_queue) { - zookeeper->create(fs::path(replica_path) / "queue/queue-", entry, zkutil::CreateMode::PersistentSequential); + if (entry_info.parsed_entry && !entry_info.parsed_entry->new_part_name.empty()) + { + const String & part_name = entry_info.parsed_entry->new_part_name; + const String & entry_name = entry_info.parsed_entry->znode_name; + const auto & entry_type = entry_info.parsed_entry->type; + + if (should_ignore_log_entry(part_name, fmt::format("Not copying {} {} ", entry_name, entry_type))) + continue; + + if (entry_info.parsed_entry->type == LogEntry::GET_PART) + created_get_parts.insert(part_name); + } + + LOG_TEST(log, "Copying entry {}", entry_info.data); + zookeeper->create(fs::path(replica_path) / "queue/queue-", entry_info.data, zkutil::CreateMode::PersistentSequential); + ++total_entries_to_copy; } - LOG_DEBUG(log, "Copied {} queue entries", source_queue.size()); + LOG_DEBUG(log, "Copied {} queue entries, {} entries ignored", total_entries_to_copy, source_queue.size() - total_entries_to_copy); } @@ -3196,6 +3268,34 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n { auto zookeeper = getZooKeeper(); + /// We don't know exactly what happened to broken part + /// and we are going to remove all covered log entries. + /// It's quite dangerous, so clone covered parts to detached. + auto broken_part_info = MergeTreePartInfo::fromPartName(part_name, format_version); + + auto partition_range = getDataPartsPartitionRange(broken_part_info.partition_id); + for (const auto & part : partition_range) + { + if (!broken_part_info.contains(part->info)) + continue; + + /// Broken part itself ether already moved to detached or does not exist. + assert(broken_part_info != part->info); + part->makeCloneInDetached("covered-by-broken", getInMemoryMetadataPtr()); + } + + /// It's possible that queue contains entries covered by part_name. + /// For example, we had GET_PART all_1_42_5 and MUTATE_PART all_1_42_5_63, + /// then all_1_42_5_63 was executed by fetching, but part was written to disk incorrectly. + /// In this case we have to remove it as broken and create GET_PART all_1_42_5_63 to fetch it again, + /// but GET_PART all_1_42_5 may be still in the queue. + /// We should remove all covered entries before creating GET_PART entry, because: + /// 1. In the situation described above, we do not know how to merge/mutate all_1_42_5_63 from all_1_42_5, + /// so GET_PART all_1_42_5 (and all source parts) is useless. The only thing we can do is to fetch all_1_42_5_63. + /// 2. If all_1_42_5_63 is lost, then replication may stuck waiting for all_1_42_5_63 to appear, + /// because we may have some covered parts (more precisely, parts with the same min and max blocks) + queue.removePartProducingOpsInRange(zookeeper, broken_part_info, {}); + String part_path = fs::path(replica_path) / "parts" / part_name; Coordination::Requests ops; @@ -3218,11 +3318,7 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n fs::path(replica_path) / "queue/queue-", log_entry->toString(), zkutil::CreateMode::PersistentSequential)); - auto results = zookeeper->multi(ops); - - String path_created = dynamic_cast(*results.back()).path_created; - log_entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1); - queue.insert(zookeeper, log_entry); + zookeeper->multi(ops); } From 6dad340f109261e8e796c40428c4474ae5960170 Mon Sep 17 00:00:00 2001 From: lehasm Date: Tue, 19 Oct 2021 08:21:45 +0300 Subject: [PATCH 203/919] Update docs/en/operations/settings/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2d3f0b812b4..5893c19fd15 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3821,8 +3821,8 @@ Enables describing subcolumns for a [DESCRIBE](../../sql-reference/statements/de Possible values: -- 0 — Disabled. -- 1 — Enabled. +- 0 — Subcolumns are not included in DESCRIBE queries. +- 1 — Subcolumns are included in DESCRIBE queries. Default value: `0`. From ebee5025ebf278b4e42268f89046ad0e4a27c974 Mon Sep 17 00:00:00 2001 From: lehasm Date: Tue, 19 Oct 2021 08:22:09 +0300 Subject: [PATCH 204/919] Update docs/en/operations/settings/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 5893c19fd15..2cf9263a05d 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3826,7 +3826,7 @@ Possible values: Default value: `0`. -**EXAMPLE** +**Example** Query: From 49106f407ffbc8cacca0f71babddbf81934230ec Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 19 Oct 2021 09:34:06 +0300 Subject: [PATCH 205/919] Fix some checks --- src/Compression/CachedCompressedReadBuffer.h | 6 +++++- src/Disks/ReadBufferFromRemoteFSGather.cpp | 18 +++++++++--------- src/Disks/ReadBufferFromRemoteFSGather.h | 4 ++-- src/Storages/MergeTree/MergeTreeReadPool.cpp | 2 +- 4 files changed, 17 insertions(+), 13 deletions(-) diff --git a/src/Compression/CachedCompressedReadBuffer.h b/src/Compression/CachedCompressedReadBuffer.h index eb45667dc25..86c7d3d1ce4 100644 --- a/src/Compression/CachedCompressedReadBuffer.h +++ b/src/Compression/CachedCompressedReadBuffer.h @@ -59,7 +59,11 @@ public: clock_type = clock_type_; } - void setReadUntilPosition(size_t position) override { file_in->setReadUntilPosition(position); } + void setReadUntilPosition(size_t position) override + { + if (file_in) + file_in->setReadUntilPosition(position); + } }; } diff --git a/src/Disks/ReadBufferFromRemoteFSGather.cpp b/src/Disks/ReadBufferFromRemoteFSGather.cpp index f8246f41340..ff9231bf9d3 100644 --- a/src/Disks/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/ReadBufferFromRemoteFSGather.cpp @@ -28,24 +28,24 @@ namespace ErrorCodes #if USE_AWS_S3 -SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const String & path, size_t last_offset) const +SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const String & path, size_t offset) const { return std::make_unique(client_ptr, bucket, - fs::path(metadata.remote_fs_root_path) / path, max_single_read_retries, settings, threadpool_read, last_offset); + fs::path(metadata.remote_fs_root_path) / path, max_single_read_retries, settings, threadpool_read, offset); } #endif -SeekableReadBufferPtr ReadBufferFromWebServerGather::createImplementationBuffer(const String & path, size_t last_offset) const +SeekableReadBufferPtr ReadBufferFromWebServerGather::createImplementationBuffer(const String & path, size_t offset) const { - return std::make_unique(fs::path(uri) / path, context, settings, threadpool_read, last_offset); + return std::make_unique(fs::path(uri) / path, context, settings, threadpool_read, offset); } #if USE_HDFS -SeekableReadBufferPtr ReadBufferFromHDFSGather::createImplementationBuffer(const String & path, size_t last_offset) const +SeekableReadBufferPtr ReadBufferFromHDFSGather::createImplementationBuffer(const String & path, size_t offset) const { - return std::make_unique(hdfs_uri, fs::path(hdfs_directory) / path, config, buf_size, last_offset); + return std::make_unique(hdfs_uri, fs::path(hdfs_directory) / path, config, buf_size, offset); } #endif @@ -53,7 +53,7 @@ SeekableReadBufferPtr ReadBufferFromHDFSGather::createImplementationBuffer(const ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather(const RemoteMetadata & metadata_, const String & path_) : ReadBuffer(nullptr, 0) , metadata(metadata_) - , path(path_) + , canonical_path(path_) { } @@ -147,7 +147,7 @@ bool ReadBufferFromRemoteFSGather::readImpl() if (bytes_to_ignore) current_buf->ignore(bytes_to_ignore); - LOG_DEBUG(&Poco::Logger::get("Gather"), "Reading from path: {}", path); + LOG_DEBUG(&Poco::Logger::get("Gather"), "Reading from path: {}", canonical_path); auto result = current_buf->next(); swap(*current_buf); @@ -182,7 +182,7 @@ void ReadBufferFromRemoteFSGather::reset() String ReadBufferFromRemoteFSGather::getFileName() const { - return path; + return canonical_path; // if (current_buf) // return fs::path(metadata.metadata_file_path) / metadata.remote_fs_objects[buf_idx].first; // return metadata.metadata_file_path; diff --git a/src/Disks/ReadBufferFromRemoteFSGather.h b/src/Disks/ReadBufferFromRemoteFSGather.h index 70d3adc8fb1..72088012aeb 100644 --- a/src/Disks/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/ReadBufferFromRemoteFSGather.h @@ -37,7 +37,7 @@ public: size_t readInto(char * data, size_t size, size_t offset, size_t ignore = 0); protected: - virtual SeekableReadBufferPtr createImplementationBuffer(const String & path, size_t last_offset) const = 0; + virtual SeekableReadBufferPtr createImplementationBuffer(const String & path, size_t offset) const = 0; RemoteMetadata metadata; @@ -60,7 +60,7 @@ private: size_t last_offset = 0; - String path; + String canonical_path; }; diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 8f0f20c7d2c..326ad843e7e 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -48,7 +48,7 @@ MergeTreeReadPool::MergeTreeReadPool( do_not_steal_tasks = true; min_marks_for_concurrent_read = std::max(min_marks_for_concurrent_read, sum_marks_ / threads_); } - fillPerThreadInfo(threads_, sum_marks_, per_part_sum_marks, parts_ranges, min_marks_for_concurrent_read_); + fillPerThreadInfo(threads_, sum_marks_, per_part_sum_marks, parts_ranges, min_marks_for_concurrent_read); } From a992895b09eeea4dd7a6a7c1943cd04e5ad1ace8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Oct 2021 10:14:53 +0300 Subject: [PATCH 206/919] Debug --- src/Coordination/KeeperServer.cpp | 4 +++ src/Coordination/KeeperStateManager.cpp | 34 +++++++++++++++++-------- 2 files changed, 27 insertions(+), 11 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 6838c383be1..02afd858b1d 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -391,6 +391,10 @@ void KeeperServer::updateConfiguration(const Poco::Util::AbstractConfiguration & { LOG_WARNING(log, "Configuration changed for more than one server ({}) from cluster, it's strictly not recommended", diff.size()); } + else + { + LOG_WARNING(log, "Configuration change size ({})", diff.size()); + } for (auto & task : diff) { diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index cca9f503294..0039c08c58c 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -150,24 +150,35 @@ ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::A std::unordered_map new_ids, old_ids; for (auto new_server : new_servers_configuration.cluster_config->get_servers()) + { + LOG_INFO(log, "NEW SERVER {}", new_server->get_id()); new_ids[new_server->get_id()] = new_server; + } for (auto old_server : servers_configuration.cluster_config->get_servers()) + { + LOG_INFO(log, "OLD SERVER {}", old_server->get_id()); old_ids[old_server->get_id()] = old_server; - - std::unordered_map servers_to_remove, servers_to_add; - - auto comp = [] (auto & a, auto & b) { return a.first < b.first; }; - std::set_difference(old_ids.begin(), old_ids.end(), new_ids.begin(), new_ids.end(), std::inserter(servers_to_remove, servers_to_remove.begin()), comp); - std::set_difference(new_ids.begin(), new_ids.end(), old_ids.begin(), old_ids.end(), std::inserter(servers_to_add, servers_to_add.begin()), comp); + } ConfigUpdateActions result; + for (auto [old_id, server_config] : old_ids) + { + if (!new_ids.count(old_id)) + { + LOG_INFO(log, "REMOVING SERVER {}", old_id); + result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::RemoveServer, server_config}); + } + } - for (auto & [_, server_config] : servers_to_remove) - result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::RemoveServer, server_config}); - - for (auto & [_, server_config] : servers_to_add) - result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::AddServer, server_config}); + for (auto [new_id, server_config] : new_ids) + { + if (!old_ids.count(new_id)) + { + LOG_INFO(log, "ADDING SERVER {}", new_id); + result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::AddServer, server_config}); + } + } for (const auto & old_server : servers_configuration.cluster_config->get_servers()) { @@ -175,6 +186,7 @@ ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::A { if (old_server->get_id() == new_server->get_id()) { + LOG_INFO(log, "UPDATE PRIORITY {}", new_server->get_id()); if (old_server->get_priority() != new_server->get_priority()) result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::UpdatePriority, new_server}); break; From df5dc4e991371c9da82054506c6d8bc19260ef8e Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 19 Oct 2021 10:51:18 +0300 Subject: [PATCH 207/919] Adjust MergeTreeIndexReader --- src/Storages/MergeTree/MergeTreeIndexReader.cpp | 11 +++++++++++ src/Storages/MergeTree/MergeTreeReaderCompact.cpp | 8 +++++--- src/Storages/MergeTree/MergeTreeReaderCompact.h | 2 +- 3 files changed, 17 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexReader.cpp b/src/Storages/MergeTree/MergeTreeIndexReader.cpp index fd7358967f3..2e83ac8d427 100644 --- a/src/Storages/MergeTree/MergeTreeIndexReader.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexReader.cpp @@ -54,6 +54,17 @@ MergeTreeIndexReader::MergeTreeIndexReader( std::move(settings)); version = index_format.version; + auto current_task_last_mark_range = std::max_element(all_mark_ranges_.begin(), all_mark_ranges_.end(), + [&](const MarkRange & range1, const MarkRange & range2) + { + return range1.end < range2.end; + }); + + size_t current_task_last_mark = 0; + if (current_task_last_mark_range != all_mark_ranges_.end()) + current_task_last_mark = current_task_last_mark_range->end; + + stream->adjustForRange(0, current_task_last_mark); stream->seekToStart(); } diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 35da14319ba..e2a55f5f31f 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -122,7 +122,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( } size_t MergeTreeReaderCompact::readRows( - size_t from_mark, size_t /* current_task_last_mark */, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) + size_t from_mark, size_t current_task_last_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) { if (continue_reading) from_mark = next_mark; @@ -158,7 +158,7 @@ size_t MergeTreeReaderCompact::readRows( auto & column = res_columns[pos]; size_t column_size_before_reading = column->size(); - readData(column_from_part, column, from_mark, *column_positions[pos], rows_to_read, read_only_offsets[pos]); + readData(column_from_part, column, from_mark, current_task_last_mark, *column_positions[pos], rows_to_read, read_only_offsets[pos]); size_t read_rows_in_column = column->size() - column_size_before_reading; if (read_rows_in_column < rows_to_read) @@ -192,7 +192,7 @@ size_t MergeTreeReaderCompact::readRows( void MergeTreeReaderCompact::readData( const NameAndTypePair & name_and_type, ColumnPtr & column, - size_t from_mark, size_t column_position, size_t rows_to_read, bool only_offsets) + size_t from_mark, size_t current_task_last_mark, size_t column_position, size_t rows_to_read, bool only_offsets) { const auto & [name, type] = name_and_type; @@ -204,6 +204,8 @@ void MergeTreeReaderCompact::readData( if (only_offsets && (substream_path.size() != 1 || substream_path[0].type != ISerialization::Substream::ArraySizes)) return nullptr; + /// For asynchronous reading from remote fs. + data_buffer->setReadUntilPosition(marks_loader.getMark(current_task_last_mark).offset_in_compressed_file); return data_buffer; }; diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index 5a419a23642..350c8427eff 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -58,7 +58,7 @@ private: void seekToMark(size_t row_index, size_t column_index); void readData(const NameAndTypePair & name_and_type, ColumnPtr & column, size_t from_mark, - size_t column_position, size_t rows_to_read, bool only_offsets); + size_t current_task_last_mark, size_t column_position, size_t rows_to_read, bool only_offsets); /// Returns maximal value of granule size in compressed file from @mark_ranges. /// This value is used as size of read buffer. From 254eecf2270548206b60d9fbeab8b7795e87b3a8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 19 Oct 2021 11:11:54 +0300 Subject: [PATCH 208/919] Update some tests. --- ...monotonous_functions_in_order_by.reference | 14 ++-- .../01576_alias_column_rewrite.reference | 18 +++-- .../01591_window_functions.reference | 26 +++---- .../01655_plan_optimizations.reference | 12 ++-- .../0_stateless/01655_plan_optimizations.sh | 4 +- ...1951_distributed_push_down_limit.reference | 28 ++++---- ...istributed_group_by_sharding_key.reference | 68 ++++++++----------- 7 files changed, 72 insertions(+), 98 deletions(-) diff --git a/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference index bf9bff06959..d8c9b88d8e8 100644 --- a/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference +++ b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference @@ -6,12 +6,10 @@ ORDER BY timestamp ASC LIMIT 10 Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) - MergingSorted (Merge sorted streams for ORDER BY) - MergeSorting (Merge sorted blocks for ORDER BY) - PartialSorting (Sort each block for ORDER BY) - Expression (Before ORDER BY) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - ReadFromMergeTree + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromMergeTree SELECT timestamp, key @@ -20,7 +18,7 @@ ORDER BY toDate(timestamp) ASC LIMIT 10 Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) - FinishSorting + Sorting Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) ReadFromMergeTree @@ -34,7 +32,7 @@ ORDER BY LIMIT 10 Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) - FinishSorting + Sorting Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) ReadFromMergeTree diff --git a/tests/queries/0_stateless/01576_alias_column_rewrite.reference b/tests/queries/0_stateless/01576_alias_column_rewrite.reference index 2a824e62158..07d361cfa46 100644 --- a/tests/queries/0_stateless/01576_alias_column_rewrite.reference +++ b/tests/queries/0_stateless/01576_alias_column_rewrite.reference @@ -23,21 +23,19 @@ lambda optimize_read_in_order Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) - MergingSorted (Merge sorted streams for ORDER BY) - MergeSorting (Merge sorted blocks for ORDER BY) - PartialSorting (Sort each block for ORDER BY) - Expression (Before ORDER BY) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - ReadFromMergeTree -Expression (Projection) - Limit (preliminary LIMIT (without OFFSET)) - FinishSorting + Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) ReadFromMergeTree Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) - FinishSorting + Sorting + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromMergeTree +Expression (Projection) + Limit (preliminary LIMIT (without OFFSET)) + Sorting Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) ReadFromMergeTree diff --git a/tests/queries/0_stateless/01591_window_functions.reference b/tests/queries/0_stateless/01591_window_functions.reference index f54c10ee8b9..07562557369 100644 --- a/tests/queries/0_stateless/01591_window_functions.reference +++ b/tests/queries/0_stateless/01591_window_functions.reference @@ -908,12 +908,10 @@ Expression ((Projection + Before ORDER BY)) Window (Window step for window \'\') Window (Window step for window \'PARTITION BY p\') Window (Window step for window \'PARTITION BY p ORDER BY o ASC\') - MergingSorted (Merge sorted streams for window \'PARTITION BY p ORDER BY o ASC\') - MergeSorting (Merge sorted blocks for window \'PARTITION BY p ORDER BY o ASC\') - PartialSorting (Sort each block for window \'PARTITION BY p ORDER BY o ASC\') - Expression ((Before window functions + (Projection + Before ORDER BY))) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - ReadFromStorage (SystemNumbers) + Sorting (Sorting for window \'PARTITION BY p ORDER BY o ASC\') + Expression ((Before window functions + (Projection + Before ORDER BY))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) explain select count(*) over (order by o, number), count(*) over (order by number) @@ -923,16 +921,12 @@ from ; Expression ((Projection + Before ORDER BY)) Window (Window step for window \'ORDER BY o ASC, number ASC\') - MergingSorted (Merge sorted streams for window \'ORDER BY o ASC, number ASC\') - MergeSorting (Merge sorted blocks for window \'ORDER BY o ASC, number ASC\') - PartialSorting (Sort each block for window \'ORDER BY o ASC, number ASC\') - Window (Window step for window \'ORDER BY number ASC\') - MergingSorted (Merge sorted streams for window \'ORDER BY number ASC\') - MergeSorting (Merge sorted blocks for window \'ORDER BY number ASC\') - PartialSorting (Sort each block for window \'ORDER BY number ASC\') - Expression ((Before window functions + (Projection + Before ORDER BY))) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - ReadFromStorage (SystemNumbers) + Sorting (Sorting for window \'ORDER BY o ASC, number ASC\') + Window (Window step for window \'ORDER BY number ASC\') + Sorting (Sorting for window \'ORDER BY number ASC\') + Expression ((Before window functions + (Projection + Before ORDER BY))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) -- A test case for the sort comparator found by fuzzer. SELECT max(number) OVER (ORDER BY number DESC NULLS FIRST), diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index 22876207862..33a7ff44b74 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -5,11 +5,8 @@ FUNCTION sipHash64 Filter column: equals > sorting steps should know about limit Limit 10 -MergingSorted -Limit 10 -MergeSorting -Limit 10 -PartialSorting +Sorting +Sorting Limit 10 -- filter push down -- > filter should be pushed down after aggregating @@ -108,9 +105,8 @@ Filter column: notEquals(y, 2) 1 0 1 1 > filter is pushed down before sorting steps -MergingSorted -MergeSorting -PartialSorting +Sorting +Sorting Filter column: and(notEquals(x, 0), notEquals(y, 0)) 1 2 1 1 diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index b835bae0e27..de3d3ac3eb6 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -10,7 +10,7 @@ $CLICKHOUSE_CLIENT -q "select x + 1 from (select y + 2 as x from (select dummy + echo "> sipHash should be calculated after filtration" $CLICKHOUSE_CLIENT -q "explain actions = 1 select sum(x), sum(y) from (select sipHash64(number) as x, bitAnd(number, 1024) as y from numbers_mt(1000000000) limit 1000000000) where y = 0" | grep -o "FUNCTION sipHash64\|Filter column: equals" echo "> sorting steps should know about limit" -$CLICKHOUSE_CLIENT -q "explain actions = 1 select number from (select number from numbers(500000000) order by -number) limit 10" | grep -o "MergingSorted\|MergeSorting\|PartialSorting\|Limit 10" +$CLICKHOUSE_CLIENT -q "explain actions = 1 select number from (select number from numbers(500000000) order by -number) limit 10" | grep -o "Sorting\|Limit 10" echo "-- filter push down --" echo "> filter should be pushed down after aggregating" @@ -132,7 +132,7 @@ $CLICKHOUSE_CLIENT -q " select number % 2 as x, number % 3 as y from numbers(6) order by y desc ) where x != 0 and y != 0 settings enable_optimize_predicate_expression = 0" | - grep -o "MergingSorted\|MergeSorting\|PartialSorting\|Filter column: and(notEquals(x, 0), notEquals(y, 0))" + grep -o "Sorting\|Filter column: and(notEquals(x, 0), notEquals(y, 0))" $CLICKHOUSE_CLIENT -q " select x, y from ( select number % 2 as x, number % 3 as y from numbers(6) order by y desc diff --git a/tests/queries/0_stateless/01951_distributed_push_down_limit.reference b/tests/queries/0_stateless/01951_distributed_push_down_limit.reference index ca3bbc4cb33..d0e7a9ef15b 100644 --- a/tests/queries/0_stateless/01951_distributed_push_down_limit.reference +++ b/tests/queries/0_stateless/01951_distributed_push_down_limit.reference @@ -2,31 +2,27 @@ explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=0; Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) - MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) + Sorting (Merge sorted streams after aggregation stage for ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) Union - MergingSorted (Merge sorted streams for ORDER BY) - MergeSorting (Merge sorted blocks for ORDER BY) - PartialSorting (Sort each block for ORDER BY) - Expression (Before ORDER BY) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) SettingQuotaAndLimits (Set limits and quota after reading from storage) - Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - ReadFromStorage (SystemNumbers) + ReadFromStorage (SystemNumbers) ReadFromRemote (Read from remote replica) explain select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1; Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) - MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) + Sorting (Merge sorted streams after aggregation stage for ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) Union Limit (preliminary LIMIT (with OFFSET)) - MergingSorted (Merge sorted streams for ORDER BY) - MergeSorting (Merge sorted blocks for ORDER BY) - PartialSorting (Sort each block for ORDER BY) - Expression (Before ORDER BY) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) SettingQuotaAndLimits (Set limits and quota after reading from storage) - Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - ReadFromStorage (SystemNumbers) + ReadFromStorage (SystemNumbers) ReadFromRemote (Read from remote replica) diff --git a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference index 10787068f43..b23631395ff 100644 --- a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference +++ b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference @@ -50,66 +50,58 @@ SettingQuotaAndLimits (Set limits and quota after reading from storage) explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized Expression (Projection) Distinct - MergingSorted (Merge sorted streams for ORDER BY, without aggregation) + Sorting (Merge sorted streams for ORDER BY, without aggregation) SettingQuotaAndLimits (Set limits and quota after reading from storage) Union - MergingSorted (Merge sorted streams for ORDER BY) - MergeSorting (Merge sorted blocks for ORDER BY) - PartialSorting (Sort each block for ORDER BY) - Distinct (Preliminary DISTINCT) - Expression (Before ORDER BY) + Sorting (Sorting for ORDER BY) + Distinct (Preliminary DISTINCT) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) SettingQuotaAndLimits (Set limits and quota after reading from storage) - Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - ReadFromStorage (SystemNumbers) + ReadFromStorage (SystemNumbers) ReadFromRemote (Read from remote replica) explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized Expression (Projection) - MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) + Sorting (Merge sorted streams after aggregation stage for ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) Union Distinct - MergingSorted (Merge sorted streams for ORDER BY) - MergeSorting (Merge sorted blocks for ORDER BY) - PartialSorting (Sort each block for ORDER BY) - Distinct (Preliminary DISTINCT) - Expression (Before ORDER BY) + Sorting (Sorting for ORDER BY) + Distinct (Preliminary DISTINCT) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) SettingQuotaAndLimits (Set limits and quota after reading from storage) - Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - ReadFromStorage (SystemNumbers) + ReadFromStorage (SystemNumbers) ReadFromRemote (Read from remote replica) explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized Expression (Projection) LimitBy Expression (Before LIMIT BY) - MergingSorted (Merge sorted streams for ORDER BY, without aggregation) + Sorting (Merge sorted streams for ORDER BY, without aggregation) SettingQuotaAndLimits (Set limits and quota after reading from storage) Union LimitBy Expression (Before LIMIT BY) - MergingSorted (Merge sorted streams for ORDER BY) - MergeSorting (Merge sorted blocks for ORDER BY) - PartialSorting (Sort each block for ORDER BY) - Expression (Before ORDER BY) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - ReadFromStorage (SystemNumbers) - ReadFromRemote (Read from remote replica) -explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized -Expression (Projection) - MergingSorted (Merge sorted streams after aggregation stage for ORDER BY) - SettingQuotaAndLimits (Set limits and quota after reading from storage) - Union - LimitBy - Expression (Before LIMIT BY) - MergingSorted (Merge sorted streams for ORDER BY) - MergeSorting (Merge sorted blocks for ORDER BY) - PartialSorting (Sort each block for ORDER BY) + Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) SettingQuotaAndLimits (Set limits and quota after reading from storage) Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) SettingQuotaAndLimits (Set limits and quota after reading from storage) ReadFromStorage (SystemNumbers) + ReadFromRemote (Read from remote replica) +explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized +Expression (Projection) + Sorting (Merge sorted streams after aggregation stage for ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Union + LimitBy + Expression (Before LIMIT BY) + Sorting (Sorting for ORDER BY) + Expression (Before ORDER BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + Expression ((Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) ReadFromRemote (Read from remote replica) From 40677bffa51f7ad8c905e47f81b32c80c16e5dd1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 19 Oct 2021 12:28:49 +0300 Subject: [PATCH 209/919] Fix PVS warning. --- src/Processors/QueryPlan/SortingStep.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index e34ee2842da..8e253e71f44 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -66,11 +66,11 @@ private: UInt64 limit; SizeLimits size_limits; - size_t max_bytes_before_remerge; - double remerge_lowered_memory_bytes_ratio; - size_t max_bytes_before_external_sort; + size_t max_bytes_before_remerge = 0; + double remerge_lowered_memory_bytes_ratio = 0; + size_t max_bytes_before_external_sort = 0; VolumePtr tmp_volume; - size_t min_free_disk_space; + size_t min_free_disk_space = 0; }; } From a16eda68dc54ff0c3be7362dfec02061074f6e04 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 19 Oct 2021 12:39:14 +0300 Subject: [PATCH 210/919] Fix tests and style --- src/DataTypes/Serializations/SerializationNullable.cpp | 4 +--- src/Processors/Formats/Impl/BinaryRowInputFormat.cpp | 1 - src/Processors/Formats/Impl/BinaryRowInputFormat.h | 2 +- src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 5 ----- src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp | 5 ----- tests/queries/0_stateless/00938_template_input_format.sh | 2 ++ 6 files changed, 4 insertions(+), 15 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index 193d1beea2f..2ac4bcc036b 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -12,8 +12,6 @@ #include #include #include -#include -#include #include namespace DB @@ -275,7 +273,7 @@ void SerializationNullable::serializeTextRaw(const IColumn & column, size_t row_ template ReturnType SerializationNullable::deserializeTextRawImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested) { - return deserializeTextEscapedAndRawImpl(column, istr, settings, nested); + return deserializeTextEscapedAndRawImpl(column, istr, settings, nested); } template diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp index d3cea169eab..d05a2187ac8 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp @@ -18,7 +18,6 @@ BinaryRowInputFormat::BinaryRowInputFormat(ReadBuffer & in_, Block header, Param { } - std::vector BinaryRowInputFormat::readHeaderRow() { std::vector fields; diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.h b/src/Processors/Formats/Impl/BinaryRowInputFormat.h index 020be4f4db6..78c2a626e24 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.h +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.h @@ -38,7 +38,7 @@ private: std::vector readHeaderRow(); DataTypes read_data_types; - UInt64 read_columns; + UInt64 read_columns = 0; }; } diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 5439d902369..bf5f3ea468b 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -8,7 +8,6 @@ #include #include - namespace DB { @@ -19,7 +18,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } - CSVRowInputFormat::CSVRowInputFormat( const Block & header_, ReadBuffer & in_, @@ -37,7 +35,6 @@ CSVRowInputFormat::CSVRowInputFormat( ErrorCodes::BAD_ARGUMENTS); } - static void skipEndOfLine(ReadBuffer & in) { /// \n (Unix) or \r\n (DOS/Windows) or \n\r (Mac OS Classic) @@ -233,8 +230,6 @@ bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, co } } - - void registerInputFormatCSV(FormatFactory & factory) { auto register_func = [&](const String & format_name, bool with_names, bool with_types) diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index ae7ecb955b0..6f04da77ac4 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include @@ -30,7 +29,6 @@ static void checkForCarriageReturn(ReadBuffer & in) ErrorCodes::INCORRECT_DATA); } - TabSeparatedRowInputFormat::TabSeparatedRowInputFormat( const Block & header_, ReadBuffer & in_, @@ -96,7 +94,6 @@ std::vector TabSeparatedRowInputFormat::readHeaderRow() return fields; } - bool TabSeparatedRowInputFormat::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & /*column_name*/) { @@ -120,8 +117,6 @@ bool TabSeparatedRowInputFormat::readField(IColumn & column, const DataTypePtr & return true; } - - if (as_nullable) return SerializationNullable::deserializeTextEscapedImpl(column, *in, format_settings, serialization); diff --git a/tests/queries/0_stateless/00938_template_input_format.sh b/tests/queries/0_stateless/00938_template_input_format.sh index bf7631cf3d5..9218f4bebca 100755 --- a/tests/queries/0_stateless/00938_template_input_format.sh +++ b/tests/queries/0_stateless/00938_template_input_format.sh @@ -1,4 +1,6 @@ #!/usr/bin/env bash +# Tags: no-parallel + # shellcheck disable=SC2016,SC2028 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From cfa685c29ca99679eb2130e99afafc53e9c3b9e7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 19 Oct 2021 13:16:30 +0300 Subject: [PATCH 211/919] Fix another test. --- .../0_stateless/01823_explain_json.reference | 28 ------------------- 1 file changed, 28 deletions(-) diff --git a/tests/queries/0_stateless/01823_explain_json.reference b/tests/queries/0_stateless/01823_explain_json.reference index 5c7845a22d5..f75cd69dbf3 100644 --- a/tests/queries/0_stateless/01823_explain_json.reference +++ b/tests/queries/0_stateless/01823_explain_json.reference @@ -111,31 +111,3 @@ } ], "Limit": 3, --- - "Sort Description": [ - { - "Column": "number", - "Ascending": false, - "With Fill": false - }, - { - "Column": "plus(number, 1)", - "Ascending": true, - "With Fill": false - } - ], - "Limit": 3, --- - "Sort Description": [ - { - "Column": "number", - "Ascending": false, - "With Fill": false - }, - { - "Column": "plus(number, 1)", - "Ascending": true, - "With Fill": false - } - ], - "Limit": 3, From 74fd33b62eb56f88e6b372382ef4dd3d9e92f5d6 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 19 Oct 2021 13:25:22 +0300 Subject: [PATCH 212/919] Fix style --- src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index 1c5843ce97b..95e92709881 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -14,7 +14,6 @@ namespace DB namespace ErrorCodes { extern const int INCORRECT_DATA; - extern const int CANNOT_READ_ALL_DATA; } From caf972db48c38a24c588c5c333016979812ed6b5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 19 Oct 2021 14:56:04 +0300 Subject: [PATCH 213/919] fix --- src/Storages/StorageReplicatedMergeTree.cpp | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 6980d523ed5..9a3c88ca83f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2308,10 +2308,9 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo struct QueueEntryInfo { - String data; - Coordination::Stat stat; - LogEntryPtr parsed_entry; - bool need_copy = true; + String data = {}; + Coordination::Stat stat = {}; + LogEntryPtr parsed_entry = {}; }; std::vector source_queue; @@ -2320,11 +2319,12 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo { std::vector queue_get_futures; - source_queue.resize(source_queue_names.size()); queue_get_futures.reserve(source_queue_names.size()); for (const String & entry_name : source_queue_names) queue_get_futures.push_back(zookeeper->asyncTryGet(fs::path(source_path) / "queue" / entry_name)); + + source_queue.reserve(source_queue_names.size()); for (size_t i = 0; i < source_queue_names.size(); ++i) { auto res = queue_get_futures[i].get(); @@ -2333,7 +2333,8 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo continue; assert(res.error == Coordination::Error::ZOK); - auto & info = source_queue[i]; + source_queue.emplace_back(); + auto & info = source_queue.back(); info.data = std::move(res.data); info.stat = std::move(res.stat); try @@ -2448,6 +2449,11 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo return true; } + /// NOTE: It does not completely avoids duplication of GET_PART entries, + /// because it's possible that source replica has executed some GET_PART after we copied it's queue, + /// but before we copied its active parts set. In this case we will GET_PART entry in our queue + /// and later will pull the original GET_PART from replication log. + /// It should not cause any issues, but it does not allow to get rid of duplicated entries and add an assertion. if (created_get_parts.count(part_name)) { /// NOTE It would be better to copy log entry instead of creating GET_PART @@ -2513,6 +2519,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo size_t total_entries_to_copy = 0; for (const auto & entry_info : source_queue) { + assert(!entry_info.data.empty()); if (entry_info.parsed_entry && !entry_info.parsed_entry->new_part_name.empty()) { const String & part_name = entry_info.parsed_entry->new_part_name; From 2d4b601d3827d905935f2e649f19c4c00dd674b9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Oct 2021 15:00:26 +0300 Subject: [PATCH 214/919] Better --- src/Coordination/CoordinationSettings.h | 3 +- src/Coordination/KeeperDispatcher.cpp | 71 +++++++++- src/Coordination/KeeperDispatcher.h | 7 + src/Coordination/KeeperServer.cpp | 126 ++++++++++++------ src/Coordination/KeeperServer.h | 4 +- src/Coordination/KeeperStateManager.cpp | 55 +++----- src/Coordination/KeeperStateManager.h | 19 ++- .../test_keeper_nodes_add/__init__.py | 1 + .../configs/enable_keeper1.xml | 22 +++ .../configs/enable_keeper_three_nodes_1.xml | 32 +++++ .../configs/enable_keeper_three_nodes_2.xml | 32 +++++ .../configs/enable_keeper_three_nodes_3.xml | 32 +++++ .../configs/enable_keeper_two_nodes_1.xml | 27 ++++ .../configs/enable_keeper_two_nodes_2.xml | 27 ++++ .../integration/test_keeper_nodes_add/test.py | 80 +++++++++++ .../test_keeper_nodes_move/__init__.py | 1 + .../configs/enable_keeper1.xml | 34 +++++ .../configs/enable_keeper2.xml | 34 +++++ .../configs/enable_keeper3.xml | 34 +++++ .../configs/enable_keeper_node4_1.xml | 34 +++++ .../configs/enable_keeper_node4_2.xml | 34 +++++ .../configs/enable_keeper_node4_4.xml | 34 +++++ .../test_keeper_nodes_move/test.py | 85 ++++++++++++ .../test_keeper_nodes_remove/__init__.py | 1 + .../configs/enable_keeper1.xml | 34 +++++ .../configs/enable_keeper2.xml | 34 +++++ .../configs/enable_keeper3.xml | 34 +++++ .../configs/enable_keeper_two_nodes_1.xml | 27 ++++ .../configs/enable_keeper_two_nodes_2.xml | 27 ++++ .../configs/enable_single_keeper1.xml | 22 +++ .../test_keeper_nodes_remove/test.py | 97 ++++++++++++++ 31 files changed, 1011 insertions(+), 93 deletions(-) create mode 100644 tests/integration/test_keeper_nodes_add/__init__.py create mode 100644 tests/integration/test_keeper_nodes_add/configs/enable_keeper1.xml create mode 100644 tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_1.xml create mode 100644 tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_2.xml create mode 100644 tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_3.xml create mode 100644 tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_1.xml create mode 100644 tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_2.xml create mode 100644 tests/integration/test_keeper_nodes_add/test.py create mode 100644 tests/integration/test_keeper_nodes_move/__init__.py create mode 100644 tests/integration/test_keeper_nodes_move/configs/enable_keeper1.xml create mode 100644 tests/integration/test_keeper_nodes_move/configs/enable_keeper2.xml create mode 100644 tests/integration/test_keeper_nodes_move/configs/enable_keeper3.xml create mode 100644 tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_1.xml create mode 100644 tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_2.xml create mode 100644 tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_4.xml create mode 100644 tests/integration/test_keeper_nodes_move/test.py create mode 100644 tests/integration/test_keeper_nodes_remove/__init__.py create mode 100644 tests/integration/test_keeper_nodes_remove/configs/enable_keeper1.xml create mode 100644 tests/integration/test_keeper_nodes_remove/configs/enable_keeper2.xml create mode 100644 tests/integration/test_keeper_nodes_remove/configs/enable_keeper3.xml create mode 100644 tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_1.xml create mode 100644 tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_2.xml create mode 100644 tests/integration/test_keeper_nodes_remove/configs/enable_single_keeper1.xml create mode 100644 tests/integration/test_keeper_nodes_remove/test.py diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 00d443822e6..bec5e7f1f7a 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -38,7 +38,8 @@ struct Settings; M(Bool, quorum_reads, false, "Execute read requests as writes through whole RAFT consesus with similar speed", 0) \ M(Bool, force_sync, true, "Call fsync on each change in RAFT changelog", 0) \ M(Bool, compress_logs, true, "Write compressed coordination logs in ZSTD format", 0) \ - M(Bool, compress_snapshots_with_zstd_format, true, "Write compressed snapshots in ZSTD format (instead of custom LZ4)", 0) + M(Bool, compress_snapshots_with_zstd_format, true, "Write compressed snapshots in ZSTD format (instead of custom LZ4)", 0) \ + M(UInt64, configuration_change_tries_count, 20, "How many times we will try to apply configuration change (add/remove server) to the cluster", 0) DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 27316a0f045..53d8b5e3585 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -275,7 +275,6 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf { server->waitInit(); LOG_DEBUG(log, "Quorum initialized"); - updateConfiguration(config); } else { @@ -290,6 +289,8 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf /// Start it after keeper server start session_cleaner_thread = ThreadFromGlobalPool([this] { sessionCleanerTask(); }); + update_configuration_thread = ThreadFromGlobalPool([this] { updateConfigurationThread(); }); + updateConfiguration(config); LOG_DEBUG(log, "Dispatcher initialized"); } @@ -325,6 +326,10 @@ void KeeperDispatcher::shutdown() snapshots_queue.finish(); if (snapshot_thread.joinable()) snapshot_thread.join(); + + update_configuration_queue.finish(); + if (update_configuration_thread.joinable()) + update_configuration_thread.join(); } if (server) @@ -505,17 +510,69 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms) return future.get(); } + +void KeeperDispatcher::updateConfigurationThread() +{ + while (true) + { + if (shutdown_called) + return; + + try + { + if (!server->checkInit()) + { + LOG_INFO(log, "Server still not initialized, will not apply configuration until initialization finished"); + std::this_thread::sleep_for(std::chrono::milliseconds(5000)); + continue; + } + + ConfigUpdateAction action; + if (!update_configuration_queue.pop(action)) + break; + /// Only leader node must check dead sessions + if (isLeader()) + { + server->applyConfigurationUpdate(action); + } + else + { + String message; + if (action.type == ConfigUpdateActionType::RemoveServer) + message += "remove"; + else if (action.type == ConfigUpdateActionType::AddServer) + message += "add"; + else if (action.type == ConfigUpdateActionType::UpdatePriority) + message += "update priority for"; + else + message += "unknown action for"; + + LOG_INFO(log, "Configuration changed ({} server {}), but we are not leader, so we will wait update from leader", message, task.server->get_id()); + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } +} + void KeeperDispatcher::updateConfiguration(const Poco::Util::AbstractConfiguration & config) { - if (isLeader()) - { - server->updateConfiguration(config); - } + auto diff = server->getConfigurationDiff(config); + if (diff.empty()) + LOG_TRACE(log, "Configuration update triggered, but nothing changed for RAFT"); + else if (diff.size() > 1) + LOG_WARNING(log, "Configuration changed for more than one server ({}) from cluster, it's strictly not recommended", diff.size()); else + LOG_DEBUG(log, "Configuration change size ({})", diff.size()); + + for (auto & change : diff) { - LOG_INFO(log, "Configuration changed, but we are not leader, so we will wait update from leader"); + bool push_result = update_configuration_queue.push(change); + if (!push_result) + throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push configuration update to queue"); } - } } diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index 4274d76fa18..bb1d41b8cfc 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -33,11 +33,13 @@ private: CoordinationSettingsPtr coordination_settings; using RequestsQueue = ConcurrentBoundedQueue; using SessionToResponseCallback = std::unordered_map; + using UpdateConfigurationQueue = ConcurrentBoundedQueue; /// Size depends on coordination settings std::unique_ptr requests_queue; ResponsesQueue responses_queue; SnapshotsQueue snapshots_queue{1}; + UpdateConfigurationQueue update_configuration_queue{1000}; std::atomic shutdown_called{false}; @@ -62,6 +64,9 @@ private: ThreadFromGlobalPool session_cleaner_thread; /// Dumping new snapshots to disk ThreadFromGlobalPool snapshot_thread; + + ThreadFromGlobalPool update_configuration_thread; + /// RAFT wrapper. std::unique_ptr server; @@ -80,6 +85,8 @@ private: /// Thread create snapshots in the background void snapshotThread(); + void updateConfigurationThread(); + void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response); /// Add error responses for requests to responses queue. diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index f3f948fd186..b06301f2e67 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -127,33 +127,36 @@ void KeeperServer::startup() if (latest_snapshot_config && latest_log_store_config) { if (latest_snapshot_config->get_log_idx() > latest_log_store_config->get_log_idx()) + { + LOG_INFO(log, "Will use config from snapshot with log index {}", latest_snapshot_config->get_log_idx()); state_manager->setClusterConfig(latest_snapshot_config); + } else + { + LOG_INFO(log, "Will use config from log store with log index {}", latest_snapshot_config->get_log_idx()); state_manager->setClusterConfig(latest_log_store_config); + } } else if (latest_snapshot_config) - state_manager->setClusterConfig(latest_snapshot_config); - else if (latest_log_store_config) - state_manager->setClusterConfig(latest_log_store_config); -/// else use parsed config in state_manager constructor (first start) - - bool single_server = state_manager->getTotalServers() == 1; - - nuraft::raft_params params; - if (single_server) { - /// Don't make sense in single server mode - params.heart_beat_interval_ = 0; - params.election_timeout_lower_bound_ = 0; - params.election_timeout_upper_bound_ = 0; + LOG_INFO(log, "No config in log store, will use config from snapshot with log index {}", latest_snapshot_config->get_log_idx()); + state_manager->setClusterConfig(latest_snapshot_config); + } + else if (latest_log_store_config) + { + LOG_INFO(log, "No config in snapshot, will use config from log store with log index {}", latest_log_store_config->get_log_idx()); + state_manager->setClusterConfig(latest_log_store_config); } else { - params.heart_beat_interval_ = coordination_settings->heart_beat_interval_ms.totalMilliseconds(); - params.election_timeout_lower_bound_ = coordination_settings->election_timeout_lower_bound_ms.totalMilliseconds(); - params.election_timeout_upper_bound_ = coordination_settings->election_timeout_upper_bound_ms.totalMilliseconds(); + LOG_INFO(log, "No config in log store and snapshot, probably it's initial run. Will use config from .xml on disk"); } + nuraft::raft_params params; + params.heart_beat_interval_ = coordination_settings->heart_beat_interval_ms.totalMilliseconds(); + params.election_timeout_lower_bound_ = coordination_settings->election_timeout_lower_bound_ms.totalMilliseconds(); + params.election_timeout_upper_bound_ = coordination_settings->election_timeout_upper_bound_ms.totalMilliseconds(); + params.reserved_log_items_ = coordination_settings->reserved_log_items; params.snapshot_distance_ = coordination_settings->snapshot_distance; params.stale_log_gap_ = coordination_settings->stale_log_gap; @@ -380,43 +383,84 @@ std::vector KeeperServer::getDeadSessions() return state_machine->getDeadSessions(); } -void KeeperServer::updateConfiguration(const Poco::Util::AbstractConfiguration & config) +ConfigUpdateActions KeeperServer::getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) { - auto diff = state_manager->getConfigurationDiff(config); - if (diff.empty()) - { - LOG_TRACE(log, "Configuration update triggered, but nothing changed for RAFT"); - return; - } - else if (diff.size() > 1) - { - LOG_WARNING(log, "Configuration changed for more than one server ({}) from cluster, it's strictly not recommended", diff.size()); - } - else - { - LOG_WARNING(log, "Configuration change size ({})", diff.size()); - } + return state_manager->getConfigurationDiff(config); +} - for (auto & task : diff) +void KeeperServer::applyConfigurationUpdate(const ConfigUpdateAction & task) +{ + size_t sleep_ms = 500; + if (task.action_type == ConfigUpdateActionType::AddServer) { - if (task.action_type == ConfigUpdateActionType::AddServer) + LOG_INFO(log, "Will try to add server with id {}", task.server->get_id()); + bool added = false; + for (size_t i = 0; i < coordination_settings->configuration_change_tries_count; ++i) { + if (raft_instance->get_srv_config(task.server->get_id()) != nullptr) + { + LOG_INFO(log, "Server with id {} was successfully added", task.server->get_id()); + added = true; + break; + } + + if (!isLeader()) + { + LOG_INFO(log, "We are not leader anymore, will not try to add server {}", task.server->get_id()); + break; + } + auto result = raft_instance->add_srv(*task.server); if (!result->get_accepted()) - throw Exception(ErrorCodes::RAFT_ERROR, "Configuration change to add server (id {}) was not accepted by RAFT", task.server->get_id()); + LOG_INFO(log, "Command to add server {} was not accepted for the {} time, will sleep for {} ms and retry", task.server->get_id(), i + 1, sleep_ms * (i + 1)); + + std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1))); } - else if (task.action_type == ConfigUpdateActionType::RemoveServer) + if (!added) + throw Exception(ErrorCodes::RAFT_ERROR, "Configuration change to add server (id {}) was not accepted by RAFT after all {} retries", task.server->get_id(), coordination_settings->configuration_change_tries_count); + } + else if (task.action_type == ConfigUpdateActionType::RemoveServer) + { + LOG_INFO(log, "Will try to remove server with id {}", task.server->get_id()); + + bool removed = false; + if (task.server->get_id() == stage_manager->server_id()) { + LOG_INFO(log, "Trying to remove leader node (ourself), so will yield leadership and some other node (new leader) will try remove us. " + "Probably you will have to run SYSTEM RELOAD CONFIG on the new leader node"); + + raft_instance->yield_leadership(); + return; + } + + for (size_t i = 0; i < coordination_settings->configuration_change_tries_count; ++i) + { + if (raft_instance->get_srv_config(task.server->get_id()) == nullptr) + { + LOG_INFO(log, "Server with id {} was successfully removed", task.server->get_id()); + removed = true; + break; + } + + if (!isLeader()) + { + LOG_INFO(log, "We are not leader anymore, will not try to remove server {}", task.server->get_id()); + break; + } + auto result = raft_instance->remove_srv(task.server->get_id()); if (!result->get_accepted()) - throw Exception(ErrorCodes::RAFT_ERROR, "Configuration change to remove server (id {}) was not accepted by RAFT", task.server->get_id()); + LOG_INFO(log, "Command to remove server {} was not accepted for the {} time, will sleep for {} ms and retry", task.server->get_id(), i + 1, sleep_ms * (i + 1)); + + std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1))); } - else if (task.action_type == ConfigUpdateActionType::UpdatePriority) - raft_instance->set_priority(task.server->get_id(), task.server->get_priority()); - else - LOG_WARNING(log, "Unknown configuration update type {}", static_cast(task.action_type)); + if (!removed) + throw Exception(ErrorCodes::RAFT_ERROR, "Configuration change to remove server (id {}) was not accepted by RAFT after all {} retries", task.server->get_id(), coordination_settings->configuration_change_tries_count); } - + else if (task.action_type == ConfigUpdateActionType::UpdatePriority) + raft_instance->set_priority(task.server->get_id(), task.server->get_priority()); + else + LOG_WARNING(log, "Unknown configuration update type {}", static_cast(task.action_type)); } } diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 3a9d04643b1..6db146e9881 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -90,7 +90,9 @@ public: int getServerID() const { return server_id; } - void updateConfiguration(const Poco::Util::AbstractConfiguration & config); + ConfigUpdateActions getConfigurationDiff(const Poco::Util::AbstractConfiguration & config); + + void applyConfigurationUpdate(const ConfigUpdateAction & action); }; } diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 0039c08c58c..17db2b43dba 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -81,13 +81,12 @@ KeeperStateManager::KeeperStateManager(int server_id_, const std::string & host, : my_server_id(server_id_) , secure(false) , log_store(nuraft::cs_new(logs_path, 5000, false, false)) - , log(&Poco::Logger::get("KeeperStateManager")) { auto peer_config = nuraft::cs_new(my_server_id, host + ":" + std::to_string(port)); - servers_configuration.cluster_config = nuraft::cs_new(); - servers_configuration.port = port; - servers_configuration.config = peer_config; - servers_configuration.cluster_config->get_servers().push_back(peer_config); + configuration_wrapper.cluster_config = nuraft::cs_new(); + configuration_wrapper.port = port; + configuration_wrapper.config = peer_config; + configuration_wrapper.cluster_config->get_servers().push_back(peer_config); } KeeperStateManager::KeeperStateManager( @@ -99,11 +98,10 @@ KeeperStateManager::KeeperStateManager( : my_server_id(server_id_) , secure(config.getBool(config_prefix_ + ".raft_configuration.secure", false)) , config_prefix(config_prefix_) - , servers_configuration(parseServersConfiguration(config)) + , configuration_wrapper(parseServersConfiguration(config)) , log_store(nuraft::cs_new( getLogsPathFromConfig(config_prefix_, config, standalone_keeper), coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync, coordination_settings->compress_logs)) - , log(&Poco::Logger::get("KeeperStateManager")) { } @@ -122,7 +120,7 @@ ClusterConfigPtr KeeperStateManager::getLatestConfigFromLogStore() const void KeeperStateManager::setClusterConfig(const ClusterConfigPtr & cluster_config) { - servers_configuration.cluster_config = cluster_config; + configuration_wrapper.cluster_config = cluster_config; } void KeeperStateManager::flushLogStore() @@ -133,7 +131,7 @@ void KeeperStateManager::flushLogStore() void KeeperStateManager::save_config(const nuraft::cluster_config & config) { nuraft::ptr buf = config.serialize(); - servers_configuration.cluster_config = nuraft::cluster_config::deserialize(*buf); + configuration_wrapper.cluster_config = nuraft::cluster_config::deserialize(*buf); } void KeeperStateManager::save_state(const nuraft::srv_state & state) @@ -144,51 +142,42 @@ void KeeperStateManager::save_state(const nuraft::srv_state & state) ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const { - auto new_servers_configuration = parseServersConfiguration(config); - if (new_servers_configuration.port != servers_configuration.port) + auto new_configuration_wrapper = parseServersConfiguration(config); + if (new_configuration_wrapper.port != configuration_wrapper.port) throw Exception(ErrorCodes::RAFT_ERROR, "Cannot change port of already running RAFT server"); std::unordered_map new_ids, old_ids; - for (auto new_server : new_servers_configuration.cluster_config->get_servers()) - { - LOG_INFO(log, "NEW SERVER {}", new_server->get_id()); + for (auto new_server : new_configuration_wrapper.cluster_config->get_servers()) new_ids[new_server->get_id()] = new_server; - } - for (auto old_server : servers_configuration.cluster_config->get_servers()) - { - LOG_INFO(log, "OLD SERVER {}", old_server->get_id()); + for (auto old_server : configuration_wrapper.cluster_config->get_servers()) old_ids[old_server->get_id()] = old_server; - } ConfigUpdateActions result; - for (auto [old_id, server_config] : old_ids) - { - if (!new_ids.count(old_id)) - { - LOG_INFO(log, "REMOVING SERVER {}", old_id); - result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::RemoveServer, server_config}); - } - } for (auto [new_id, server_config] : new_ids) { if (!old_ids.count(new_id)) - { - LOG_INFO(log, "ADDING SERVER {}", new_id); result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::AddServer, server_config}); - } } - for (const auto & old_server : servers_configuration.cluster_config->get_servers()) + for (auto [old_id, server_config] : old_ids) { - for (const auto & new_server : new_servers_configuration.cluster_config->get_servers()) + if (!new_ids.count(old_id)) + result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::RemoveServer, server_config}); + } + + /// And update priority if required + for (const auto & old_server : configuration_wrapper.cluster_config->get_servers()) + { + for (const auto & new_server : new_configuration_wrapper.cluster_config->get_servers()) { if (old_server->get_id() == new_server->get_id()) { - LOG_INFO(log, "UPDATE PRIORITY {}", new_server->get_id()); if (old_server->get_priority() != new_server->get_priority()) + { result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::UpdatePriority, new_server}); + } break; } } diff --git a/src/Coordination/KeeperStateManager.h b/src/Coordination/KeeperStateManager.h index ccef1fc68c3..181535c4b05 100644 --- a/src/Coordination/KeeperStateManager.h +++ b/src/Coordination/KeeperStateManager.h @@ -13,7 +13,7 @@ namespace DB using KeeperServerConfigPtr = nuraft::ptr; -struct KeeperServersConfiguration +struct KeeperConfigurationWrapper { int port; KeeperServerConfigPtr config; @@ -56,7 +56,7 @@ public: void flushLogStore(); - nuraft::ptr load_config() override { return servers_configuration.cluster_config; } + nuraft::ptr load_config() override { return configuration_wrapper.cluster_config; } void save_config(const nuraft::cluster_config & config) override; @@ -68,15 +68,15 @@ public: int32_t server_id() override { return my_server_id; } - nuraft::ptr get_srv_config() const { return servers_configuration.config; } + nuraft::ptr get_srv_config() const { return configuration_wrapper.config; } void system_exit(const int /* exit_code */) override {} - int getPort() const { return servers_configuration.port; } + int getPort() const { return configuration_wrapper.port; } bool shouldStartAsFollower() const { - return servers_configuration.servers_start_as_followers.count(my_server_id); + return configuration_wrapper.servers_start_as_followers.count(my_server_id); } bool isSecure() const @@ -86,7 +86,7 @@ public: nuraft::ptr getLogStore() const { return log_store; } - uint64_t getTotalServers() const { return servers_configuration.cluster_config->get_servers().size(); } + uint64_t getTotalServers() const { return configuration_wrapper.cluster_config->get_servers().size(); } ClusterConfigPtr getLatestConfigFromLogStore() const; @@ -98,13 +98,12 @@ private: int my_server_id; bool secure; std::string config_prefix; - KeeperServersConfiguration servers_configuration; + + KeeperConfigurationWrapper configuration_wrapper; nuraft::ptr log_store; nuraft::ptr server_state; - Poco::Logger * log; - - KeeperServersConfiguration parseServersConfiguration(const Poco::Util::AbstractConfiguration & config) const; + KeeperConfigurationWrapper parseServersConfiguration(const Poco::Util::AbstractConfiguration & config) const; }; } diff --git a/tests/integration/test_keeper_nodes_add/__init__.py b/tests/integration/test_keeper_nodes_add/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_keeper_nodes_add/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper1.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper1.xml new file mode 100644 index 00000000000..bd695389fa6 --- /dev/null +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper1.xml @@ -0,0 +1,22 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + + diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_1.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_1.xml new file mode 100644 index 00000000000..510424715c4 --- /dev/null +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_1.xml @@ -0,0 +1,32 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + + + 3 + node3 + 44444 + + + + diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_2.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_2.xml new file mode 100644 index 00000000000..264601d8c98 --- /dev/null +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_2.xml @@ -0,0 +1,32 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + + + 3 + node3 + 44444 + + + + diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_3.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_3.xml new file mode 100644 index 00000000000..7f9775939bb --- /dev/null +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_three_nodes_3.xml @@ -0,0 +1,32 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + + + 3 + node3 + 44444 + + + + diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_1.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_1.xml new file mode 100644 index 00000000000..3f6baf1ccc7 --- /dev/null +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_1.xml @@ -0,0 +1,27 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + + + + diff --git a/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_2.xml b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_2.xml new file mode 100644 index 00000000000..61c9a00ffd6 --- /dev/null +++ b/tests/integration/test_keeper_nodes_add/configs/enable_keeper_two_nodes_2.xml @@ -0,0 +1,27 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + + + + diff --git a/tests/integration/test_keeper_nodes_add/test.py b/tests/integration/test_keeper_nodes_add/test.py new file mode 100644 index 00000000000..ae4a996f6b3 --- /dev/null +++ b/tests/integration/test_keeper_nodes_add/test.py @@ -0,0 +1,80 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +import random +import string +import os +import time +from multiprocessing.dummy import Pool +from helpers.network import PartitionManager +from helpers.test_tools import assert_eq_with_retry +from kazoo.client import KazooClient, KazooState + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), 'configs') + +node1 = cluster.add_instance('node1', main_configs=['configs/enable_keeper1.xml'], stay_alive=True) +node2 = cluster.add_instance('node2', main_configs=[], stay_alive=True) +node3 = cluster.add_instance('node3', main_configs=[], stay_alive=True) + + +def get_fake_zk(node, timeout=30.0): + _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(node.name) + ":9181", timeout=timeout) + _fake_zk_instance.start() + return _fake_zk_instance + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + +def start(node): + node.start_clickhouse() + + +def test_nodes_add(started_cluster): + zk_conn = get_fake_zk(node1) + + for i in range(100): + zk_conn.create("/test_two_" + str(i), b"somedata") + + p = Pool(3) + node2.stop_clickhouse() + node2.copy_file_to_container(os.path.join(CONFIG_DIR, "enable_keeper_two_nodes_2.xml"), "/etc/clickhouse-server/config.d/enable_keeper2.xml") + waiter = p.apply_async(start, (node2,)) + node1.copy_file_to_container(os.path.join(CONFIG_DIR, "enable_keeper_two_nodes_1.xml"), "/etc/clickhouse-server/config.d/enable_keeper1.xml") + node1.query("SYSTEM RELOAD CONFIG") + waiter.wait() + + zk_conn2 = get_fake_zk(node2) + + for i in range(100): + assert zk_conn2.exists("/test_two_" + str(i)) is not None + + zk_conn = get_fake_zk(node1) + + for i in range(100): + zk_conn.create("/test_three_" + str(i), b"somedata") + + node3.stop_clickhouse() + + node3.copy_file_to_container(os.path.join(CONFIG_DIR, "enable_keeper_three_nodes_3.xml"), "/etc/clickhouse-server/config.d/enable_keeper3.xml") + waiter = p.apply_async(start, (node3,)) + node2.copy_file_to_container(os.path.join(CONFIG_DIR, "enable_keeper_three_nodes_2.xml"), "/etc/clickhouse-server/config.d/enable_keeper2.xml") + node1.copy_file_to_container(os.path.join(CONFIG_DIR, "enable_keeper_three_nodes_1.xml"), "/etc/clickhouse-server/config.d/enable_keeper1.xml") + + node1.query("SYSTEM RELOAD CONFIG") + node2.query("SYSTEM RELOAD CONFIG") + + waiter.wait() + zk_conn3 = get_fake_zk(node3) + + for i in range(100): + assert zk_conn3.exists("/test_three_" + str(i)) is not None diff --git a/tests/integration/test_keeper_nodes_move/__init__.py b/tests/integration/test_keeper_nodes_move/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_keeper_nodes_move/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_keeper_nodes_move/configs/enable_keeper1.xml b/tests/integration/test_keeper_nodes_move/configs/enable_keeper1.xml new file mode 100644 index 00000000000..a8a1a17b831 --- /dev/null +++ b/tests/integration/test_keeper_nodes_move/configs/enable_keeper1.xml @@ -0,0 +1,34 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + true + + + 3 + node3 + 44444 + true + + + + diff --git a/tests/integration/test_keeper_nodes_move/configs/enable_keeper2.xml b/tests/integration/test_keeper_nodes_move/configs/enable_keeper2.xml new file mode 100644 index 00000000000..7820158680c --- /dev/null +++ b/tests/integration/test_keeper_nodes_move/configs/enable_keeper2.xml @@ -0,0 +1,34 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + true + + + 3 + node3 + 44444 + true + + + + diff --git a/tests/integration/test_keeper_nodes_move/configs/enable_keeper3.xml b/tests/integration/test_keeper_nodes_move/configs/enable_keeper3.xml new file mode 100644 index 00000000000..cd76b0d216e --- /dev/null +++ b/tests/integration/test_keeper_nodes_move/configs/enable_keeper3.xml @@ -0,0 +1,34 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + true + + + 3 + node3 + 44444 + true + + + + diff --git a/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_1.xml b/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_1.xml new file mode 100644 index 00000000000..d99502e2959 --- /dev/null +++ b/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_1.xml @@ -0,0 +1,34 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + true + + + 4 + node4 + 44444 + true + + + + diff --git a/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_2.xml b/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_2.xml new file mode 100644 index 00000000000..04d8776782f --- /dev/null +++ b/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_2.xml @@ -0,0 +1,34 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + true + + + 4 + node4 + 44444 + true + + + + diff --git a/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_4.xml b/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_4.xml new file mode 100644 index 00000000000..10abf71b9f6 --- /dev/null +++ b/tests/integration/test_keeper_nodes_move/configs/enable_keeper_node4_4.xml @@ -0,0 +1,34 @@ + + + 9181 + 4 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + true + + + 4 + node4 + 44444 + true + + + + diff --git a/tests/integration/test_keeper_nodes_move/test.py b/tests/integration/test_keeper_nodes_move/test.py new file mode 100644 index 00000000000..e3f1a161b07 --- /dev/null +++ b/tests/integration/test_keeper_nodes_move/test.py @@ -0,0 +1,85 @@ +#!/usr/bin/env python3 + + +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +import random +import string +import os +import time +from multiprocessing.dummy import Pool +from helpers.network import PartitionManager +from helpers.test_tools import assert_eq_with_retry +from kazoo.client import KazooClient, KazooState + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), 'configs') + +node1 = cluster.add_instance('node1', main_configs=['configs/enable_keeper1.xml'], stay_alive=True) +node2 = cluster.add_instance('node2', main_configs=['configs/enable_keeper2.xml'], stay_alive=True) +node3 = cluster.add_instance('node3', main_configs=['configs/enable_keeper3.xml'], stay_alive=True) +node4 = cluster.add_instance('node4', stay_alive=True) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def start(node): + node.start_clickhouse() + +def get_fake_zk(node, timeout=30.0): + _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(node.name) + ":9181", timeout=timeout) + _fake_zk_instance.start() + return _fake_zk_instance + + +def test_node_move(started_cluster): + zk_conn = get_fake_zk(node1) + + for i in range(100): + zk_conn.create("/test_four_" + str(i), b"somedata") + + zk_conn2 = get_fake_zk(node2) + zk_conn2.sync("/test_four_0") + + zk_conn3 = get_fake_zk(node3) + zk_conn3.sync("/test_four_0") + + for i in range(100): + assert zk_conn2.exists("test_four_" + str(i)) is not None + assert zk_conn3.exists("test_four_" + str(i)) is not None + + node4.stop_clickhouse() + node4.copy_file_to_container(os.path.join(CONFIG_DIR, "enable_keeper_node4_4.xml"), "/etc/clickhouse-server/config.d/enable_keeper4.xml") + p = Pool(3) + waiter = p.apply_async(start, (node4,)) + node1.copy_file_to_container(os.path.join(CONFIG_DIR, "enable_keeper_node4_1.xml"), "/etc/clickhouse-server/config.d/enable_keeper1.xml") + node2.copy_file_to_container(os.path.join(CONFIG_DIR, "enable_keeper_node4_2.xml"), "/etc/clickhouse-server/config.d/enable_keeper2.xml") + + node1.query("SYSTEM RELOAD CONFIG") + node2.query("SYSTEM RELOAD CONFIG") + + waiter.wait() + + zk_conn4 = get_fake_zk(node4) + zk_conn4.sync("/test_four_0") + + for i in range(100): + assert zk_conn4.exists("/test_four_" + str(i)) is not None + + with pytest.raises(Exception): + # Adding and removing nodes is async operation + for i in range(10): + zk_conn3 = get_fake_zk(node3) + zk_conn3.sync("/test_four_0") + time.sleep(i) diff --git a/tests/integration/test_keeper_nodes_remove/__init__.py b/tests/integration/test_keeper_nodes_remove/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_keeper_nodes_remove/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper1.xml b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper1.xml new file mode 100644 index 00000000000..a8a1a17b831 --- /dev/null +++ b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper1.xml @@ -0,0 +1,34 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + true + + + 3 + node3 + 44444 + true + + + + diff --git a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper2.xml b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper2.xml new file mode 100644 index 00000000000..7820158680c --- /dev/null +++ b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper2.xml @@ -0,0 +1,34 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + true + + + 3 + node3 + 44444 + true + + + + diff --git a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper3.xml b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper3.xml new file mode 100644 index 00000000000..cd76b0d216e --- /dev/null +++ b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper3.xml @@ -0,0 +1,34 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + true + + + 3 + node3 + 44444 + true + + + + diff --git a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_1.xml b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_1.xml new file mode 100644 index 00000000000..3f6baf1ccc7 --- /dev/null +++ b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_1.xml @@ -0,0 +1,27 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + + + + diff --git a/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_2.xml b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_2.xml new file mode 100644 index 00000000000..61c9a00ffd6 --- /dev/null +++ b/tests/integration/test_keeper_nodes_remove/configs/enable_keeper_two_nodes_2.xml @@ -0,0 +1,27 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + 2 + node2 + 44444 + + + + diff --git a/tests/integration/test_keeper_nodes_remove/configs/enable_single_keeper1.xml b/tests/integration/test_keeper_nodes_remove/configs/enable_single_keeper1.xml new file mode 100644 index 00000000000..bd695389fa6 --- /dev/null +++ b/tests/integration/test_keeper_nodes_remove/configs/enable_single_keeper1.xml @@ -0,0 +1,22 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + + + + diff --git a/tests/integration/test_keeper_nodes_remove/test.py b/tests/integration/test_keeper_nodes_remove/test.py new file mode 100644 index 00000000000..72b02c0661f --- /dev/null +++ b/tests/integration/test_keeper_nodes_remove/test.py @@ -0,0 +1,97 @@ +#!/usr/bin/env python3 + +import pytest +from helpers.cluster import ClickHouseCluster +import random +import string +import os +import time +from multiprocessing.dummy import Pool +from helpers.network import PartitionManager +from helpers.test_tools import assert_eq_with_retry +from kazoo.client import KazooClient, KazooState + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), 'configs') + +node1 = cluster.add_instance('node1', main_configs=['configs/enable_keeper1.xml'], stay_alive=True) +node2 = cluster.add_instance('node2', main_configs=['configs/enable_keeper2.xml'], stay_alive=True) +node3 = cluster.add_instance('node3', main_configs=['configs/enable_keeper3.xml'], stay_alive=True) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def start(node): + node.start_clickhouse() + + +def get_fake_zk(node, timeout=30.0): + _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(node.name) + ":9181", timeout=timeout) + _fake_zk_instance.start() + return _fake_zk_instance + + +def test_nodes_remove(started_cluster): + zk_conn = get_fake_zk(node1) + + for i in range(100): + zk_conn.create("/test_two_" + str(i), b"somedata") + + zk_conn2 = get_fake_zk(node2) + zk_conn2.sync("/test_two_0") + + zk_conn3 = get_fake_zk(node3) + zk_conn3.sync("/test_two_0") + + for i in range(100): + assert zk_conn2.exists("test_two_" + str(i)) is not None + assert zk_conn3.exists("test_two_" + str(i)) is not None + + node2.copy_file_to_container(os.path.join(CONFIG_DIR, "enable_keeper_two_nodes_2.xml"), "/etc/clickhouse-server/config.d/enable_keeper2.xml") + node1.copy_file_to_container(os.path.join(CONFIG_DIR, "enable_keeper_two_nodes_1.xml"), "/etc/clickhouse-server/config.d/enable_keeper1.xml") + + node1.query("SYSTEM RELOAD CONFIG") + node2.query("SYSTEM RELOAD CONFIG") + + zk_conn2 = get_fake_zk(node2) + + for i in range(100): + assert zk_conn2.exists("test_two_" + str(i)) is not None + zk_conn2.create("/test_two_" + str(100 + i), b"otherdata") + + zk_conn = get_fake_zk(node1) + zk_conn.sync("/test_two_0") + + for i in range(100): + assert zk_conn.exists("test_two_" + str(i)) is not None + assert zk_conn.exists("test_two_" + str(100 + i)) is not None + + with pytest.raises(Exception): + zk_conn3 = get_fake_zk(node3) + zk_conn3.sync("/test_two_0") + + node3.stop_clickhouse() + + node1.copy_file_to_container(os.path.join(CONFIG_DIR, "enable_single_keeper1.xml"), "/etc/clickhouse-server/config.d/enable_keeper1.xml") + + node1.query("SYSTEM RELOAD CONFIG") + zk_conn = get_fake_zk(node1) + zk_conn.sync("/test_two_0") + + for i in range(100): + assert zk_conn.exists("test_two_" + str(i)) is not None + assert zk_conn.exists("test_two_" + str(100 + i)) is not None + + with pytest.raises(Exception): + zk_conn2 = get_fake_zk(node2) + zk_conn2.sync("/test_two_0") + + node2.stop_clickhouse() From 647856658f50b61e259f2ec7072ed2f4ea75c119 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Oct 2021 16:11:29 +0300 Subject: [PATCH 215/919] Add synchronization and active wait --- src/Coordination/KeeperDispatcher.cpp | 36 +++++++------- src/Coordination/KeeperDispatcher.h | 8 ++- src/Coordination/KeeperServer.cpp | 65 +++++++++++++++++++++++-- src/Coordination/KeeperServer.h | 6 +++ src/Coordination/KeeperStateManager.cpp | 40 ++++++++------- src/Coordination/KeeperStateManager.h | 39 +++++++++++++-- 6 files changed, 147 insertions(+), 47 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 53d8b5e3585..c29ea4fac45 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -530,24 +530,26 @@ void KeeperDispatcher::updateConfigurationThread() ConfigUpdateAction action; if (!update_configuration_queue.pop(action)) break; - /// Only leader node must check dead sessions - if (isLeader()) - { - server->applyConfigurationUpdate(action); - } - else - { - String message; - if (action.type == ConfigUpdateActionType::RemoveServer) - message += "remove"; - else if (action.type == ConfigUpdateActionType::AddServer) - message += "add"; - else if (action.type == ConfigUpdateActionType::UpdatePriority) - message += "update priority for"; - else - message += "unknown action for"; - LOG_INFO(log, "Configuration changed ({} server {}), but we are not leader, so we will wait update from leader", message, task.server->get_id()); + + /// We must wait this update from leader or apply it ourself (if we are leader) + bool done = false; + while (!done) + { + if (shutdown_called) + return; + + if (isLeader()) + { + server->applyConfigurationUpdate(action); + done = true; + } + else + { + done = server->waitConfigurationUpdate(action); + if (!done) + LOG_INFO(log, "Cannot wait for configuration update, maybe we become leader, or maybe update is invalid, will try to wait one more time"); + } } } catch (...) diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index bb1d41b8cfc..e958ed2c623 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -39,6 +39,8 @@ private: std::unique_ptr requests_queue; ResponsesQueue responses_queue; SnapshotsQueue snapshots_queue{1}; + + /// More than 1k updates is definitely misconfiguration. UpdateConfigurationQueue update_configuration_queue{1000}; std::atomic shutdown_called{false}; @@ -64,7 +66,7 @@ private: ThreadFromGlobalPool session_cleaner_thread; /// Dumping new snapshots to disk ThreadFromGlobalPool snapshot_thread; - + /// Apply or wait for configuration changes ThreadFromGlobalPool update_configuration_thread; /// RAFT wrapper. @@ -84,7 +86,7 @@ private: void sessionCleanerTask(); /// Thread create snapshots in the background void snapshotThread(); - + /// Thread apply or wait configuration changes from leader void updateConfigurationThread(); void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response); @@ -113,6 +115,8 @@ public: return server && server->checkInit(); } + /// Registered in ConfigReloader callback. Add new confugration changes to + /// update_configuration_queue. Keeper Dispatcher apply them asynchronously. void updateConfiguration(const Poco::Util::AbstractConfiguration & config); /// Shutdown internal keeper parts (server, state machine, log storage, etc) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index b06301f2e67..bd0f06d3ad4 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -129,23 +129,23 @@ void KeeperServer::startup() if (latest_snapshot_config->get_log_idx() > latest_log_store_config->get_log_idx()) { LOG_INFO(log, "Will use config from snapshot with log index {}", latest_snapshot_config->get_log_idx()); - state_manager->setClusterConfig(latest_snapshot_config); + state_manager->save_config(*latest_snapshot_config); } else { LOG_INFO(log, "Will use config from log store with log index {}", latest_snapshot_config->get_log_idx()); - state_manager->setClusterConfig(latest_log_store_config); + state_manager->save_config(*latest_log_store_config); } } else if (latest_snapshot_config) { LOG_INFO(log, "No config in log store, will use config from snapshot with log index {}", latest_snapshot_config->get_log_idx()); - state_manager->setClusterConfig(latest_snapshot_config); + state_manager->save_config(*latest_snapshot_config); } else if (latest_log_store_config) { LOG_INFO(log, "No config in snapshot, will use config from log store with log index {}", latest_log_store_config->get_log_idx()); - state_manager->setClusterConfig(latest_log_store_config); + state_manager->save_config(*latest_log_store_config); } else { @@ -424,7 +424,7 @@ void KeeperServer::applyConfigurationUpdate(const ConfigUpdateAction & task) LOG_INFO(log, "Will try to remove server with id {}", task.server->get_id()); bool removed = false; - if (task.server->get_id() == stage_manager->server_id()) + if (task.server->get_id() == state_manager->server_id()) { LOG_INFO(log, "Trying to remove leader node (ourself), so will yield leadership and some other node (new leader) will try remove us. " "Probably you will have to run SYSTEM RELOAD CONFIG on the new leader node"); @@ -463,4 +463,59 @@ void KeeperServer::applyConfigurationUpdate(const ConfigUpdateAction & task) LOG_WARNING(log, "Unknown configuration update type {}", static_cast(task.action_type)); } + +bool KeeperServer::waitConfigurationUpdate(const ConfigUpdateAction & task) +{ + + size_t sleep_ms = 500; + if (task.action_type == ConfigUpdateActionType::AddServer) + { + LOG_INFO(log, "Will try to wait server with id {} to be added", task.server->get_id()); + for (size_t i = 0; i < coordination_settings->configuration_change_tries_count; ++i) + { + if (raft_instance->get_srv_config(task.server->get_id()) != nullptr) + { + LOG_INFO(log, "Server with id {} was successfully added", task.server->get_id()); + return true; + } + + if (isLeader()) + { + LOG_INFO(log, "We are leader now, probably we will have to add server {}", task.server->get_id()); + return false; + } + + std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1))); + } + return false; + } + else if (task.action_type == ConfigUpdateActionType::RemoveServer) + { + LOG_INFO(log, "Will try to remove server with id {}", task.server->get_id()); + + for (size_t i = 0; i < coordination_settings->configuration_change_tries_count; ++i) + { + if (raft_instance->get_srv_config(task.server->get_id()) == nullptr) + { + LOG_INFO(log, "Server with id {} was successfully removed", task.server->get_id()); + return true; + } + + if (isLeader()) + { + LOG_INFO(log, "We are leader now, probably we will have to remove server {}", task.server->get_id()); + return false; + } + + std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1))); + } + return false; + } + else if (task.action_type == ConfigUpdateActionType::UpdatePriority) + return true; + else + LOG_WARNING(log, "Unknown configuration update type {}", static_cast(task.action_type)); + return true; +} + } diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 6db146e9881..fd49b3ceab5 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -90,9 +90,15 @@ public: int getServerID() const { return server_id; } + /// Get configuration diff between current configuration in RAFT and in XML file ConfigUpdateActions getConfigurationDiff(const Poco::Util::AbstractConfiguration & config); + /// Apply action fo configuration update. Actually call raft_instance->remove_srv or raft_instance->add_srv. + /// Synchronously check for update results with retries. void applyConfigurationUpdate(const ConfigUpdateAction & action); + + + bool waitConfigurationUpdate(const ConfigUpdateAction & action); }; } diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 17db2b43dba..8b95778b65a 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -32,9 +32,9 @@ namespace } -KeeperServersConfiguration KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfiguration & config) const +KeeperConfigurationWrapper KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfiguration & config) const { - KeeperServersConfiguration result; + KeeperConfigurationWrapper result; result.cluster_config = std::make_shared(); Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix + ".raft_configuration", keys); @@ -118,11 +118,6 @@ ClusterConfigPtr KeeperStateManager::getLatestConfigFromLogStore() const return nullptr; } -void KeeperStateManager::setClusterConfig(const ClusterConfigPtr & cluster_config) -{ - configuration_wrapper.cluster_config = cluster_config; -} - void KeeperStateManager::flushLogStore() { log_store->flush(); @@ -130,14 +125,15 @@ void KeeperStateManager::flushLogStore() void KeeperStateManager::save_config(const nuraft::cluster_config & config) { + std::lock_guard lock(configuration_wrapper_mutex); nuraft::ptr buf = config.serialize(); configuration_wrapper.cluster_config = nuraft::cluster_config::deserialize(*buf); } void KeeperStateManager::save_state(const nuraft::srv_state & state) { - nuraft::ptr buf = state.serialize(); - server_state = nuraft::srv_state::deserialize(*buf); + nuraft::ptr buf = state.serialize(); + server_state = nuraft::srv_state::deserialize(*buf); } ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const @@ -150,35 +146,43 @@ ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::A for (auto new_server : new_configuration_wrapper.cluster_config->get_servers()) new_ids[new_server->get_id()] = new_server; - for (auto old_server : configuration_wrapper.cluster_config->get_servers()) - old_ids[old_server->get_id()] = old_server; + { + std::lock_guard lock(configuration_wrapper_mutex); + for (auto old_server : configuration_wrapper.cluster_config->get_servers()) + old_ids[old_server->get_id()] = old_server; + } ConfigUpdateActions result; + /// First of all add new servers for (auto [new_id, server_config] : new_ids) { if (!old_ids.count(new_id)) result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::AddServer, server_config}); } + /// After that remove old ones for (auto [old_id, server_config] : old_ids) { if (!new_ids.count(old_id)) result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::RemoveServer, server_config}); } - /// And update priority if required - for (const auto & old_server : configuration_wrapper.cluster_config->get_servers()) { - for (const auto & new_server : new_configuration_wrapper.cluster_config->get_servers()) + std::lock_guard lock(configuration_wrapper_mutex); + /// And update priority if required + for (const auto & old_server : configuration_wrapper.cluster_config->get_servers()) { - if (old_server->get_id() == new_server->get_id()) + for (const auto & new_server : new_configuration_wrapper.cluster_config->get_servers()) { - if (old_server->get_priority() != new_server->get_priority()) + if (old_server->get_id() == new_server->get_id()) { - result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::UpdatePriority, new_server}); + if (old_server->get_priority() != new_server->get_priority()) + { + result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::UpdatePriority, new_server}); + } + break; } - break; } } } diff --git a/src/Coordination/KeeperStateManager.h b/src/Coordination/KeeperStateManager.h index 181535c4b05..f42da9abd35 100644 --- a/src/Coordination/KeeperStateManager.h +++ b/src/Coordination/KeeperStateManager.h @@ -13,14 +13,22 @@ namespace DB using KeeperServerConfigPtr = nuraft::ptr; +/// Wrapper struct for Keeper cluster config. We parse this +/// info from XML files. struct KeeperConfigurationWrapper { + /// Our port int port; + /// Our config KeeperServerConfigPtr config; + /// Servers id's to start as followers std::unordered_set servers_start_as_followers; + /// Cluster config ClusterConfigPtr cluster_config; }; +/// When our configuration changes the following action types +/// can happen enum class ConfigUpdateActionType { RemoveServer, @@ -28,6 +36,7 @@ enum class ConfigUpdateActionType UpdatePriority, }; +/// Action to update configuration struct ConfigUpdateAction { ConfigUpdateActionType action_type; @@ -36,6 +45,7 @@ struct ConfigUpdateAction using ConfigUpdateActions = std::vector; +/// Responsible for managing our and cluster configuration class KeeperStateManager : public nuraft::state_mgr { public: @@ -46,6 +56,7 @@ public: const CoordinationSettingsPtr & coordination_settings, bool standalone_keeper); + /// Constructor for tests KeeperStateManager( int server_id_, const std::string & host, @@ -56,8 +67,14 @@ public: void flushLogStore(); - nuraft::ptr load_config() override { return configuration_wrapper.cluster_config; } + /// Called on server start, in our case we don't use any separate logic for load + nuraft::ptr load_config() override + { + std::lock_guard lock(configuration_wrapper_mutex); + return configuration_wrapper.cluster_config; + } + /// Save cluster config (i.e. nodes, their priorities and so on) void save_config(const nuraft::cluster_config & config) override; void save_state(const nuraft::srv_state & state) override; @@ -72,10 +89,15 @@ public: void system_exit(const int /* exit_code */) override {} - int getPort() const { return configuration_wrapper.port; } + int getPort() const + { + std::lock_guard lock(configuration_wrapper_mutex); + return configuration_wrapper.port; + } bool shouldStartAsFollower() const { + std::lock_guard lock(configuration_wrapper_mutex); return configuration_wrapper.servers_start_as_followers.count(my_server_id); } @@ -86,12 +108,16 @@ public: nuraft::ptr getLogStore() const { return log_store; } - uint64_t getTotalServers() const { return configuration_wrapper.cluster_config->get_servers().size(); } + uint64_t getTotalServers() const + { + std::lock_guard lock(configuration_wrapper_mutex); + return configuration_wrapper.cluster_config->get_servers().size(); + } + /// Read all log entries in log store from the begging and return latest config (with largest log_index) ClusterConfigPtr getLatestConfigFromLogStore() const; - void setClusterConfig(const ClusterConfigPtr & cluster_config); - + /// Get configuration diff between proposed XML and current state in RAFT ConfigUpdateActions getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const; private: @@ -99,10 +125,13 @@ private: bool secure; std::string config_prefix; + mutable std::mutex configuration_wrapper_mutex; KeeperConfigurationWrapper configuration_wrapper; + nuraft::ptr log_store; nuraft::ptr server_state; + /// Parse configuration from xml config. KeeperConfigurationWrapper parseServersConfiguration(const Poco::Util::AbstractConfiguration & config) const; }; From c547ffa31395b15dcbfadeaafcb26941269c895b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 19 Oct 2021 16:36:31 +0300 Subject: [PATCH 216/919] improve performance of aggregation in order primary key with function with heavy states --- src/Core/Settings.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 1 + .../FinishAggregatingInOrderAlgorithm.cpp | 37 +++++-- .../FinishAggregatingInOrderAlgorithm.h | 15 ++- .../FinishAggregatingInOrderTransform.h | 6 +- src/Processors/QueryPlan/AggregatingStep.cpp | 17 ++- src/Processors/QueryPlan/AggregatingStep.h | 2 + .../AggregatingInOrderTransform.cpp | 103 +++++++++++------- .../Transforms/AggregatingInOrderTransform.h | 19 +++- tests/performance/aggregation_in_order_2.xml | 30 +++++ 10 files changed, 167 insertions(+), 64 deletions(-) create mode 100644 tests/performance/aggregation_in_order_2.xml diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a5767955045..6bd37432c2c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -386,6 +386,7 @@ class IColumn; M(Bool, enable_unaligned_array_join, false, "Allow ARRAY JOIN with multiple arrays that have different sizes. When this settings is enabled, arrays will be resized to the longest one.", 0) \ M(Bool, optimize_read_in_order, true, "Enable ORDER BY optimization for reading data in corresponding order in MergeTree tables.", 0) \ M(Bool, optimize_aggregation_in_order, false, "Enable GROUP BY optimization for aggregating data in corresponding order in MergeTree tables.", 0) \ + M(UInt64, aggregation_in_order_max_block_bytes, 50000000, "Maximal size of block in bytes accumulated during aggregation in order of primary key. Lower block size allows to parallelize more final merge stage of aggregation.", 0) \ M(UInt64, read_in_order_two_level_merge_threshold, 100, "Minimal number of parts to read to run preliminary merge step during multithread reading in order of primary key.", 0) \ M(Bool, low_cardinality_allow_in_native_format, true, "Use LowCardinality type in Native format. Otherwise, convert LowCardinality columns to ordinary for select query, and convert ordinary columns to required LowCardinality for insert query.", 0) \ M(Bool, cancel_http_readonly_queries_on_client_close, false, "Cancel HTTP readonly queries when a client closes the connection without waiting for response.", 0) \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 85cc889319f..a0432607af5 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2105,6 +2105,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac params, final, settings.max_block_size, + settings.aggregation_in_order_max_block_bytes, merge_threads, temporary_data_merge_threads, storage_has_evenly_distributed_read, diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp index f54331d5550..5fc57fb42d8 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -9,9 +10,10 @@ namespace DB { FinishAggregatingInOrderAlgorithm::State::State( - const Chunk & chunk, const SortDescription & desc) - : num_rows(chunk.getNumRows()) - , all_columns(chunk.getColumns()) + const Chunk & chunk, const SortDescription & desc, Int64 total_bytes_) + : all_columns(chunk.getColumns()) + , num_rows(chunk.getNumRows()) + , total_bytes(total_bytes_) { if (!chunk) return; @@ -26,12 +28,14 @@ FinishAggregatingInOrderAlgorithm::FinishAggregatingInOrderAlgorithm( size_t num_inputs_, AggregatingTransformParamsPtr params_, SortDescription description_, - size_t max_block_size_) + size_t max_block_size_, + size_t max_block_bytes_) : header(header_) , num_inputs(num_inputs_) , params(params_) , description(std::move(description_)) , max_block_size(max_block_size_) + , max_block_bytes(max_block_bytes_) { /// Replace column names in description to positions. for (auto & column_description : description) @@ -47,14 +51,22 @@ FinishAggregatingInOrderAlgorithm::FinishAggregatingInOrderAlgorithm( void FinishAggregatingInOrderAlgorithm::initialize(Inputs inputs) { current_inputs = std::move(inputs); - states.reserve(num_inputs); + states.resize(num_inputs); for (size_t i = 0; i < num_inputs; ++i) - states.emplace_back(current_inputs[i].chunk, description); + consume(current_inputs[i], i); } void FinishAggregatingInOrderAlgorithm::consume(Input & input, size_t source_num) { - states[source_num] = State{input.chunk, description}; + const auto & info = input.chunk.getChunkInfo(); + if (!info) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in FinishAggregatingInOrderAlgorithm"); + + const auto * arenas_info = typeid_cast(info.get()); + if (!arenas_info) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should have ChunkInfoWithAllocatedBytes in FinishAggregatingInOrderAlgorithm"); + + states[source_num] = State{input.chunk, description, arenas_info->allocated_bytes}; } IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge() @@ -111,8 +123,8 @@ IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge() Status status(inputs_to_update.back()); inputs_to_update.pop_back(); - /// Do not merge blocks, if there are too few rows. - if (accumulated_rows >= max_block_size) + /// Do not merge blocks, if there are too few rows or bytes. + if (accumulated_rows >= max_block_size || accumulated_bytes >= max_block_bytes) status.chunk = prepareToMerge(); return status; @@ -121,6 +133,8 @@ IMergingAlgorithm::Status FinishAggregatingInOrderAlgorithm::merge() Chunk FinishAggregatingInOrderAlgorithm::prepareToMerge() { accumulated_rows = 0; + accumulated_bytes = 0; + auto info = std::make_shared(); info->chunks = std::make_unique(std::move(chunks)); @@ -153,10 +167,13 @@ void FinishAggregatingInOrderAlgorithm::addToAggregation() } chunks.back().setChunkInfo(std::make_shared()); - states[i].current_row = states[i].to_row; + + /// We assume that sizes in bytes of rows are almost the same. + accumulated_bytes += states[i].total_bytes * (static_cast(current_rows) / states[i].num_rows); accumulated_rows += current_rows; + if (!states[i].isValid()) inputs_to_update.push_back(i); } diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h index 209bb9e4045..f3a1bd40635 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.h @@ -42,7 +42,8 @@ public: size_t num_inputs_, AggregatingTransformParamsPtr params_, SortDescription description_, - size_t max_block_size_); + size_t max_block_size_, + size_t max_block_bytes_); void initialize(Inputs inputs) override; void consume(Input & input, size_t source_num) override; @@ -54,17 +55,23 @@ private: struct State { - size_t num_rows; Columns all_columns; ColumnRawPtrs sorting_columns; + size_t num_rows = 0; + /// Number of row starting from which need to aggregate. size_t current_row = 0; /// Number of row up to which need to aggregate (not included). size_t to_row = 0; - State(const Chunk & chunk, const SortDescription & description); + /// Number of bytes in all columns + number of bytes in arena, related to current chunk. + size_t total_bytes = 0; + + State(const Chunk & chunk, const SortDescription & description, Int64 total_bytes_); + State() = default; + bool isValid() const { return current_row < num_rows; } }; @@ -73,6 +80,7 @@ private: AggregatingTransformParamsPtr params; SortDescription description; size_t max_block_size; + size_t max_block_bytes; Inputs current_inputs; @@ -81,6 +89,7 @@ private: std::vector chunks; size_t accumulated_rows = 0; + size_t accumulated_bytes = 0; }; } diff --git a/src/Processors/Merges/FinishAggregatingInOrderTransform.h b/src/Processors/Merges/FinishAggregatingInOrderTransform.h index 5605cea8e39..5dd875c8412 100644 --- a/src/Processors/Merges/FinishAggregatingInOrderTransform.h +++ b/src/Processors/Merges/FinishAggregatingInOrderTransform.h @@ -17,14 +17,16 @@ public: size_t num_inputs, AggregatingTransformParamsPtr params, SortDescription description, - size_t max_block_size) + size_t max_block_size, + size_t max_block_bytes) : IMergingTransform( num_inputs, header, {}, /*have_all_inputs_=*/ true, /*has_limit_below_one_block_=*/ false, header, num_inputs, params, std::move(description), - max_block_size) + max_block_size, + max_block_bytes) { } diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 51bd65a209b..219d5efa027 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -30,6 +30,7 @@ AggregatingStep::AggregatingStep( Aggregator::Params params_, bool final_, size_t max_block_size_, + size_t aggregation_in_order_max_block_bytes_, size_t merge_threads_, size_t temporary_data_merge_threads_, bool storage_has_evenly_distributed_read_, @@ -39,6 +40,7 @@ AggregatingStep::AggregatingStep( , params(std::move(params_)) , final(std::move(final_)) , max_block_size(max_block_size_) + , aggregation_in_order_max_block_bytes(aggregation_in_order_max_block_bytes_) , merge_threads(merge_threads_) , temporary_data_merge_threads(temporary_data_merge_threads_) , storage_has_evenly_distributed_read(storage_has_evenly_distributed_read_) @@ -90,7 +92,13 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B size_t counter = 0; pipeline.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, transform_params, group_by_sort_description, max_block_size, many_data, counter++); + /// We want to merge aggregated data in batches of size + /// not greater than 'aggregation_in_order_max_block_bytes'. + /// So, we reduce 'max_bytes' value for aggregation in 'merge_threads' times. + return std::make_shared( + header, transform_params, group_by_sort_description, + max_block_size, aggregation_in_order_max_block_bytes / merge_threads, + many_data, counter++); }); aggregating_in_order = collector.detachProcessors(0); @@ -100,7 +108,8 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B pipeline.getNumStreams(), transform_params, group_by_sort_description, - max_block_size); + max_block_size, + aggregation_in_order_max_block_bytes); pipeline.addTransform(std::move(transform)); @@ -118,7 +127,9 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B { pipeline.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, transform_params, group_by_sort_description, max_block_size); + return std::make_shared( + header, transform_params, group_by_sort_description, + max_block_size, aggregation_in_order_max_block_bytes); }); pipeline.addSimpleTransform([&](const Block & header) diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 8583e5be485..1a4f36a6711 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -19,6 +19,7 @@ public: Aggregator::Params params_, bool final_, size_t max_block_size_, + size_t aggregation_in_order_max_block_bytes_, size_t merge_threads_, size_t temporary_data_merge_threads_, bool storage_has_evenly_distributed_read_, @@ -40,6 +41,7 @@ private: Aggregator::Params params; bool final; size_t max_block_size; + size_t aggregation_in_order_max_block_bytes; size_t merge_threads; size_t temporary_data_merge_threads; diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index d6c5491f45c..a1cae41fb9d 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -13,19 +13,24 @@ namespace ErrorCodes AggregatingInOrderTransform::AggregatingInOrderTransform( - Block header, AggregatingTransformParamsPtr params_, - const SortDescription & group_by_description_, size_t res_block_size_) - : AggregatingInOrderTransform(std::move(header), std::move(params_) - , group_by_description_, res_block_size_, std::make_unique(1), 0) + Block header, + AggregatingTransformParamsPtr params_, + const SortDescription & group_by_description_, + size_t max_block_size_, size_t max_block_bytes_) + : AggregatingInOrderTransform(std::move(header), std::move(params_), + group_by_description_, max_block_size_, max_block_bytes_, + std::make_unique(1), 0) { } AggregatingInOrderTransform::AggregatingInOrderTransform( Block header, AggregatingTransformParamsPtr params_, - const SortDescription & group_by_description_, size_t res_block_size_, + const SortDescription & group_by_description_, + size_t max_block_size_, size_t max_block_bytes_, ManyAggregatedDataPtr many_data_, size_t current_variant) : IProcessor({std::move(header)}, {params_->getCustomHeader(false)}) - , res_block_size(res_block_size_) + , max_block_size(max_block_size_) + , max_block_bytes(max_block_bytes_) , params(std::move(params_)) , group_by_description(group_by_description_) , aggregate_columns(params->params.aggregates_size) @@ -48,8 +53,18 @@ AggregatingInOrderTransform::AggregatingInOrderTransform( AggregatingInOrderTransform::~AggregatingInOrderTransform() = default; +static Int64 getCurrentMemoryUsage() +{ + Int64 current_memory_usage = 0; + if (auto * memory_tracker = CurrentThread::getMemoryTracker()) + current_memory_usage = memory_tracker->get(); + return current_memory_usage; +} + void AggregatingInOrderTransform::consume(Chunk chunk) { + Int64 initial_memory_usage = getCurrentMemoryUsage(); + size_t rows = chunk.getNumRows(); if (rows == 0) return; @@ -77,6 +92,7 @@ void AggregatingInOrderTransform::consume(Chunk chunk) size_t key_end = 0; size_t key_begin = 0; + /// If we don't have a block we create it and fill with first key if (!cur_block_size) { @@ -94,6 +110,7 @@ void AggregatingInOrderTransform::consume(Chunk chunk) ++cur_block_size; } + Int64 current_memory_usage = 0; /// Will split block into segments with the same key while (key_end != rows) @@ -112,44 +129,18 @@ void AggregatingInOrderTransform::consume(Chunk chunk) if (key_begin != key_end) params->aggregator.executeOnIntervalWithoutKeyImpl(variants.without_key, key_begin, key_end, aggregate_function_instructions.data(), variants.aggregates_pool); + current_memory_usage = getCurrentMemoryUsage() - initial_memory_usage; + /// We finalize last key aggregation state if a new key found. if (key_end != rows) { params->aggregator.addSingleKeyToAggregateColumns(variants, res_aggregate_columns); - /// If res_block_size is reached we have to stop consuming and generate the block. Save the extra rows into new chunk. - if (cur_block_size == res_block_size) + /// If max_block_size is reached we have to stop consuming and generate the block. Save the extra rows into new chunk. + if (cur_block_size >= max_block_size || cur_block_bytes + current_memory_usage >= max_block_bytes) { - Columns source_columns = chunk.detachColumns(); - - for (auto & source_column : source_columns) - source_column = source_column->cut(key_end, rows - key_end); - - current_chunk = Chunk(source_columns, rows - key_end); - src_rows -= current_chunk.getNumRows(); - block_end_reached = true; - need_generate = true; - cur_block_size = 0; - - variants.without_key = nullptr; - - /// Arenas cannot be destroyed here, since later, in FinalizeAggregatedTransform - /// there will be finalizeChunk(), but even after - /// finalizeChunk() we cannot destroy arena, since some memory - /// from Arena still in use, so we attach it to the Chunk to - /// remove it once it will be consumed. - if (params->final) - { - if (variants.aggregates_pools.size() != 1) - throw Exception("Too much arenas", ErrorCodes::LOGICAL_ERROR); - - Arenas arenas(1, std::make_shared()); - std::swap(variants.aggregates_pools, arenas); - variants.aggregates_pool = variants.aggregates_pools.at(0).get(); - - chunk.setChunkInfo(std::make_shared(std::move(arenas))); - } - + cur_block_bytes += current_memory_usage; + finalizeCurrentChunk(std::move(chunk), key_end); return; } @@ -161,9 +152,25 @@ void AggregatingInOrderTransform::consume(Chunk chunk) key_begin = key_end; } + cur_block_bytes += current_memory_usage; block_end_reached = false; } +void AggregatingInOrderTransform::finalizeCurrentChunk(Chunk chunk, size_t key_end) +{ + size_t rows = chunk.getNumRows(); + Columns source_columns = chunk.detachColumns(); + + for (auto & source_column : source_columns) + source_column = source_column->cut(key_end, rows - key_end); + + current_chunk = Chunk(source_columns, rows - key_end); + src_rows -= current_chunk.getNumRows(); + + block_end_reached = true; + need_generate = true; + variants.without_key = nullptr; +} void AggregatingInOrderTransform::work() { @@ -218,6 +225,7 @@ IProcessor::Status AggregatingInOrderTransform::prepare() src_rows, res_rows, formatReadableSizeWithBinarySuffix(src_bytes)); return Status::Finished; } + if (input.isFinished()) { is_consume_finished = true; @@ -245,14 +253,25 @@ void AggregatingInOrderTransform::generate() Block res = res_header.cloneEmpty(); for (size_t i = 0; i < res_key_columns.size(); ++i) - { res.getByPosition(i).column = std::move(res_key_columns[i]); - } + for (size_t i = 0; i < res_aggregate_columns.size(); ++i) - { res.getByPosition(i + res_key_columns.size()).column = std::move(res_aggregate_columns[i]); - } + to_push_chunk = convertToChunk(res); + to_push_chunk.setChunkInfo(std::make_shared(cur_block_bytes)); + + /// Clear arenas to allow to free them, when chunk will reach the end of pipeline. + /// It's safe clear them here, because columns with aggregate functions already holds them. + variants.aggregates_pools = { std::make_shared() }; + variants.aggregates_pool = variants.aggregates_pools.at(0).get(); + + /// Pass info about used memory by aggregate functions further. + to_push_chunk.setChunkInfo(std::make_shared(cur_block_bytes)); + + cur_block_bytes = 0; + cur_block_size = 0; + res_rows += to_push_chunk.getNumRows(); need_generate = false; } diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.h b/src/Processors/Transforms/AggregatingInOrderTransform.h index e0cb9b80967..929ab98d6e6 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.h +++ b/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -9,16 +9,24 @@ namespace DB { +struct ChunkInfoWithAllocatedBytes : public ChunkInfo +{ + ChunkInfoWithAllocatedBytes(Int64 allocated_bytes_) + : allocated_bytes(allocated_bytes_) {} + Int64 allocated_bytes; +}; + class AggregatingInOrderTransform : public IProcessor { - public: AggregatingInOrderTransform(Block header, AggregatingTransformParamsPtr params, - const SortDescription & group_by_description, size_t res_block_size, + const SortDescription & group_by_description, + size_t max_block_size_, size_t max_block_bytes_, ManyAggregatedDataPtr many_data, size_t current_variant); AggregatingInOrderTransform(Block header, AggregatingTransformParamsPtr params, - const SortDescription & group_by_description, size_t res_block_size); + const SortDescription & group_by_description, + size_t max_block_size_, size_t max_block_bytes_); ~AggregatingInOrderTransform() override; @@ -32,9 +40,12 @@ public: private: void generate(); + void finalizeCurrentChunk(Chunk chunk, size_t key_end); - size_t res_block_size; + size_t max_block_size; + size_t max_block_bytes; size_t cur_block_size = 0; + size_t cur_block_bytes = 0; MutableColumns res_key_columns; MutableColumns res_aggregate_columns; diff --git a/tests/performance/aggregation_in_order_2.xml b/tests/performance/aggregation_in_order_2.xml new file mode 100644 index 00000000000..94a33a98c3e --- /dev/null +++ b/tests/performance/aggregation_in_order_2.xml @@ -0,0 +1,30 @@ + + + 1 + 10000000 + 30 + + + + + uniqs + + 100 + 10000 + 1000000 + + + + + CREATE TABLE mt_50_parts_{uniqs}_uniqs(id UInt32, val1 UInt32, val2 UInt32) ENGINE = MergeTree ORDER BY val1 PARTITION BY id % 50 SETTINGS parts_to_throw_insert=10000, parts_to_delay_insert=10000 + SYSTEM STOP MERGES mt_50_parts_{uniqs}_uniqs + INSERT INTO mt_50_parts_{uniqs}_uniqs SELECT number, rand() % {uniqs}, rand() FROM numbers_mt(100000000) SETTINGS max_block_size=10000000 + SYSTEM START MERGES mt_50_parts_{uniqs}_uniqs + OPTIMIZE TABLE mt_50_parts_{uniqs}_uniqs FINAL + + SELECT sum(val2) FROM mt_50_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null + SELECT groupArray(val2) FROM mt_50_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null + SELECT uniqExact(val2) FROM mt_50_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null + + DROP TABLE IF EXISTS mt_50_parts_{uniqs}_uniqs + From 042eebd98118e7a94415c495ec4991fa1da5fc7e Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Oct 2021 16:37:28 +0300 Subject: [PATCH 217/919] Add test for leader remove --- src/Coordination/KeeperServer.cpp | 6 +++--- src/Coordination/KeeperStateManager.cpp | 10 ++++------ src/Coordination/KeeperStateManager.h | 2 +- tests/integration/test_keeper_nodes_remove/test.py | 10 ---------- 4 files changed, 8 insertions(+), 20 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index bd0f06d3ad4..840a0ee90bb 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -475,7 +475,7 @@ bool KeeperServer::waitConfigurationUpdate(const ConfigUpdateAction & task) { if (raft_instance->get_srv_config(task.server->get_id()) != nullptr) { - LOG_INFO(log, "Server with id {} was successfully added", task.server->get_id()); + LOG_INFO(log, "Server with id {} was successfully added by leader", task.server->get_id()); return true; } @@ -491,13 +491,13 @@ bool KeeperServer::waitConfigurationUpdate(const ConfigUpdateAction & task) } else if (task.action_type == ConfigUpdateActionType::RemoveServer) { - LOG_INFO(log, "Will try to remove server with id {}", task.server->get_id()); + LOG_INFO(log, "Will try to wait remove of server with id {}", task.server->get_id()); for (size_t i = 0; i < coordination_settings->configuration_change_tries_count; ++i) { if (raft_instance->get_srv_config(task.server->get_id()) == nullptr) { - LOG_INFO(log, "Server with id {} was successfully removed", task.server->get_id()); + LOG_INFO(log, "Server with id {} was successfully removed by leader", task.server->get_id()); return true; } diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 8b95778b65a..6a8d424a345 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -32,7 +32,7 @@ namespace } -KeeperConfigurationWrapper KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfiguration & config) const +KeeperConfigurationWrapper KeeperStateManager::parseServersConfiguration(const Poco::Util::AbstractConfiguration & config, bool allow_without_us) const { KeeperConfigurationWrapper result; result.cluster_config = std::make_shared(); @@ -68,7 +68,7 @@ KeeperConfigurationWrapper KeeperStateManager::parseServersConfiguration(const P total_servers++; } - if (!result.config) + if (!result.config && !allow_without_us) throw Exception(ErrorCodes::RAFT_ERROR, "Our server id {} not found in raft_configuration section", my_server_id); if (result.servers_start_as_followers.size() == total_servers) @@ -98,7 +98,7 @@ KeeperStateManager::KeeperStateManager( : my_server_id(server_id_) , secure(config.getBool(config_prefix_ + ".raft_configuration.secure", false)) , config_prefix(config_prefix_) - , configuration_wrapper(parseServersConfiguration(config)) + , configuration_wrapper(parseServersConfiguration(config, false)) , log_store(nuraft::cs_new( getLogsPathFromConfig(config_prefix_, config, standalone_keeper), coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync, coordination_settings->compress_logs)) @@ -138,9 +138,7 @@ void KeeperStateManager::save_state(const nuraft::srv_state & state) ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const { - auto new_configuration_wrapper = parseServersConfiguration(config); - if (new_configuration_wrapper.port != configuration_wrapper.port) - throw Exception(ErrorCodes::RAFT_ERROR, "Cannot change port of already running RAFT server"); + auto new_configuration_wrapper = parseServersConfiguration(config, true); std::unordered_map new_ids, old_ids; for (auto new_server : new_configuration_wrapper.cluster_config->get_servers()) diff --git a/src/Coordination/KeeperStateManager.h b/src/Coordination/KeeperStateManager.h index f42da9abd35..5f051fcfc30 100644 --- a/src/Coordination/KeeperStateManager.h +++ b/src/Coordination/KeeperStateManager.h @@ -132,7 +132,7 @@ private: nuraft::ptr server_state; /// Parse configuration from xml config. - KeeperConfigurationWrapper parseServersConfiguration(const Poco::Util::AbstractConfiguration & config) const; + KeeperConfigurationWrapper parseServersConfiguration(const Poco::Util::AbstractConfiguration & config, bool allow_without_us) const; }; } diff --git a/tests/integration/test_keeper_nodes_remove/test.py b/tests/integration/test_keeper_nodes_remove/test.py index 72b02c0661f..6df4ee1c497 100644 --- a/tests/integration/test_keeper_nodes_remove/test.py +++ b/tests/integration/test_keeper_nodes_remove/test.py @@ -2,13 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster -import random -import string import os -import time -from multiprocessing.dummy import Pool -from helpers.network import PartitionManager -from helpers.test_tools import assert_eq_with_retry from kazoo.client import KazooClient, KazooState cluster = ClickHouseCluster(__file__) @@ -29,10 +23,6 @@ def started_cluster(): cluster.shutdown() -def start(node): - node.start_clickhouse() - - def get_fake_zk(node, timeout=30.0): _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(node.name) + ":9181", timeout=timeout) _fake_zk_instance.start() From 32dd422cc707a00ec470435e29b0812220eefba2 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Tue, 19 Oct 2021 16:42:16 +0300 Subject: [PATCH 218/919] Update map container --- .../functions/tuple-map-functions.md | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/ru/sql-reference/functions/tuple-map-functions.md b/docs/ru/sql-reference/functions/tuple-map-functions.md index 0d4cae7f2d5..5348e175269 100644 --- a/docs/ru/sql-reference/functions/tuple-map-functions.md +++ b/docs/ru/sql-reference/functions/tuple-map-functions.md @@ -128,13 +128,13 @@ mapSubtract(Tuple(Array, Array), Tuple(Array, Array) [, ...]) **Аргументы** -Аргументами являются [maps](../../sql-reference/data-types/map.md) или [кортежи](../../sql-reference/data-types/tuple.md#tuplet1-t2) из двух [массивов](../../sql-reference/data-types/array.md#data-type-array), где элементы в первом массиве представляют ключи, а второй массив содержит значения для каждого ключа. +Аргументами являются контейнеры [Map](../../sql-reference/data-types/map.md) или [кортежи](../../sql-reference/data-types/tuple.md#tuplet1-t2) из двух [массивов](../../sql-reference/data-types/array.md#data-type-array), где элементы в первом массиве представляют ключи, а второй массив содержит значения для каждого ключа. Все массивы ключей должны иметь один и тот же тип, а все массивы значений должны содержать элементы, которые можно приводить к одному типу ([Int64](../../sql-reference/data-types/int-uint.md#int-ranges), [UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges) или [Float64](../../sql-reference/data-types/float.md#float32-float64)). Общий приведенный тип используется в качестве типа для результирующего массива. **Возвращаемое значение** -- В зависимости от аргумента возвращает [map] или [кортеж](../../sql-reference/data-types/tuple.md#tuplet1-t2), в котором первый массив содержит отсортированные ключи, а второй - значения. +- В зависимости от аргумента возвращает один [Map](../../sql-reference/data-types/map.md) или [кортеж](../../sql-reference/data-types/tuple.md#tuplet1-t2), в котором первый массив содержит отсортированные ключи, а второй - значения. **Пример** @@ -152,7 +152,7 @@ SELECT mapSubtract(([toUInt8(1), 2], [toInt32(1), 1]), ([toUInt8(1), 2], [toInt3 └────────────────┴───────────────────────────────────┘ ``` -Запрос с типом `Map`: +Запрос с контейнером `Map`: ```sql SELECT mapSubtract(map(1,1), map(1,1)); @@ -170,7 +170,7 @@ SELECT mapSubtract(map(1,1), map(1,1)); Заполняет недостающие ключи в контейнере map (пара массивов ключей и значений), где ключи являются целыми числами. Кроме того, он поддерживает указание максимального ключа, который используется для расширения массива ключей. -Аргументами являются [map](../../sql-reference/data-types/map.md) или два [массива](../../sql-reference/data-types/array.md#data-type-array), где первый массив представляет ключи, а второй массив содержит значения для каждого ключа. +Аргументами являются контейнер [Map](../../sql-reference/data-types/map.md) или два [массива](../../sql-reference/data-types/array.md#data-type-array), где первый массив представляет ключи, а второй массив содержит значения для каждого ключа. **Синтаксис** @@ -187,16 +187,16 @@ mapPopulateSeries(map[, max]) Сопоставленные массивы: -- `keys` — массив ключей [Array](../../sql-reference/data-types/array.md#data-type-array)([Int](../../sql-reference/data-types/int-uint.md#int-ranges)). +- `keys` — массив ключей. [Array](../../sql-reference/data-types/array.md#data-type-array)([Int](../../sql-reference/data-types/int-uint.md#int-ranges)). - `values` — массив значений. [Array](../../sql-reference/data-types/array.md#data-type-array)([Int](../../sql-reference/data-types/int-uint.md#int-ranges)). или -- `map` — карта с целочисленными ключами. [Map](../../sql-reference/data-types/map.md). +- `map` — контейнер Map с целочисленными ключами. [Map](../../sql-reference/data-types/map.md). **Возвращаемое значение** -- В зависимости от аргумента возвращает [map](../../sql-reference/data-types/map.md) или [кортеж](../../sql-reference/data-types/tuple.md#tuplet1-t2) из двух [массивов](../../sql-reference/data-types/array.md#data-type-array): ключи отсортированные по порядку и значения соответствующих ключей. +- В зависимости от аргумента возвращает контейнер [Map](../../sql-reference/data-types/map.md) или [кортеж](../../sql-reference/data-types/tuple.md#tuplet1-t2) из двух [массивов](../../sql-reference/data-types/array.md#data-type-array): ключи отсортированные по порядку и значения соответствующих ключей. **Пример** @@ -214,7 +214,7 @@ SELECT mapPopulateSeries([1,2,4], [11,22,44], 5) AS res, toTypeName(res) AS type └──────────────────────────────┴───────────────────────────────────┘ ``` -Запрос с типом `Map`: +Запрос с контейнером `Map`: ```sql SELECT mapPopulateSeries(map(1, 10, 5, 20), 6); From da14e5e784884e2c1baf81d1487220de98f9ef14 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Oct 2021 16:49:36 +0300 Subject: [PATCH 219/919] Fix typos --- src/Coordination/KeeperDispatcher.h | 2 +- src/Coordination/KeeperServer.h | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperDispatcher.h b/src/Coordination/KeeperDispatcher.h index e958ed2c623..97152ae80b8 100644 --- a/src/Coordination/KeeperDispatcher.h +++ b/src/Coordination/KeeperDispatcher.h @@ -115,7 +115,7 @@ public: return server && server->checkInit(); } - /// Registered in ConfigReloader callback. Add new confugration changes to + /// Registered in ConfigReloader callback. Add new configuration changes to /// update_configuration_queue. Keeper Dispatcher apply them asynchronously. void updateConfiguration(const Poco::Util::AbstractConfiguration & config); diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index fd49b3ceab5..e25f240ecf7 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -93,11 +93,13 @@ public: /// Get configuration diff between current configuration in RAFT and in XML file ConfigUpdateActions getConfigurationDiff(const Poco::Util::AbstractConfiguration & config); - /// Apply action fo configuration update. Actually call raft_instance->remove_srv or raft_instance->add_srv. + /// Apply action for configuration update. Actually call raft_instance->remove_srv or raft_instance->add_srv. /// Synchronously check for update results with retries. void applyConfigurationUpdate(const ConfigUpdateAction & action); + /// Wait configuration update for action. Used by followers. + /// Return true if update was successfully received. bool waitConfigurationUpdate(const ConfigUpdateAction & action); }; From 8d5ed8a342a00d1d73148300ce05bf057e492133 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 19 Oct 2021 17:05:54 +0300 Subject: [PATCH 220/919] remove useless line --- src/Processors/Transforms/AggregatingInOrderTransform.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index a1cae41fb9d..e8ac969245d 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -259,7 +259,6 @@ void AggregatingInOrderTransform::generate() res.getByPosition(i + res_key_columns.size()).column = std::move(res_aggregate_columns[i]); to_push_chunk = convertToChunk(res); - to_push_chunk.setChunkInfo(std::make_shared(cur_block_bytes)); /// Clear arenas to allow to free them, when chunk will reach the end of pipeline. /// It's safe clear them here, because columns with aggregate functions already holds them. From 9f939364e3f94626f378792cb11e07d1c4e02de6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Oct 2021 17:10:09 +0300 Subject: [PATCH 221/919] Add some comments --- src/Coordination/Changelog.h | 1 + src/Coordination/KeeperLogStore.h | 1 + src/Coordination/KeeperSnapshotManager.cpp | 1 + src/Coordination/KeeperSnapshotManager.h | 4 ++++ src/Coordination/KeeperStateMachine.h | 4 ++++ 5 files changed, 11 insertions(+) diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 196c7c4a1e5..2fd0a927f5d 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -102,6 +102,7 @@ public: /// Last entry in log, or fake entry with term 0 if log is empty LogEntryPtr getLastEntry() const; + /// Get entry with latest config in logstore LogEntryPtr getLatestConfigChange() const; /// Return log entries between [start, end) diff --git a/src/Coordination/KeeperLogStore.h b/src/Coordination/KeeperLogStore.h index 9e840e9699e..76343582311 100644 --- a/src/Coordination/KeeperLogStore.h +++ b/src/Coordination/KeeperLogStore.h @@ -58,6 +58,7 @@ public: /// Flush batch of appended entries void end_of_append_batch(uint64_t start_index, uint64_t count) override; + /// Get entry with latest config in logstore nuraft::ptr getLatestConfigChange() const; private: diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index d7ca7b82d78..be6d4db4219 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -206,6 +206,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr } } + /// Serialize cluster config if (snapshot.cluster_config) { auto buffer = snapshot.cluster_config->serialize(); diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 8dd86f2f94e..cac27f0faa8 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -22,10 +22,14 @@ enum SnapshotVersion : uint8_t static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V3; +/// What is stored in binary shapsnot struct SnapshotDeserializationResult { + /// Storage KeeperStoragePtr storage; + /// Snapshot metadata (up_to_log_idx and so on) SnapshotMetadataPtr snapshot_meta; + /// Cluster config ClusterConfigPtr cluster_config; }; diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index 1afc99aabc1..b8085c032c9 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -32,6 +32,7 @@ public: nuraft::ptr commit(const uint64_t log_idx, nuraft::buffer & data) override; + /// Save new cluster config to our snapshot (copy of the config stored in StateManager) void commit_config(const uint64_t log_idx, nuraft::ptr & new_conf) override; /// Currently not supported @@ -116,6 +117,9 @@ private: Poco::Logger * log; + /// Cluster config for our quorum. + /// It's a copy of config stored in StateManager, but here + /// we also write it to disk during snapshot. Must be used with lock. mutable std::mutex cluster_config_lock; ClusterConfigPtr cluster_config; From 82b409e2b0dfc82de78e37a75e52fb7b6bacd1b2 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 19 Oct 2021 17:13:52 +0300 Subject: [PATCH 222/919] better perf test --- tests/performance/aggregation_in_order_2.xml | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/performance/aggregation_in_order_2.xml b/tests/performance/aggregation_in_order_2.xml index 94a33a98c3e..083a8031d46 100644 --- a/tests/performance/aggregation_in_order_2.xml +++ b/tests/performance/aggregation_in_order_2.xml @@ -16,15 +16,15 @@ - CREATE TABLE mt_50_parts_{uniqs}_uniqs(id UInt32, val1 UInt32, val2 UInt32) ENGINE = MergeTree ORDER BY val1 PARTITION BY id % 50 SETTINGS parts_to_throw_insert=10000, parts_to_delay_insert=10000 - SYSTEM STOP MERGES mt_50_parts_{uniqs}_uniqs - INSERT INTO mt_50_parts_{uniqs}_uniqs SELECT number, rand() % {uniqs}, rand() FROM numbers_mt(100000000) SETTINGS max_block_size=10000000 - SYSTEM START MERGES mt_50_parts_{uniqs}_uniqs - OPTIMIZE TABLE mt_50_parts_{uniqs}_uniqs FINAL + CREATE TABLE mt_30_parts_{uniqs}_uniqs(id UInt32, val1 UInt32, val2 UInt32) ENGINE = MergeTree ORDER BY val1 PARTITION BY id % 30 SETTINGS parts_to_throw_insert=10000, parts_to_delay_insert=10000 + SYSTEM STOP MERGES mt_30_parts_{uniqs}_uniqs + INSERT INTO mt_30_parts_{uniqs}_uniqs SELECT number, rand() % {uniqs}, rand() FROM numbers_mt(100000000) SETTINGS max_block_size=10000000 + SYSTEM START MERGES mt_30_parts_{uniqs}_uniqs + OPTIMIZE TABLE mt_30_parts_{uniqs}_uniqs FINAL - SELECT sum(val2) FROM mt_50_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null - SELECT groupArray(val2) FROM mt_50_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null - SELECT uniqExact(val2) FROM mt_50_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null + SELECT sum(val2) FROM mt_30_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null + SELECT groupArray(val2) FROM mt_30_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null + SELECT uniqExact(val2) FROM mt_30_parts_{uniqs}_uniqs GROUP BY val1 FORMAT Null - DROP TABLE IF EXISTS mt_50_parts_{uniqs}_uniqs + DROP TABLE IF EXISTS mt_30_parts_{uniqs}_uniqs From 18cceedc00cbd8db070839e55403d1a7e549727b Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Oct 2021 17:29:49 +0300 Subject: [PATCH 223/919] Fix build and update for clickhouse-keeper --- programs/keeper/Keeper.cpp | 23 +++++++++++++++++++ src/Coordination/tests/gtest_coordination.cpp | 10 ++++---- 2 files changed, 28 insertions(+), 5 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index da3c42a3213..6e001c57e75 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -331,6 +332,8 @@ int Keeper::main(const std::vector & /*args*/) const Settings & settings = global_context->getSettingsRef(); + std::string include_from_path = config().getString("include_from", "/etc/metrika.xml"); + GlobalThreadPool::initialize(config().getUInt("max_thread_pool_size", 100)); static ServerErrorHandler error_handler; @@ -402,8 +405,27 @@ int Keeper::main(const std::vector & /*args*/) for (auto & server : *servers) server.start(); + zkutil::EventPtr unused_event = std::make_shared(); + zkutil::ZooKeeperNodeCache unused_cache([] { return nullptr; }); + /// ConfigReloader have to strict parameters which are redundant in our case + auto main_config_reloader = std::make_unique( + config_path, + include_from_path, + config().getString("path", ""), + std::move(unused_cache), + unused_event, + [&](ConfigurationPtr config, bool /* initial_loading */) + { + if (config->has("keeper_server")) + global_context->updateKeeperConfiguration(*config); + }, + /* already_loaded = */ false); /// Reload it right now (initial loading) + SCOPE_EXIT({ LOG_INFO(log, "Shutting down."); + /// Stop reloading of the main config. This must be done before `global_context->shutdown()` because + /// otherwise the reloading may pass a changed config to some destroyed parts of ContextSharedPart. + main_config_reloader.reset(); global_context->shutdown(); @@ -450,6 +472,7 @@ int Keeper::main(const std::vector & /*args*/) buildLoggers(config(), logger()); + main_config_reloader->start(); LOG_INFO(log, "Ready for connections."); diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 0cb5972f718..86b36c2bf08 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -962,7 +962,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) auto debuf = manager.deserializeSnapshotBufferFromDisk(2); - auto [snapshot_meta, restored_storage] = manager.deserializeSnapshotFromBuffer(debuf); + auto [restored_storage, snapshot_meta, _] = manager.deserializeSnapshotFromBuffer(debuf); EXPECT_EQ(restored_storage->container.size(), 3); EXPECT_EQ(restored_storage->container.getValue("/").children.size(), 1); @@ -1011,7 +1011,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) auto debuf = manager.deserializeSnapshotBufferFromDisk(50); - auto [meta, restored_storage] = manager.deserializeSnapshotFromBuffer(debuf); + auto [restored_storage, meta, _] = manager.deserializeSnapshotFromBuffer(debuf); EXPECT_EQ(restored_storage->container.size(), 51); for (size_t i = 0; i < 50; ++i) @@ -1050,7 +1050,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots) EXPECT_TRUE(fs::exists("./snapshots/snapshot_250.bin" + params.extension)); - auto [meta, restored_storage] = manager.restoreFromLatestSnapshot(); + auto [restored_storage, meta, _] = manager.restoreFromLatestSnapshot(); EXPECT_EQ(restored_storage->container.size(), 251); @@ -1103,7 +1103,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode) EXPECT_FALSE(storage.container.contains("/hello_" + std::to_string(i))); } - auto [meta, restored_storage] = manager.restoreFromLatestSnapshot(); + auto [restored_storage, meta, _] = manager.restoreFromLatestSnapshot(); for (size_t i = 0; i < 50; ++i) { @@ -1498,7 +1498,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions) auto debuf = new_manager.deserializeSnapshotBufferFromDisk(2); - auto [snapshot_meta, restored_storage] = new_manager.deserializeSnapshotFromBuffer(debuf); + auto [restored_storage, snapshot_meta, _] = new_manager.deserializeSnapshotFromBuffer(debuf); EXPECT_EQ(restored_storage->container.size(), 3); EXPECT_EQ(restored_storage->container.getValue("/").children.size(), 1); From 905418b46868d1345d1de444003b5f057c616829 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Tue, 19 Oct 2021 18:33:22 +0300 Subject: [PATCH 224/919] Update argmin.md because https://github.com/ClickHouse/ClickHouse/pull/23393 --- .../sql-reference/aggregate-functions/reference/argmin.md | 6 ------ 1 file changed, 6 deletions(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/argmin.md b/docs/ru/sql-reference/aggregate-functions/reference/argmin.md index 4ee78a73a84..6205cd0cf09 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/argmin.md @@ -14,12 +14,6 @@ toc_priority: 105 argMin(arg, val) ``` -или - -``` sql -argMin(tuple(arg, val)) -``` - **Аргументы** - `arg` — аргумент. From 28b96ff20f8f9398f6d8efa5cdd0a73a6a6210d1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Oct 2021 18:38:20 +0300 Subject: [PATCH 225/919] Update gtest_coordination.cpp --- src/Coordination/tests/gtest_coordination.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 86b36c2bf08..12902252328 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -962,7 +962,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) auto debuf = manager.deserializeSnapshotBufferFromDisk(2); - auto [restored_storage, snapshot_meta, _] = manager.deserializeSnapshotFromBuffer(debuf); + auto [restored_storage, snapshot_meta, _] = manager.deserializeSnapshotFromBuffer(debuf); EXPECT_EQ(restored_storage->container.size(), 3); EXPECT_EQ(restored_storage->container.getValue("/").children.size(), 1); From 0f2620a31ef37179c2009385835e490fc3baf89e Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Tue, 19 Oct 2021 19:07:04 +0300 Subject: [PATCH 226/919] Update argmin.md --- .../aggregate-functions/reference/argmin.md | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/argmin.md b/docs/ru/sql-reference/aggregate-functions/reference/argmin.md index 6205cd0cf09..4d2363abe6d 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/argmin.md @@ -25,12 +25,6 @@ argMin(arg, val) Тип: соответствует типу `arg`. -Если передан кортеж: - -- Кортеж `(arg, val)` c минимальным значением `val` и соответствующим ему `arg`. - -Тип: [Tuple](../../../sql-reference/data-types/tuple.md). - **Пример** Исходная таблица: @@ -46,14 +40,14 @@ argMin(arg, val) Запрос: ``` sql -SELECT argMin(user, salary), argMin(tuple(user, salary)) FROM salary; +SELECT argMin(user, salary) FROM salary; ``` Результат: ``` text -┌─argMin(user, salary)─┬─argMin(tuple(user, salary))─┐ -│ worker │ ('worker',1000) │ -└──────────────────────┴─────────────────────────────┘ +┌─argMin(user, salary)─┐ +│ worker │ +└──────────────────────┘ ``` From c6af087f4bf89dd27f06cd6bb9972f6fc89e781b Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Tue, 19 Oct 2021 19:08:50 +0300 Subject: [PATCH 227/919] Update argmax.md --- .../aggregate-functions/reference/argmax.md | 20 ++++--------------- 1 file changed, 4 insertions(+), 16 deletions(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/argmax.md b/docs/ru/sql-reference/aggregate-functions/reference/argmax.md index 71289423035..84419523beb 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/argmax.md @@ -14,12 +14,6 @@ toc_priority: 106 argMax(arg, val) ``` -или - -``` sql -argMax(tuple(arg, val)) -``` - **Аргументы** - `arg` — аргумент. @@ -31,12 +25,6 @@ argMax(tuple(arg, val)) Тип: соответствует типу `arg`. -Если передан кортеж: - -- кортеж `(arg, val)` c максимальным значением `val` и соответствующим ему `arg`. - -Тип: [Tuple](../../../sql-reference/data-types/tuple.md). - **Пример** Исходная таблица: @@ -52,14 +40,14 @@ argMax(tuple(arg, val)) Запрос: ``` sql -SELECT argMax(user, salary), argMax(tuple(user, salary), salary), argMax(tuple(user, salary)) FROM salary; +SELECT argMax(user, salary), argMax(tuple(user, salary), salary) FROM salary; ``` Результат: ``` text -┌─argMax(user, salary)─┬─argMax(tuple(user, salary), salary)─┬─argMax(tuple(user, salary))─┐ -│ director │ ('director',5000) │ ('director',5000) │ -└──────────────────────┴─────────────────────────────────────┴─────────────────────────────┘ +┌─argMax(user, salary)─┬─argMax(tuple(user, salary), salary)─┐ +│ director │ ('director',5000) │ +└──────────────────────┴─────────────────────────────────────┘ ``` From f49e946943fe6bd6d9da9d0b1a99539bd5eac37a Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Tue, 19 Oct 2021 19:11:15 +0300 Subject: [PATCH 228/919] Update argmax.md --- .../aggregate-functions/reference/argmax.md | 20 ++++--------------- 1 file changed, 4 insertions(+), 16 deletions(-) diff --git a/docs/zh/sql-reference/aggregate-functions/reference/argmax.md b/docs/zh/sql-reference/aggregate-functions/reference/argmax.md index 9d90590b2f1..0c82cb8a4bb 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/argmax.md @@ -14,12 +14,6 @@ toc_priority: 106 argMax(arg, val) ``` -或 - -``` sql -argMax(tuple(arg, val)) -``` - **参数** - `arg` — Argument. @@ -31,12 +25,6 @@ argMax(tuple(arg, val)) 类型: 匹配 `arg` 类型。 -对于输入中的元组: - -- 元组 `(arg, val)`, 其中 `val` 最大值,`arg` 是对应的值。 - -类型: [元组](../../../sql-reference/data-types/tuple.md)。 - **示例** 输入表: @@ -52,13 +40,13 @@ argMax(tuple(arg, val)) 查询: ``` sql -SELECT argMax(user, salary), argMax(tuple(user, salary), salary), argMax(tuple(user, salary)) FROM salary; +SELECT argMax(user, salary), argMax(tuple(user, salary), salary) FROM salary; ``` 结果: ``` text -┌─argMax(user, salary)─┬─argMax(tuple(user, salary), salary)─┬─argMax(tuple(user, salary))─┐ -│ director │ ('director',5000) │ ('director',5000) │ -└──────────────────────┴─────────────────────────────────────┴─────────────────────────────┘ +┌─argMax(user, salary)─┬─argMax(tuple(user, salary), salary)─┐ +│ director │ ('director',5000) │ +└──────────────────────┴─────────────────────────────────────┘ ``` From 2b3841f0030c0a702ea88ef6fe4499f9ad6d0705 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Tue, 19 Oct 2021 19:14:14 +0300 Subject: [PATCH 229/919] Update argmin.md --- .../sql-reference/aggregate-functions/reference/argmin.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/zh/sql-reference/aggregate-functions/reference/argmin.md b/docs/zh/sql-reference/aggregate-functions/reference/argmin.md index 0dd4625ac0d..6c2839062eb 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/argmin.md @@ -25,13 +25,13 @@ toc_priority: 105 查询: ``` sql -SELECT argMin(user, salary), argMin(tuple(user, salary)) FROM salary; +SELECT argMin(user, salary), argMin(tuple(user, salary), salary) FROM salary; ``` 结果: ``` text -┌─argMin(user, salary)─┬─argMin(tuple(user, salary))─┐ -│ worker │ ('worker',1000) │ -└──────────────────────┴─────────────────────────────┘ +┌─argMin(user, salary)─┬─argMin(tuple(user, salary), salary)─┐ +│ worker │ ('worker',1000) │ +└──────────────────────┴─────────────────────────────────────┘ ``` From facfa16978a2ef625761e1c128dae41fa3f2a737 Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Tue, 19 Oct 2021 21:40:04 +0300 Subject: [PATCH 230/919] check of github update removed the hyphen --- docs/ru/faq/general/olap.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/faq/general/olap.md b/docs/ru/faq/general/olap.md index 42715a195ad..b5eb8758b66 100644 --- a/docs/ru/faq/general/olap.md +++ b/docs/ru/faq/general/olap.md @@ -19,7 +19,7 @@ toc_priority: 100 ## OLAP с точки зрения бизнеса {#olap-from-the-business-perspective} -В последние годы бизнес-сообщество стало осознавать ценность данных. Компании, которые принимают решения вслепую, чаще всего отстают от конкурентов. Управление бизнесом на основе данных, которое применяется успешными компаниями, побуждает собирать все данные, которые могут быть полезны в будущем для принятия бизнес-решений, а также подбирать механизмы, чтобы своевременно эти данные анализировать. Именно для этого и нужны СУБД с OLAP. +В последние годы бизнес сообщество стало осознавать ценность данных. Компании, которые принимают решения вслепую, чаще всего отстают от конкурентов. Управление бизнесом на основе данных, которое применяется успешными компаниями, побуждает собирать все данные, которые могут быть полезны в будущем для принятия бизнес-решений, а также подбирать механизмы, чтобы своевременно эти данные анализировать. Именно для этого и нужны СУБД с OLAP. С точки зрения бизнеса, OLAP позволяет компаниям постоянно планировать, анализировать и оценивать операционную деятельность, чтобы повышать её эффективность, уменьшать затраты и как следствие — увеличивать долю рынка. Это можно делать как в собственной системе, так и в облачной (SaaS), в веб или мобильных аналитических приложениях, CRM-системах и т.д. Технология OLAP используется во многих приложениях BI (Business Intelligence — бизнес-аналитика). From a1a4715ad1cbf0037ef35667c695dc974241a994 Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Tue, 19 Oct 2021 21:46:04 +0300 Subject: [PATCH 231/919] mistake --- docs/ru/faq/general/olap.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/faq/general/olap.md b/docs/ru/faq/general/olap.md index b5eb8758b66..42715a195ad 100644 --- a/docs/ru/faq/general/olap.md +++ b/docs/ru/faq/general/olap.md @@ -19,7 +19,7 @@ toc_priority: 100 ## OLAP с точки зрения бизнеса {#olap-from-the-business-perspective} -В последние годы бизнес сообщество стало осознавать ценность данных. Компании, которые принимают решения вслепую, чаще всего отстают от конкурентов. Управление бизнесом на основе данных, которое применяется успешными компаниями, побуждает собирать все данные, которые могут быть полезны в будущем для принятия бизнес-решений, а также подбирать механизмы, чтобы своевременно эти данные анализировать. Именно для этого и нужны СУБД с OLAP. +В последние годы бизнес-сообщество стало осознавать ценность данных. Компании, которые принимают решения вслепую, чаще всего отстают от конкурентов. Управление бизнесом на основе данных, которое применяется успешными компаниями, побуждает собирать все данные, которые могут быть полезны в будущем для принятия бизнес-решений, а также подбирать механизмы, чтобы своевременно эти данные анализировать. Именно для этого и нужны СУБД с OLAP. С точки зрения бизнеса, OLAP позволяет компаниям постоянно планировать, анализировать и оценивать операционную деятельность, чтобы повышать её эффективность, уменьшать затраты и как следствие — увеличивать долю рынка. Это можно делать как в собственной системе, так и в облачной (SaaS), в веб или мобильных аналитических приложениях, CRM-системах и т.д. Технология OLAP используется во многих приложениях BI (Business Intelligence — бизнес-аналитика). From 225f16fd549215fa79f026ea32341b7cb7ea5065 Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Tue, 19 Oct 2021 21:46:46 +0300 Subject: [PATCH 232/919] checking --- docs/ru/faq/general/olap.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/faq/general/olap.md b/docs/ru/faq/general/olap.md index 42715a195ad..b5eb8758b66 100644 --- a/docs/ru/faq/general/olap.md +++ b/docs/ru/faq/general/olap.md @@ -19,7 +19,7 @@ toc_priority: 100 ## OLAP с точки зрения бизнеса {#olap-from-the-business-perspective} -В последние годы бизнес-сообщество стало осознавать ценность данных. Компании, которые принимают решения вслепую, чаще всего отстают от конкурентов. Управление бизнесом на основе данных, которое применяется успешными компаниями, побуждает собирать все данные, которые могут быть полезны в будущем для принятия бизнес-решений, а также подбирать механизмы, чтобы своевременно эти данные анализировать. Именно для этого и нужны СУБД с OLAP. +В последние годы бизнес сообщество стало осознавать ценность данных. Компании, которые принимают решения вслепую, чаще всего отстают от конкурентов. Управление бизнесом на основе данных, которое применяется успешными компаниями, побуждает собирать все данные, которые могут быть полезны в будущем для принятия бизнес-решений, а также подбирать механизмы, чтобы своевременно эти данные анализировать. Именно для этого и нужны СУБД с OLAP. С точки зрения бизнеса, OLAP позволяет компаниям постоянно планировать, анализировать и оценивать операционную деятельность, чтобы повышать её эффективность, уменьшать затраты и как следствие — увеличивать долю рынка. Это можно делать как в собственной системе, так и в облачной (SaaS), в веб или мобильных аналитических приложениях, CRM-системах и т.д. Технология OLAP используется во многих приложениях BI (Business Intelligence — бизнес-аналитика). From 377f6cf3c251bfc27f2f3c2ccf162c0ef8dab903 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Tue, 19 Oct 2021 21:51:46 +0300 Subject: [PATCH 233/919] Edit and translate to Russian MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Выполнил перевод на русский язык. --- .../external-dicts-dict-layout.md | 14 +++-- .../external-dicts-dict-layout.md | 60 ++++++++++++++++++- 2 files changed, 69 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index c4a359efca4..766ce9a7a7e 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -52,6 +52,7 @@ LAYOUT(LAYOUT_TYPE(param value)) -- layout settings - [flat](#flat) - [hashed](#dicts-external_dicts_dict_layout-hashed) +- [hashed_array](#dicts-external_dicts_dict_layout-hashed-array) - [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) - [cache](#cache) - [ssd_cache](#ssd-cache) @@ -151,13 +152,15 @@ Configuration example:
``` +or + ``` sql LAYOUT(COMPLEX_KEY_HASHED()) ``` ### complex_key_sparse_hashed {#complex-key-sparse-hashed} -This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `sparse_hashed`. +This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed). Configuration example: @@ -167,13 +170,15 @@ Configuration example:
``` +or + ``` sql LAYOUT(COMPLEX_KEY_SPARSE_HASHED()) ``` ### hashed_array {#dicts-external_dicts_dict_layout-hashed-array} -The dictionary is completely stored in memory. Each attribute is stored in array. Key attribute is stored in the form of hashed table where value is index in attributes array. The dictionary can contain any number of elements with any identifiers In practice, the number of keys can reach tens of millions of items. +The dictionary is completely stored in memory. Each attribute is stored in an array. The key attribute is stored in the form of a hashed table where value is an index in the attributes array. The dictionary can contain any number of elements with any identifiers. In practice, the number of keys can reach tens of millions of items. All types of sources are supported. When updating, data (from a file or from a table) is read in its entirety. @@ -194,7 +199,7 @@ LAYOUT(HASHED_ARRAY()) ### complex_key_hashed_array {#complex-key-hashed-array} -This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `hashed_array`. +This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to [hashed_array](#dicts-external_dicts_dict_layout-hashed-array). Configuration example: @@ -204,11 +209,12 @@ Configuration example: ``` +or + ``` sql LAYOUT(COMPLEX_KEY_HASHED_ARRAY()) ``` - ### range_hashed {#range-hashed} The dictionary is stored in memory in the form of a hash table with an ordered array of ranges and their corresponding values. diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 73736344c57..82937a90ee5 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -52,6 +52,7 @@ LAYOUT(LAYOUT_TYPE(param value)) -- layout settings - [flat](#flat) - [hashed](#dicts-external_dicts_dict_layout-hashed) +- [hashed_array](#dicts-external_dicts_dict_layout-hashed-array) - [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) - [cache](#cache) - [ssd_cache](#ssd-cache) @@ -140,7 +141,7 @@ LAYOUT(SPARSE_HASHED([PREALLOCATE 0])) ### complex_key_hashed {#complex-key-hashed} -Тип размещения предназначен для использования с составными [ключами](external-dicts-dict-structure.md). Аналогичен `hashed`. +Тип размещения предназначен для использования с составными [ключами](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Аналогичен `hashed`. Пример конфигурации: @@ -156,6 +157,63 @@ LAYOUT(SPARSE_HASHED([PREALLOCATE 0])) LAYOUT(COMPLEX_KEY_HASHED()) ``` +### complex_key_sparse_hashed {#complex-key-sparse-hashed} + +Тип размещения предназначен для использования с составными [ключами](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Аналогичен [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed). + +Пример конфигурации: + +``` xml + + + +``` + +или + +``` sql +LAYOUT(COMPLEX_KEY_SPARSE_HASHED()) +``` + +### hashed_array {#dicts-external_dicts_dict_layout-hashed-array} + +Словарь полностью хранится в оперативной памяти. Каждый атрибут хранится в массиве. Ключевой атрибут хранится в виде хэш-таблицы, где его значение является индексом в массиве атрибутов. Словарь может содержать произвольное количество элементов с произвольными идентификаторами. На практике количество ключей может достигать десятков миллионов элементов. + +Поддерживаются все виды источников. При обновлении данные (из файла, из таблицы) читаются целиком. + +Пример конфигурации: + +``` xml + + + + +``` + +или + +``` sql +LAYOUT(HASHED_ARRAY()) +``` + +### complex_key_hashed_array {#complex-key-hashed-array} + +Тип размещения предназначен для использования с составными [ключами](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Аналогичен [hashed_array](#dicts-external_dicts_dict_layout-hashed-array). + +Пример конфигурации: + +``` xml + + + +``` + +или + +``` sql +LAYOUT(COMPLEX_KEY_HASHED_ARRAY()) +``` + ### range_hashed {#range-hashed} Словарь хранится в оперативной памяти в виде хэш-таблицы с упорядоченным массивом диапазонов и соответствующих им значений. From 1b763284a23cbe30adb3b687fe71ed3cef962258 Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Tue, 19 Oct 2021 21:56:11 +0300 Subject: [PATCH 234/919] check of update --- docs/ru/faq/general/olap.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/faq/general/olap.md b/docs/ru/faq/general/olap.md index b5eb8758b66..37f45a776b0 100644 --- a/docs/ru/faq/general/olap.md +++ b/docs/ru/faq/general/olap.md @@ -19,7 +19,7 @@ toc_priority: 100 ## OLAP с точки зрения бизнеса {#olap-from-the-business-perspective} -В последние годы бизнес сообщество стало осознавать ценность данных. Компании, которые принимают решения вслепую, чаще всего отстают от конкурентов. Управление бизнесом на основе данных, которое применяется успешными компаниями, побуждает собирать все данные, которые могут быть полезны в будущем для принятия бизнес-решений, а также подбирать механизмы, чтобы своевременно эти данные анализировать. Именно для этого и нужны СУБД с OLAP. +В последние годы бизнессообщество стало осознавать ценность данных. Компании, которые принимают решения вслепую, чаще всего отстают от конкурентов. Управление бизнесом на основе данных, которое применяется успешными компаниями, побуждает собирать все данные, которые могут быть полезны в будущем для принятия бизнес-решений, а также подбирать механизмы, чтобы своевременно эти данные анализировать. Именно для этого и нужны СУБД с OLAP. С точки зрения бизнеса, OLAP позволяет компаниям постоянно планировать, анализировать и оценивать операционную деятельность, чтобы повышать её эффективность, уменьшать затраты и как следствие — увеличивать долю рынка. Это можно делать как в собственной системе, так и в облачной (SaaS), в веб или мобильных аналитических приложениях, CRM-системах и т.д. Технология OLAP используется во многих приложениях BI (Business Intelligence — бизнес-аналитика). From 67ca1a29183c203243d895c556bbe4690f4ce394 Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Tue, 19 Oct 2021 22:15:27 +0300 Subject: [PATCH 235/919] return right text --- docs/ru/faq/general/olap.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/faq/general/olap.md b/docs/ru/faq/general/olap.md index 37f45a776b0..42715a195ad 100644 --- a/docs/ru/faq/general/olap.md +++ b/docs/ru/faq/general/olap.md @@ -19,7 +19,7 @@ toc_priority: 100 ## OLAP с точки зрения бизнеса {#olap-from-the-business-perspective} -В последние годы бизнессообщество стало осознавать ценность данных. Компании, которые принимают решения вслепую, чаще всего отстают от конкурентов. Управление бизнесом на основе данных, которое применяется успешными компаниями, побуждает собирать все данные, которые могут быть полезны в будущем для принятия бизнес-решений, а также подбирать механизмы, чтобы своевременно эти данные анализировать. Именно для этого и нужны СУБД с OLAP. +В последние годы бизнес-сообщество стало осознавать ценность данных. Компании, которые принимают решения вслепую, чаще всего отстают от конкурентов. Управление бизнесом на основе данных, которое применяется успешными компаниями, побуждает собирать все данные, которые могут быть полезны в будущем для принятия бизнес-решений, а также подбирать механизмы, чтобы своевременно эти данные анализировать. Именно для этого и нужны СУБД с OLAP. С точки зрения бизнеса, OLAP позволяет компаниям постоянно планировать, анализировать и оценивать операционную деятельность, чтобы повышать её эффективность, уменьшать затраты и как следствие — увеличивать долю рынка. Это можно делать как в собственной системе, так и в облачной (SaaS), в веб или мобильных аналитических приложениях, CRM-системах и т.д. Технология OLAP используется во многих приложениях BI (Business Intelligence — бизнес-аналитика). From b7fdf488e16b9f5023358bd22932a8b000be4abb Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Tue, 19 Oct 2021 22:38:00 +0300 Subject: [PATCH 236/919] Translate install.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Перевел на русский язык предложение в install.md. --- docs/en/getting-started/install.md | 3 +-- docs/ru/getting-started/install.md | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 06186842809..4a97ab6589d 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -27,11 +27,10 @@ It is recommended to use official pre-compiled `deb` packages for Debian or Ubun {% include 'install/deb.sh' %} ``` -You can replace `stable` with `lts` or `testing` to use different [“release trains”](../faq/operations/production.md) based on your needs. +You can replace `stable` with `lts` or `testing` to use different [release trains](../faq/operations/production.md) based on your needs. You can also download and install packages manually from [here](https://repo.clickhouse.com/deb/stable/main/). - #### Packages {#packages} - `clickhouse-common-static` — Installs ClickHouse compiled binary files. diff --git a/docs/ru/getting-started/install.md b/docs/ru/getting-started/install.md index 5cb75c12155..a12773a75b0 100644 --- a/docs/ru/getting-started/install.md +++ b/docs/ru/getting-started/install.md @@ -29,7 +29,7 @@ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not su Также эти пакеты можно скачать и установить вручную отсюда: https://repo.clickhouse.com/deb/stable/main/. -Если вы хотите использовать наиболее свежую версию, замените `stable` на `testing` (рекомендуется для тестовых окружений). +Чтобы использовать различные [версии ClickHouse](../faq/operations/production.md) в зависимости от ваших потребностей, вы можете заменить `stable` на `lts` или `testing`. Также вы можете вручную скачать и установить пакеты из [репозитория](https://repo.clickhouse.com/deb/stable/main/). From 143e912e4e2b1a7be4c9220bd781c3b1a854998b Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 19 Oct 2021 20:41:52 +0300 Subject: [PATCH 237/919] Fix split build --- src/CMakeLists.txt | 1 + src/Compression/CompressedReadBufferBase.cpp | 1 - src/Disks/DiskWebServer.cpp | 14 +++++----- src/Disks/HDFS/DiskHDFS.cpp | 10 ++++--- src/Disks/IDiskRemote.cpp | 2 +- ...chronousReadIndirectBufferFromRemoteFS.cpp | 27 ++++++++++--------- ...ynchronousReadIndirectBufferFromRemoteFS.h | 6 ++--- .../{ => IO}/ReadBufferFromRemoteFSGather.cpp | 8 +----- .../{ => IO}/ReadBufferFromRemoteFSGather.h | 0 .../IO/ReadBufferFromWebServer.cpp | 7 ++--- src/{ => Disks}/IO/ReadBufferFromWebServer.h | 0 .../IO/ReadIndirectBufferFromRemoteFS.cpp | 2 +- .../IO/ReadIndirectBufferFromRemoteFS.h | 0 .../IO/ThreadPoolRemoteFSReader.cpp | 4 +-- src/{ => Disks}/IO/ThreadPoolRemoteFSReader.h | 0 .../IO/WriteIndirectBufferFromRemoteFS.cpp | 0 .../IO/WriteIndirectBufferFromRemoteFS.h | 0 src/Disks/S3/DiskS3.cpp | 10 ++++--- 18 files changed, 47 insertions(+), 45 deletions(-) rename src/{ => Disks}/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp (93%) rename src/{ => Disks}/IO/AsynchronousReadIndirectBufferFromRemoteFS.h (92%) rename src/Disks/{ => IO}/ReadBufferFromRemoteFSGather.cpp (97%) rename src/Disks/{ => IO}/ReadBufferFromRemoteFSGather.h (100%) rename src/{ => Disks}/IO/ReadBufferFromWebServer.cpp (97%) rename src/{ => Disks}/IO/ReadBufferFromWebServer.h (100%) rename src/{ => Disks}/IO/ReadIndirectBufferFromRemoteFS.cpp (97%) rename src/{ => Disks}/IO/ReadIndirectBufferFromRemoteFS.h (100%) rename src/{ => Disks}/IO/ThreadPoolRemoteFSReader.cpp (95%) rename src/{ => Disks}/IO/ThreadPoolRemoteFSReader.h (100%) rename src/{ => Disks}/IO/WriteIndirectBufferFromRemoteFS.cpp (100%) rename src/{ => Disks}/IO/WriteIndirectBufferFromRemoteFS.h (100%) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index cac5b70f489..44c72693e49 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -77,6 +77,7 @@ add_headers_and_sources(clickhouse_common_io IO) add_headers_and_sources(clickhouse_common_io IO/S3) list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_delete.cpp) +add_headers_and_sources(dbms Disks/IO) if (USE_SQLITE) add_headers_and_sources(dbms Databases/SQLite) endif() diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index 9e3b22fac74..c9ee751b202 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Disks/DiskWebServer.cpp b/src/Disks/DiskWebServer.cpp index 148e34cf9c5..009bae355ce 100644 --- a/src/Disks/DiskWebServer.cpp +++ b/src/Disks/DiskWebServer.cpp @@ -3,17 +3,17 @@ #include #include -#include -#include - #include -#include #include #include #include -#include -#include -#include + +#include +#include +#include +#include +#include +#include #include diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index cb3e1e00277..2867fc5de34 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -1,12 +1,14 @@ #include #include -#include -#include -#include -#include #include +#include +#include +#include +#include +#include + #include #include diff --git a/src/Disks/IDiskRemote.cpp b/src/Disks/IDiskRemote.cpp index 428de44259c..cf1baafce6c 100644 --- a/src/Disks/IDiskRemote.cpp +++ b/src/Disks/IDiskRemote.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp similarity index 93% rename from src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp rename to src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp index 08beed079e8..a5661decaa9 100644 --- a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -1,7 +1,8 @@ #include "AsynchronousReadIndirectBufferFromRemoteFS.h" #include -#include +#include +#include #include @@ -46,7 +47,13 @@ AsynchronousReadIndirectBufferFromRemoteFS::AsynchronousReadIndirectBufferFromRe , min_bytes_for_seek(min_bytes_for_seek_) { ProfileEvents::increment(ProfileEvents::RemoteFSAsyncBuffers); - buffer_events += impl->getFileName() + " : "; + buffer_events += "Events for buffer: " + impl->getFileName() + " : "; +} + + +String AsynchronousReadIndirectBufferFromRemoteFS::getFileName() const +{ + return impl->getFileName(); } @@ -92,9 +99,9 @@ void AsynchronousReadIndirectBufferFromRemoteFS::prefetch() } -void AsynchronousReadIndirectBufferFromRemoteFS::setReadUntilPosition(size_t offset) +void AsynchronousReadIndirectBufferFromRemoteFS::setReadUntilPosition(size_t position) { - buffer_events += "-- Set last offset " + toString(offset) + "--"; + buffer_events += "-- Set last offset " + toString(position) + "--"; if (prefetch_future.valid()) { LOG_DEBUG(&Poco::Logger::get("kssenii"), buffer_events); @@ -108,13 +115,14 @@ void AsynchronousReadIndirectBufferFromRemoteFS::setReadUntilPosition(size_t off // prefetch_future = {}; } - last_offset = offset; - impl->setReadUntilPosition(offset); + last_offset = position; + impl->setReadUntilPosition(position); } bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() { + LOG_DEBUG(&Poco::Logger::get("kssenii"), buffer_events); /// Everything is already read. if (absolute_position == last_offset) return false; @@ -163,12 +171,7 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() prefetch_future = {}; - /// TODO: it does not really seem to improve anything to call prefetch() here, - /// but it does not make any worse at the same time. - /// Need to test, it might be useful because in fact sometimes (minority of cases though) - /// we can read without prefetching several times in a row. - prefetch(); - + LOG_DEBUG(&Poco::Logger::get("kssenii"), buffer_events); return size; } diff --git a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h similarity index 92% rename from src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h rename to src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h index f1cfbfde8f6..6842d62f22a 100644 --- a/src/IO/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -6,14 +6,14 @@ #include #include -#include -#include #include namespace DB { +class ReadBufferFromRemoteFSGather; + /** * Reads data from S3/HDFS/Web using stored paths in metadata. * This class is an asynchronous version of ReadIndirectBufferFromRemoteFS. @@ -43,7 +43,7 @@ public: off_t getPosition() override { return absolute_position - available(); } - String getFileName() const override { return impl->getFileName(); } + String getFileName() const override; void prefetch() override; diff --git a/src/Disks/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp similarity index 97% rename from src/Disks/ReadBufferFromRemoteFSGather.cpp rename to src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index ff9231bf9d3..c12af031089 100644 --- a/src/Disks/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #if USE_AWS_S3 #include @@ -21,12 +21,6 @@ namespace fs = std::filesystem; namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_SEEK_THROUGH_FILE; -} - - #if USE_AWS_S3 SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const String & path, size_t offset) const { diff --git a/src/Disks/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h similarity index 100% rename from src/Disks/ReadBufferFromRemoteFSGather.h rename to src/Disks/IO/ReadBufferFromRemoteFSGather.h diff --git a/src/IO/ReadBufferFromWebServer.cpp b/src/Disks/IO/ReadBufferFromWebServer.cpp similarity index 97% rename from src/IO/ReadBufferFromWebServer.cpp rename to src/Disks/IO/ReadBufferFromWebServer.cpp index 245364a9896..69683edec4c 100644 --- a/src/IO/ReadBufferFromWebServer.cpp +++ b/src/Disks/IO/ReadBufferFromWebServer.cpp @@ -17,6 +17,7 @@ namespace ErrorCodes { extern const int CANNOT_SEEK_THROUGH_FILE; extern const int SEEK_POSITION_OUT_OF_BOUND; + extern const int LOGICAL_ERROR; } @@ -49,7 +50,7 @@ std::unique_ptr ReadBufferFromWebServer::initialize() if (last_offset) { if (last_offset < offset) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyound right offset ({} > {})", offset, last_offset - 1); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, last_offset - 1); headers.emplace_back(std::make_pair("Range", fmt::format("bytes={}-{}", offset, last_offset - 1))); LOG_DEBUG(log, "Reading with range: {}-{}", offset, last_offset); @@ -133,7 +134,7 @@ bool ReadBufferFromWebServer::nextImpl() return false; if (last_offset < offset) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyound right offset ({} > {})", offset, last_offset - 1); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, last_offset - 1); } if (impl) @@ -155,7 +156,7 @@ bool ReadBufferFromWebServer::nextImpl() { /** * impl was initialized before, pass position() to it to make - * sure there is no pending data which was not read, becuase + * sure there is no pending data which was not read, because * this branch means we read sequentially. */ impl->position() = position(); diff --git a/src/IO/ReadBufferFromWebServer.h b/src/Disks/IO/ReadBufferFromWebServer.h similarity index 100% rename from src/IO/ReadBufferFromWebServer.h rename to src/Disks/IO/ReadBufferFromWebServer.h diff --git a/src/IO/ReadIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp similarity index 97% rename from src/IO/ReadIndirectBufferFromRemoteFS.cpp rename to src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp index d70d280871a..f0133e0fafa 100644 --- a/src/IO/ReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp @@ -1,6 +1,6 @@ #include "ReadIndirectBufferFromRemoteFS.h" -#include +#include namespace ProfileEvents diff --git a/src/IO/ReadIndirectBufferFromRemoteFS.h b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.h similarity index 100% rename from src/IO/ReadIndirectBufferFromRemoteFS.h rename to src/Disks/IO/ReadIndirectBufferFromRemoteFS.h diff --git a/src/IO/ThreadPoolRemoteFSReader.cpp b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp similarity index 95% rename from src/IO/ThreadPoolRemoteFSReader.cpp rename to src/Disks/IO/ThreadPoolRemoteFSReader.cpp index 258d20d62e9..976162ab625 100644 --- a/src/IO/ThreadPoolRemoteFSReader.cpp +++ b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp @@ -1,4 +1,4 @@ -#include +#include "ThreadPoolRemoteFSReader.h" #include #include @@ -8,7 +8,7 @@ #include #include -#include +#include #include #include diff --git a/src/IO/ThreadPoolRemoteFSReader.h b/src/Disks/IO/ThreadPoolRemoteFSReader.h similarity index 100% rename from src/IO/ThreadPoolRemoteFSReader.h rename to src/Disks/IO/ThreadPoolRemoteFSReader.h diff --git a/src/IO/WriteIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp similarity index 100% rename from src/IO/WriteIndirectBufferFromRemoteFS.cpp rename to src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp diff --git a/src/IO/WriteIndirectBufferFromRemoteFS.h b/src/Disks/IO/WriteIndirectBufferFromRemoteFS.h similarity index 100% rename from src/IO/WriteIndirectBufferFromRemoteFS.h rename to src/Disks/IO/WriteIndirectBufferFromRemoteFS.h diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index b6c94ca0802..e04802d6b8c 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -17,17 +17,19 @@ #include #include -#include #include #include -#include #include #include #include #include #include -#include -#include + +#include +#include +#include +#include +#include #include // Y_IGNORE #include // Y_IGNORE From e530682529b09318bf711b614ebba74772adf1e4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 19 Oct 2021 22:42:40 +0300 Subject: [PATCH 238/919] Build fixes --- src/Coordination/KeeperServer.h | 4 ++-- src/Coordination/KeeperStateMachine.cpp | 2 +- src/Coordination/KeeperStateManager.cpp | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index e25f240ecf7..7e4c5381f6e 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -95,12 +95,12 @@ public: /// Apply action for configuration update. Actually call raft_instance->remove_srv or raft_instance->add_srv. /// Synchronously check for update results with retries. - void applyConfigurationUpdate(const ConfigUpdateAction & action); + void applyConfigurationUpdate(const ConfigUpdateAction & task); /// Wait configuration update for action. Used by followers. /// Return true if update was successfully received. - bool waitConfigurationUpdate(const ConfigUpdateAction & action); + bool waitConfigurationUpdate(const ConfigUpdateAction & task); }; } diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 12ec2fb1470..62820b41731 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -166,7 +166,7 @@ bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s) } -void KeeperStateMachine::commit_config(const ulong /*log_idx*/, nuraft::ptr & new_conf) +void KeeperStateMachine::commit_config(const uint64_t /*log_idx*/, nuraft::ptr & new_conf) { std::lock_guard lock(cluster_config_lock); auto tmp = new_conf->serialize(); diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 6a8d424a345..31b3a5f8b87 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -141,12 +141,12 @@ ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::A auto new_configuration_wrapper = parseServersConfiguration(config, true); std::unordered_map new_ids, old_ids; - for (auto new_server : new_configuration_wrapper.cluster_config->get_servers()) + for (const auto & new_server : new_configuration_wrapper.cluster_config->get_servers()) new_ids[new_server->get_id()] = new_server; { std::lock_guard lock(configuration_wrapper_mutex); - for (auto old_server : configuration_wrapper.cluster_config->get_servers()) + for (const auto & old_server : configuration_wrapper.cluster_config->get_servers()) old_ids[old_server->get_id()] = old_server; } From 78e5727bfdd97439b99b033366eb711a2a66f681 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Tue, 19 Oct 2021 23:47:29 +0300 Subject: [PATCH 239/919] Update docs/ru/operations/settings/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 592a669ccd6..e70ccd343f4 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1713,7 +1713,7 @@ ClickHouse генерирует исключение Возможные значения: -- 0 — Выключена. +- 0 — выключена. - 1 — Включена. Значение по умолчанию: `1`. From b3adb559cd66d8f2891a3ffb6afe7dddcc2dd514 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Tue, 19 Oct 2021 23:47:34 +0300 Subject: [PATCH 240/919] Update docs/ru/operations/settings/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index e70ccd343f4..795249b5499 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1714,7 +1714,7 @@ ClickHouse генерирует исключение Возможные значения: - 0 — выключена. -- 1 — Включена. +- 1 — включена. Значение по умолчанию: `1`. From deeb00f75a24546014725bdd33b0b9980ee917e4 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Tue, 19 Oct 2021 23:47:46 +0300 Subject: [PATCH 241/919] Update docs/ru/sql-reference/functions/tuple-map-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/functions/tuple-map-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/tuple-map-functions.md b/docs/ru/sql-reference/functions/tuple-map-functions.md index 5348e175269..35a765cf3f7 100644 --- a/docs/ru/sql-reference/functions/tuple-map-functions.md +++ b/docs/ru/sql-reference/functions/tuple-map-functions.md @@ -192,7 +192,7 @@ mapPopulateSeries(map[, max]) или -- `map` — контейнер Map с целочисленными ключами. [Map](../../sql-reference/data-types/map.md). +- `map` — контейнер `Map` с целочисленными ключами. [Map](../../sql-reference/data-types/map.md). **Возвращаемое значение** From 6be11192f49c513584aba065518092a407f7dfbe Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Tue, 19 Oct 2021 23:47:56 +0300 Subject: [PATCH 242/919] Update docs/ru/sql-reference/functions/tuple-map-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/functions/tuple-map-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/tuple-map-functions.md b/docs/ru/sql-reference/functions/tuple-map-functions.md index 35a765cf3f7..ccd21d42826 100644 --- a/docs/ru/sql-reference/functions/tuple-map-functions.md +++ b/docs/ru/sql-reference/functions/tuple-map-functions.md @@ -134,7 +134,7 @@ mapSubtract(Tuple(Array, Array), Tuple(Array, Array) [, ...]) **Возвращаемое значение** -- В зависимости от аргумента возвращает один [Map](../../sql-reference/data-types/map.md) или [кортеж](../../sql-reference/data-types/tuple.md#tuplet1-t2), в котором первый массив содержит отсортированные ключи, а второй - значения. +- В зависимости от аргумента возвращает один [Map](../../sql-reference/data-types/map.md) или [кортеж](../../sql-reference/data-types/tuple.md#tuplet1-t2), в котором первый массив содержит отсортированные ключи, а второй — значения. **Пример** From 1ed3913ae60dbb0e0608cf8dd036d22916e93d0d Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Tue, 19 Oct 2021 23:48:42 +0300 Subject: [PATCH 243/919] Update docs/en/operations/settings/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 65940172f0b..a9ee6157c81 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1792,7 +1792,7 @@ Enables or disables [LIMIT](#limit) applying on each shard separatelly. Usually Possible values: -- 0 - Disabled. +- 0 — Disabled. - 1 - Enabled. Default value: `1`. From 36a8ae6af0231310a0b964c3825e5a5cc70fdcd9 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Tue, 19 Oct 2021 23:48:47 +0300 Subject: [PATCH 244/919] Update docs/en/operations/settings/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index a9ee6157c81..f81cbb989ee 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1793,7 +1793,7 @@ Enables or disables [LIMIT](#limit) applying on each shard separatelly. Usually Possible values: - 0 — Disabled. -- 1 - Enabled. +- 1 — Enabled. Default value: `1`. From 2cfc6a5fcfee915ce23d59a0ab924fb910ef92a5 Mon Sep 17 00:00:00 2001 From: Alexey Date: Tue, 19 Oct 2021 20:50:30 +0000 Subject: [PATCH 245/919] DESCRIBE refactored. Example updated and moved. --- docs/en/operations/settings/settings.md | 28 +-------- .../statements/describe-table.md | 58 ++++++++++++++++--- 2 files changed, 52 insertions(+), 34 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2cf9263a05d..ab4f3506630 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3817,7 +3817,7 @@ Default value: `0`. ## describe_include_subcolumns {#describe_include_subcolumns} -Enables describing subcolumns for a [DESCRIBE](../../sql-reference/statements/describe-table.md) query. Subcolumns appear if [Tuple](../../sql-reference/data-types/tuple.md) or [Map](../../sql-reference/data-types/map.md) data types are used. +Enables describing subcolumns for a [DESCRIBE](../../sql-reference/statements/describe-table.md) query. Subcolumns appear if [Tuple](../../sql-reference/data-types/tuple.md) or [Map](../../sql-reference/data-types/map.md#map-subcolumns) data types are used. Possible values: @@ -3826,29 +3826,3 @@ Possible values: Default value: `0`. -**Example** - -Query: - -``` sql -CREATE TABLE describe_example ( - id UInt64, user Tuple (name String, age UInt8), dict Map(String, UInt32) -) ENGINE = MergeTree() ORDER BY id; - -DESCRIBE TABLE describe_example SETTINGS describe_include_subcolumns=1; -``` - -Result: - -``` text -┌─name────────┬─type──────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┐ -│ id │ UInt64 │ │ │ │ │ │ 0 │ -│ user │ Tuple(name String, age UInt8) │ │ │ │ │ │ 0 │ -│ dict │ Map(String, UInt32) │ │ │ │ │ │ 0 │ -│ user.name │ String │ │ │ │ │ │ 1 │ -│ user.age │ UInt8 │ │ │ │ │ │ 1 │ -│ dict.size0 │ UInt64 │ │ │ │ │ │ 1 │ -│ dict.keys │ Array(String) │ │ │ │ │ │ 1 │ -│ dict.values │ Array(UInt32) │ │ │ │ │ │ 1 │ -└─────────────┴───────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┴──────────────┘ -``` diff --git a/docs/en/sql-reference/statements/describe-table.md b/docs/en/sql-reference/statements/describe-table.md index 3d71f8e875b..8e53cbc982b 100644 --- a/docs/en/sql-reference/statements/describe-table.md +++ b/docs/en/sql-reference/statements/describe-table.md @@ -5,19 +5,63 @@ toc_title: DESCRIBE # DESCRIBE TABLE Statement {#misc-describe-table} +Returns information about table columns. + +**Syntax** + ``` sql DESC|DESCRIBE TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] ``` -Returns the following `String` type columns: +The `DESCRIBE` statement returns a row for each table column with the following [String](../../sql-reference/data-types/string.md) values: -- `name` — Column name. -- `type`— Column type. -- `default_type` — Clause that is used in [default expression](../../sql-reference/statements/create/table.md#create-default-values) (`DEFAULT`, `MATERIALIZED` or `ALIAS`). Column contains an empty string, if the default expression isn’t specified. -- `default_expression` — Value specified in the `DEFAULT` clause. -- `comment_expression` — Comment text. +- `name` — a column name. +- `type` — a column type. +- `default_type` — a clause that is used in the column [default expression](../../sql-reference/statements/create/table.md#create-default-values): `DEFAULT`, `MATERIALIZED` or `ALIAS`. If there is no default expression, then empty string is returned. +- `default_expression` — an expression specified after the `DEFAULT` clause. +- `comment` — a comment. +- `codec_expression` - a [codec](../../sql-reference/statements/create/table.md#codecs) that is applied to the column. +- `ttl_expression` - a [TTL](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl) expression. +- `is_subcolumn` - a flag that is set to `1` for internal subcolumns. It is included into an output if subcolumn description is enabled. -Nested data structures are output in “expanded” format. Each column is shown separately, with the name after a dot. +[Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns are described separately. The name of each nested column is prefixed with a parent column name and a dot. +To enable internal subcolumn description, use the [describe_include_subcolumns](../../operations/settings/settings.md#describe_include_subcolumns) setting. + +**Example** + +Query: + +``` sql +CREATE TABLE describe_example ( + id UInt64, text String DEFAULT 'unknown' CODEC(ZSTD), + user Tuple (name String, age UInt8) +) ENGINE = MergeTree() ORDER BY id; + +DESCRIBE TABLE describe_example; +DESCRIBE TABLE describe_example SETTINGS describe_include_subcolumns=1; +``` + +Result: + +``` text +┌─name─┬─type──────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┐ +│ id │ UInt64 │ │ │ │ │ │ +│ text │ String │ DEFAULT │ 'unknown' │ │ ZSTD(1) │ │ +│ user │ Tuple(name String, age UInt8) │ │ │ │ │ │ +└──────┴───────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ +``` + +The second query additionally shows subcolumn information: + +``` text +┌─name──────┬─type──────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┐ +│ id │ UInt64 │ │ │ │ │ │ 0 │ +│ text │ String │ DEFAULT │ 'unknown' │ │ ZSTD(1) │ │ 0 │ +│ user │ Tuple(name String, age UInt8) │ │ │ │ │ │ 0 │ +│ user.name │ String │ │ │ │ │ │ 1 │ +│ user.age │ UInt8 │ │ │ │ │ │ 1 │ +└───────────┴───────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┴──────────────┘ +``` **See Also** From 3c9025d04eefb9ef410655203f0582bc0e3319f6 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 20 Oct 2021 00:17:13 +0300 Subject: [PATCH 246/919] Add max argument for mapPopulateSeries function --- docs/en/sql-reference/functions/tuple-map-functions.md | 7 +++---- docs/ru/sql-reference/functions/tuple-map-functions.md | 7 +++---- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index 5510fbef226..231035a7332 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -173,18 +173,19 @@ mapPopulateSeries(keys, values[, max]) mapPopulateSeries(map[, max]) ``` -Arguments are [maps](../../sql-reference/data-types/map.md) or two [arrays](../../sql-reference/data-types/array.md#data-type-array), where the first array represent keys, and the second array contains values for the each key. - Generates a map (a tuple with two arrays or a value of `Map` type, depending on the arguments), where keys are a series of numbers, from minimum to maximum keys (or `max` argument if it specified) taken from the map with a step size of one, and corresponding values. If the value is not specified for the key, then it uses the default value in the resulting map. For repeated keys, only the first value (in order of appearing) gets associated with the key. For array arguments the number of elements in `keys` and `values` must be the same for each row. **Arguments** +Arguments are [maps](../../sql-reference/data-types/map.md) or two [arrays](../../sql-reference/data-types/array.md#data-type-array), where the first array represent keys, and the second array contains values for the each key. + Mapped arrays: - `keys` — Array of keys. [Array](../../sql-reference/data-types/array.md#data-type-array)([Int](../../sql-reference/data-types/int-uint.md#uint-ranges)). - `values` — Array of values. [Array](../../sql-reference/data-types/array.md#data-type-array)([Int](../../sql-reference/data-types/int-uint.md#uint-ranges)). +- `max` — Maximum key value. Optional. [Int8, Int16, Int32, Int64, Int128, Int256](../../sql-reference/data-types/int-uint.md#int-ranges). or @@ -352,5 +353,3 @@ Result: │ ['twelve','6.0'] │ └──────────────────┘ ``` - -[Original article](https://clickhouse.com/docs/en/sql-reference/functions/tuple-map-functions/) diff --git a/docs/ru/sql-reference/functions/tuple-map-functions.md b/docs/ru/sql-reference/functions/tuple-map-functions.md index ccd21d42826..45a5018500f 100644 --- a/docs/ru/sql-reference/functions/tuple-map-functions.md +++ b/docs/ru/sql-reference/functions/tuple-map-functions.md @@ -170,8 +170,6 @@ SELECT mapSubtract(map(1,1), map(1,1)); Заполняет недостающие ключи в контейнере map (пара массивов ключей и значений), где ключи являются целыми числами. Кроме того, он поддерживает указание максимального ключа, который используется для расширения массива ключей. -Аргументами являются контейнер [Map](../../sql-reference/data-types/map.md) или два [массива](../../sql-reference/data-types/array.md#data-type-array), где первый массив представляет ключи, а второй массив содержит значения для каждого ключа. - **Синтаксис** ``` sql @@ -185,10 +183,13 @@ mapPopulateSeries(map[, max]) **Аргументы** +Аргументами являются контейнер [Map](../../sql-reference/data-types/map.md) или два [массива](../../sql-reference/data-types/array.md#data-type-array), где первый массив представляет ключи, а второй массив содержит значения для каждого ключа. + Сопоставленные массивы: - `keys` — массив ключей. [Array](../../sql-reference/data-types/array.md#data-type-array)([Int](../../sql-reference/data-types/int-uint.md#int-ranges)). - `values` — массив значений. [Array](../../sql-reference/data-types/array.md#data-type-array)([Int](../../sql-reference/data-types/int-uint.md#int-ranges)). +- `max` — максимальное значение ключа. Необязательный параметр. [Int8, Int16, Int32, Int64, Int128, Int256](../../sql-reference/data-types/int-uint.md#int-ranges). или @@ -356,5 +357,3 @@ SELECT mapValues(a) FROM test; │ ['twelve','6.0'] │ └──────────────────┘ ``` - -[Оригинальная статья](https://clickhouse.com/docs/ru/sql-reference/functions/tuple-map-functions/) From 170a4957ca28862fdf1915c9236b97872a5f0fe2 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 20 Oct 2021 01:31:39 +0300 Subject: [PATCH 247/919] Add normalizeUTF8NFC function --- .../functions/string-functions.md | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 05c291bb41d..6c5ed485a3d 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -810,6 +810,28 @@ Result: └─────┘ ``` +## normalizeUTF8NFC {#normalizeutf8nfc} + +Converts a string to [NFC normalized form](https://en.wikipedia.org/wiki/Unicode_equivalence#Normal_forms), assuming the string contains a set of bytes that make up a UTF-8 encoded text. + +**Syntax** + +``` sql +normalizeUTF8NFKD(words) +``` + +**Arguments** + +- `words` — Input string that contains UTF-8 encoded text. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +- NFC normalized string. + +Type: [String](../../sql-reference/data-types/string.md). + + + ## encodeXMLComponent {#encode-xml-component} Escapes characters to place string into XML text node or attribute. From fa6856ab5efed7f352dfc9ff9f03123facab5eea Mon Sep 17 00:00:00 2001 From: karnevil13 Date: Wed, 20 Oct 2021 01:38:34 +0300 Subject: [PATCH 248/919] edit&translate --- .../settings.md | 24 +++++++++++++++++++ .../sql-reference/functions/nlp-functions.md | 2 +- .../statements/create/dictionary.md | 8 +++---- 3 files changed, 29 insertions(+), 5 deletions(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index bab67b62752..cb53e245dd9 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -467,6 +467,30 @@ ClickHouse проверяет условия для `min_part_size` и `min_part 127.0.0.1 ``` +## listen_backlog {#server_configuration_parameters-listen_backlog} + +Бэклог (размер очереди соединений, ожидающих принятия) прослушивающего сокета. + +Значение по умолчанию: `4096` (как в linux [5.4+](https://git.kernel.org/pub/scm/linux/kernel/git/torvalds/linux.git/commit/?id=19f92a030ca6d772ab44b22ee6a01378a8cb32d4)). + +Обычно это значение незачем менять по следующим причинам: +- значение по умолчанию достаточно велико, +- для принятия соединения клиента у сервера есть отдельный поток. + +Так что даже если у вас `TcpExtListenOverflows` (из `nstat`) не на нуле +и это число растет, для сервера ClickHouse это не повод увеличивать значение +по умолчанию, поскольку: +- обычно если 4096 недостаточно, это говорит о внутренних проблемах ClickHouse с масштабированием, + так что лучше сообщить о проблеме, +- и это не значит, что сервер сможет принять еще больше подключений в дальнейшем +(а если и сможет, клиенты, вероятно, отсоединятся). + +Примеры: + +``` xml +4096 +``` + ## logger {#server_configuration_parameters-logger} Настройки логирования. diff --git a/docs/ru/sql-reference/functions/nlp-functions.md b/docs/ru/sql-reference/functions/nlp-functions.md index 58c4eb86e35..250403ab127 100644 --- a/docs/ru/sql-reference/functions/nlp-functions.md +++ b/docs/ru/sql-reference/functions/nlp-functions.md @@ -28,7 +28,7 @@ stem('language', word) Query: ``` sql -SELECT SELECT arrayMap(x -> stem('en', x), ['I', 'think', 'it', 'is', 'a', 'blessing', 'in', 'disguise']) as res; +SELECT arrayMap(x -> stem('en', x), ['I', 'think', 'it', 'is', 'a', 'blessing', 'in', 'disguise']) as res; ``` Result: diff --git a/docs/ru/sql-reference/statements/create/dictionary.md b/docs/ru/sql-reference/statements/create/dictionary.md index a41b2cb9ad5..25546549feb 100644 --- a/docs/ru/sql-reference/statements/create/dictionary.md +++ b/docs/ru/sql-reference/statements/create/dictionary.md @@ -8,10 +8,10 @@ toc_title: "Словарь" ``` sql CREATE DICTIONARY [IF NOT EXISTS] [db.]dictionary_name [ON CLUSTER cluster] ( - key1 type1 [DEFAULT|EXPRESSION expr1] [HIERARCHICAL|INJECTIVE|IS_OBJECT_ID], - key2 type2 [DEFAULT|EXPRESSION expr2] [HIERARCHICAL|INJECTIVE|IS_OBJECT_ID], - attr1 type2 [DEFAULT|EXPRESSION expr3], - attr2 type2 [DEFAULT|EXPRESSION expr4] + key1 type1 [DEFAULT|EXPRESSION expr1] [IS_OBJECT_ID], + key2 type2 [DEFAULT|EXPRESSION expr2], + attr1 type2 [DEFAULT|EXPRESSION expr3] [HIERARCHICAL|INJECTIVE], + attr2 type2 [DEFAULT|EXPRESSION expr4] [HIERARCHICAL|INJECTIVE] ) PRIMARY KEY key1, key2 SOURCE(SOURCE_NAME([param1 value1 ... paramN valueN])) From c1d24037a414dc25dc8380f81a467afbe582b2b7 Mon Sep 17 00:00:00 2001 From: gyuton Date: Wed, 20 Oct 2021 01:42:05 +0300 Subject: [PATCH 249/919] Initial draft --- .../functions/arithmetic-functions.md | 73 ++ .../functions/tuple-functions.md | 803 ++++++++++++++++++ docs/en/sql-reference/operators/index.md | 10 + 3 files changed, 886 insertions(+) diff --git a/docs/en/sql-reference/functions/arithmetic-functions.md b/docs/en/sql-reference/functions/arithmetic-functions.md index 3187f13b5b9..a8fc07d59e8 100644 --- a/docs/en/sql-reference/functions/arithmetic-functions.md +++ b/docs/en/sql-reference/functions/arithmetic-functions.md @@ -82,3 +82,76 @@ An exception is thrown when dividing by zero or when dividing a minimal negative Returns the least common multiple of the numbers. An exception is thrown when dividing by zero or when dividing a minimal negative number by minus one. +# max2 {#max2} + +Compares two values and returns the maximum. The returned value is converted to [Float64](../../sql-reference/data-types/float.md). + +**Syntax** + +```sql +max2(value1, value2) +``` + +**Arguments** + +- `value1` — First value. [Int/UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). +- `value2` — Second value. [Int/UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). + +**Returned value** + +- The maximum of 2 values. + +Type: [Float](../../sql-reference/data-types/float.md). + +**Example** + +Query: + +```sql +SELECT max2(-1, 2); +``` + +Result: + +```text +┌─max2(-1, 2)─┐ +│ 2 │ +└─────────────┘ +``` + +# min2 {#min2} + +Compares two values and returns the minimum. The returned value is converted to [Float64](../../sql-reference/data-types/float.md). + +**Syntax** + +```sql +max2(value1, value2) +``` + +**Arguments** + +- `value1` — First value. [Int/UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). +- `value2` — Second value. [Int/UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). + +**Returned value** + +- The minimum of 2 values. + +Type: [Float](../../sql-reference/data-types/float.md). + +**Example** + +Query: + +```sql +SELECT max2(-1, 2); +``` + +Result: + +```text +┌─max2(-1, 2)─┐ +│ 2 │ +└─────────────┘ +``` diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 39e59ae2ba9..4a7c0a7fc68 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -165,3 +165,806 @@ Result: │ 2 │ └─────────────────┘ ``` + +## tuplePlus {#tupleplus} + +Calculates the sum of corresponding values of two tuples of the same size. + +**Syntax** + +```sql +tuplePlus(tuple1, tuple2) +``` + +Alias: `vectorSum`. + +**Arguments** + +- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). +- `tuple2` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). + +**Returned value** + +- Tuple with the sum. + +Type: [Tuple](../../sql-reference/data-types/tuple.md). + +**Example** + +Query: + +```sql +SELECT tuplePlus((1, 2), (2, 3)); +``` + +Result: + +```text +┌─tuplePlus((1, 2), (2, 3))─┐ +│ (3,5) │ +└───────────────────────────┘ +``` + +## tupleMinus {#tupleminus} + +Calculates the subtraction of corresponding values of two tuples of the same size. + +**Syntax** + +```sql +tupleMinus(tuple1, tuple2) +``` + +Alias: `vectorDifference`. + +**Arguments** + +- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). +- `tuple2` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). + +**Returned value** + +- Tuple with the result of subtraction. + +Type: [Tuple](../../sql-reference/data-types/tuple.md). + +**Example** + +Query: + +```sql +SELECT tupleMinus((1, 2), (2, 3)); +``` + +Result: + +```text +┌─tupleMinus((1, 2), (2, 3))─┐ +│ (-1,-1) │ +└────────────────────────────┘ +``` + +## tupleMultiply {#tuplemultiply} + +Calculates the multiplication of corresponding values of two tuples of the same size. + +**Syntax** + +```sql +tupleMultiply(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** + +- Tuple with the multiplication. + +Type: [Tuple](../../sql-reference/data-types/tuple.md). + +**Example** + +Query: + +```sql +SELECT tupleMultiply((1, 2), (2, 3)); +``` + +Result: + +```text +┌─tupleMultiply((1, 2), (2, 3))─┐ +│ (2,6) │ +└───────────────────────────────┘ +``` + +## tupleDivide {#tupledivide} + +Calculates the division of corresponding values of two tuples of the same size. Note that division by zero will return `inf`. + +**Syntax** + +```sql +tupleDivide(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** + +- Tuple with the result of division. + +Type: [Tuple](../../sql-reference/data-types/tuple.md). + +**Example** + +Query: + +```sql +SELECT tupleDivide((1, 2), (2, 3)); +``` + +Result: + +```text +┌─tupleDivide((1, 2), (2, 3))─┐ +│ (0.5,0.6666666666666666) │ +└─────────────────────────────┘ +``` + +## tupleNegate {#tuplenegate} + +Calculates the negation of the tuple values. + +**Syntax** + +```sql +tupleNegate(tuple) +``` + +**Arguments** + +- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). + +**Returned value** + +- Tuple with the result of negation. + +Type: [Tuple](../../sql-reference/data-types/tuple.md). + +**Example** + +Query: + +```sql +SELECT tupleDivide((1, 2)); +``` + +Result: + +```text +┌─tupleNegate((1, 2))─┐ +│ (-1,-2) │ +└─────────────────────┘ +``` + +## tupleMultiplyByNumber {#tuplemultiplybynumber} + +Returns a tuple with all values multiplied by a number. + +**Syntax** + +```sql +tupleMultiplyByNumber(tuple, number) +``` + +**Arguments** + +- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). +- `number` — Multiplier. [Int/UInt](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). + +**Returned value** + +- Tuple with multiplied values. + +Type: [Tuple](../../sql-reference/data-types/tuple.md). + +**Example** + +Query: + +```sql +SELECT tupleMultiplyByNumber((1, 2), -2.1); +``` + +Result: + +```text +┌─tupleMultiplyByNumber((1, 2), -2.1)─┐ +│ (-2.1,-4.2) │ +└─────────────────────────────────────┘ +``` + +## tupleDivideByNumber {#tupledividebynumber} + +Returns a tuple with all values divided by a number. Note that division by zero will return `inf`. + +**Syntax** + +```sql +tupleDivideByNumber(tuple, number) +``` + +**Arguments** + +- `tuple` — [Tuple](../../sql-reference/data-types/tuple.md). +- `number` — Divider. [Int/UInt](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md). + +**Returned value** + +- Tuple with divided values. + +Type: [Tuple](../../sql-reference/data-types/tuple.md). + +**Example** + +Query: + +```sql +SELECT tupleDivideByNumber((1, 2), 0.5); +``` + +Result: + +```text +┌─tupleDivideByNumber((1, 2), 0.5)─┐ +│ (2,4) │ +└──────────────────────────────────┘ +``` + +## dotProduct {#dotproduct} + +Calculates the scalar product of two tuples of the same size. + +**Syntax** + +```sql +dotProduct(tuple1, tuple2) +``` + +Alias: `scalarProduct`. + +**Arguments** + +- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). +- `tuple2` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). + +**Returned value** + +- Scalar product. + +Type: [Int/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 dotProduct((1, 2), (2, 3)); +``` + +Result: + +```text +┌─dotProduct((1, 2), (2, 3))─┐ +│ 8 │ +└────────────────────────────┘ +``` + +## L1Norm {#l1norm} + +Calculates the sum of absolute values of a tuple. + +**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 {#l2norm} + +Calculates the square root of the sum of the squares of the tuple values. + +**Syntax** + +```sql +L1Norm(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 {#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** + +- 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 {#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: any number from [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 {#l1distance} + +Calculates the distance between two points (the values of the tuples are the coordinates) in 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 {#l2distance} + +Calculates the distance between two points (the values of the tuples are the coordinates) in 2-norm ([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 {#linfdistance} + +Calculates the distance between two points (the values of the tuples are the coordinates) in [infinity-norm distance](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 {#LpDistance} + +Calculates the distance between two points (the values of the tuples are the coordinates) in [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: any number from [1;inf). [UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). + +**Returned value** + +- Lp-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 {#l1normalize} + +Calculates the unit vector of a given vector (the values of the tuple are the coordinates) in 1-norm ([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 {#l2normalize} + +Calculates the unit vector of a given vector (the values of the tuple are the coordinates) in 2-norm ([Euclidean geometry](https://en.wikipedia.org/wiki/Euclidean_space). + +**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 {#linfnormalize} + +Calculates the unit vector of a given vector (the values of the tuple are the coordinates) in [infinity-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 {#lpnormalize} + +Calculates the unit vector of a given vector (the values of the tuple are the coordinates) in [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 + +## cosineDistance {#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` — [Tuple](../../sql-reference/data-types/tuple.md). +- `tuple1` — [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/operators/index.md b/docs/en/sql-reference/operators/index.md index dbda5f5dd69..aa9b7f3ba63 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -17,20 +17,30 @@ ClickHouse transforms operators to their corresponding functions at the query pa `-a` – The `negate (a)` function. +For tuple negation: [tupleNegate](../../sql-reference/functions/tuple-functions.md#tuplenegate). + ## Multiplication and Division Operators {#multiplication-and-division-operators} `a * b` – The `multiply (a, b)` function. +For multiplying tuple by number: [tupleMultiplyByNumber](../../sql-reference/functions/tuple-functions.md#tuplemultiplybynumber), for scalar profuct: [dotProduct](../../sql-reference/functions/tuple-functions.md#dotproduct). + `a / b` – The `divide(a, b)` function. +For dividing tuple by number: [tupleDivideByNumber](../../sql-reference/functions/tuple-functions.md#tupledividebynumber). + `a % b` – The `modulo(a, b)` function. ## Addition and Subtraction Operators {#addition-and-subtraction-operators} `a + b` – The `plus(a, b)` function. +For tuple addiction: [tuplePlus](../../sql-reference/functions/tuple-functions.md#tupleplus). + `a - b` – The `minus(a, b)` function. +For tuple subtraction: [tupleMinus](../../sql-reference/functions/tuple-functions.md#tupleminus). + ## Comparison Operators {#comparison-operators} `a = b` – The `equals(a, b)` function. From 73f65bbe75db733afb880421f222e95d75c8461f Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 20 Oct 2021 01:42:39 +0300 Subject: [PATCH 250/919] Rewrite Normalization Form C (NFC) --- docs/en/sql-reference/functions/string-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 6c5ed485a3d..4ac3a175aba 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -826,7 +826,7 @@ normalizeUTF8NFKD(words) **Returned value** -- NFC normalized string. +- String transformed to normalization form C (NFC). Type: [String](../../sql-reference/data-types/string.md). From 1d76fc9636e05019c661f206b0b3d8b271f48266 Mon Sep 17 00:00:00 2001 From: gyuton Date: Wed, 20 Oct 2021 01:59:27 +0300 Subject: [PATCH 251/919] Small fix --- .../functions/arithmetic-functions.md | 4 ++-- .../sql-reference/functions/tuple-functions.md | 16 ++++++++-------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/docs/en/sql-reference/functions/arithmetic-functions.md b/docs/en/sql-reference/functions/arithmetic-functions.md index a8fc07d59e8..61818a4c6ac 100644 --- a/docs/en/sql-reference/functions/arithmetic-functions.md +++ b/docs/en/sql-reference/functions/arithmetic-functions.md @@ -82,7 +82,7 @@ An exception is thrown when dividing by zero or when dividing a minimal negative Returns the least common multiple of the numbers. An exception is thrown when dividing by zero or when dividing a minimal negative number by minus one. -# max2 {#max2} +## max2 {#max2} Compares two values and returns the maximum. The returned value is converted to [Float64](../../sql-reference/data-types/float.md). @@ -119,7 +119,7 @@ Result: └─────────────┘ ``` -# min2 {#min2} +## min2 {#min2} Compares two values and returns the minimum. The returned value is converted to [Float64](../../sql-reference/data-types/float.md). diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 4a7c0a7fc68..c6d965ced54 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -485,7 +485,7 @@ Alias: `normL1`. **Returned value** -- L1 norm or [taxicab geometry](https://en.wikipedia.org/wiki/Taxicab_geometry) distance. +- 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). @@ -512,7 +512,7 @@ Calculates the square root of the sum of the squares of the tuple values. **Syntax** ```sql -L1Norm(tuple) +L2Norm(tuple) ``` Alias: `normL2`. @@ -523,7 +523,7 @@ Alias: `normL2`. **Returned value** -- L2 norm or [Euclidean distance](https://en.wikipedia.org/wiki/Euclidean_distance). +- L2-norm or [Euclidean distance](https://en.wikipedia.org/wiki/Euclidean_distance). Type: [Float](../../sql-reference/data-types/float.md). @@ -561,7 +561,7 @@ Alias: `normLinf`. **Returned value** -- The maximum absolute value. +- Linf-norm or the maximum absolute value. Type: [Float](../../sql-reference/data-types/float.md). @@ -757,7 +757,7 @@ Alias: `distanceLp`. **Returned value** -- Lp-norm distance. +- p-norm distance. Type: [Float](../../sql-reference/data-types/float.md). @@ -817,7 +817,7 @@ Result: ## L2Normalize {#l2normalize} -Calculates the unit vector of a given vector (the values of the tuple are the coordinates) in 2-norm ([Euclidean geometry](https://en.wikipedia.org/wiki/Euclidean_space). +Calculates the unit vector of a given vector (the values of the tuple are the coordinates) in 2-norm ([Euclidean geometry](https://en.wikipedia.org/wiki/Euclidean_space)). **Syntax** @@ -944,8 +944,8 @@ cosineDistance(tuple1, tuple2) **Arguments** -- `tuple1` — [Tuple](../../sql-reference/data-types/tuple.md). -- `tuple1` — [Tuple](../../sql-reference/data-types/tuple.md). +- `tuple1` — First tuple. [Tuple](../../sql-reference/data-types/tuple.md). +- `tuple2` — Second tuple. [Tuple](../../sql-reference/data-types/tuple.md). **Returned value** From 59ee426d48149f8d6476f21670754a3c45e717a7 Mon Sep 17 00:00:00 2001 From: gyuton Date: Wed, 20 Oct 2021 02:00:54 +0300 Subject: [PATCH 252/919] small improvement --- docs/en/sql-reference/functions/arithmetic-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/arithmetic-functions.md b/docs/en/sql-reference/functions/arithmetic-functions.md index 61818a4c6ac..8901deac818 100644 --- a/docs/en/sql-reference/functions/arithmetic-functions.md +++ b/docs/en/sql-reference/functions/arithmetic-functions.md @@ -99,7 +99,7 @@ max2(value1, value2) **Returned value** -- The maximum of 2 values. +- The maximum of two values. Type: [Float](../../sql-reference/data-types/float.md). @@ -136,7 +136,7 @@ max2(value1, value2) **Returned value** -- The minimum of 2 values. +- The minimum of two values. Type: [Float](../../sql-reference/data-types/float.md). From 5e24f337a049f2aaee9efc8dc03d354103ab0ca2 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 20 Oct 2021 09:29:41 +0300 Subject: [PATCH 253/919] Update StorageReplicatedMergeTree.cpp --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f554986402c..ba115d36a5d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5995,12 +5995,12 @@ void StorageReplicatedMergeTree::replacePartitionFrom( MutableDataPartsVector dst_parts; Strings block_id_paths; Strings part_checksums; + auto zookeeper = getZooKeeper(); std::vector ephemeral_locks; LOG_DEBUG(log, "Cloning {} parts", src_all_parts.size()); static const String TMP_PREFIX = "tmp_replace_from_"; - auto zookeeper = getZooKeeper(); String alter_partition_version_path = zookeeper_path + "/alter_partition_version"; Coordination::Stat alter_partition_version_stat; From 31cd71c849e741f6580382cd55826db534730a5f Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 20 Oct 2021 09:37:18 +0300 Subject: [PATCH 254/919] Clean up --- src/Common/ProfileEvents.cpp | 16 +++---- src/Compression/CompressedReadBufferBase.cpp | 1 - ...chronousReadIndirectBufferFromRemoteFS.cpp | 48 +++++-------------- ...ynchronousReadIndirectBufferFromRemoteFS.h | 2 - src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 17 +++---- src/Disks/IO/ReadBufferFromRemoteFSGather.h | 2 - .../IO/ReadIndirectBufferFromRemoteFS.cpp | 8 ---- src/Disks/IO/ThreadPoolRemoteFSReader.cpp | 2 +- src/IO/RestartAwareReadBuffer.h | 2 + 9 files changed, 29 insertions(+), 69 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 0d738972155..9b8274c2b5b 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -253,16 +253,12 @@ M(RemoteFSReadBytes, "Read bytes from remote filesystem.") \ \ M(RemoteFSSeeks, "Total number of seeks for async buffer") \ - M(RemoteFSPrefetches, "Total number of prefetches") \ - M(RemoteFSSeekCancelledPrefetches, "Number of cancelled prefecthes because of seek") \ - M(RemoteFSUnusedCancelledPrefetches, "Number of prefetches pending in buffer destructor") \ - M(RemoteFSPrefetchReads, "Total number of reads from prefecthed buffer") \ - M(RemoteFSAsyncBufferReads, "Number of nextImpl() calls for async buffer") \ - M(RemoteFSSimpleBufferReads, "Number of nextImpl() calls for non-async buffer") \ - M(RemoteFSNewReaders, "Number of created impl objects") \ - M(RemoteFSAsyncBuffers, "Total number of AsycnhronousReadIndirectBufferFromREmoteFS buffers") \ - M(RemoteFSSimpleBuffers, "Total number of ReadIndirectBufferFromREmoteFS buffers") \ - M(RemoteFSRedundantlyReadBytes, "") \ + M(RemoteFSPrefetches, "Number of prefetches made with asynchronous reading from remote filesystem") \ + M(RemoteFSCancelledPrefetches, "Number of cancelled prefecthes (because of seek)") \ + M(RemoteFSUnusedPrefetches, "Number of prefetches pending at buffer destruction") \ + M(RemoteFSPrefetchedReads, "Number of reads from prefecthed buffer") \ + M(RemoteFSUnprefetchedReads, "Number of reads from unprefetched buffer") \ + M(RemoteFSBuffers, "Number of buffers created for asynchronous reading from remote filesystem") \ \ M(SleepFunctionCalls, "Number of times a sleep function (sleep, sleepEachRow) has been called.") \ M(SleepFunctionMicroseconds, "Time spent sleeping due to a sleep function call.") \ diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index c9ee751b202..81e49e445a7 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -13,7 +13,6 @@ #include #include #include -#include namespace ProfileEvents diff --git a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp index a5661decaa9..8c41cd3df39 100644 --- a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -16,11 +16,11 @@ namespace ProfileEvents extern const Event AsynchronousReadWaitMicroseconds; extern const Event RemoteFSSeeks; extern const Event RemoteFSPrefetches; - extern const Event RemoteFSSeekCancelledPrefetches; - extern const Event RemoteFSUnusedCancelledPrefetches; - extern const Event RemoteFSPrefetchReads; - extern const Event RemoteFSAsyncBufferReads; - extern const Event RemoteFSAsyncBuffers; + extern const Event RemoteFSCancelledPrefetches; + extern const Event RemoteFSUnusedPrefetches; + extern const Event RemoteFSPrefetchedReads; + extern const Event RemoteFSUnprefetchedReads; + extern const Event RemoteFSBuffers; } namespace DB @@ -46,8 +46,7 @@ AsynchronousReadIndirectBufferFromRemoteFS::AsynchronousReadIndirectBufferFromRe , prefetch_buffer(buf_size_) , min_bytes_for_seek(min_bytes_for_seek_) { - ProfileEvents::increment(ProfileEvents::RemoteFSAsyncBuffers); - buffer_events += "Events for buffer: " + impl->getFileName() + " : "; + ProfileEvents::increment(ProfileEvents::RemoteFSBuffers); } @@ -86,33 +85,23 @@ void AsynchronousReadIndirectBufferFromRemoteFS::prefetch() if (absolute_position > last_offset) { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Read beyond last offset ({} > {}) {}", - absolute_position, last_offset, buffer_events); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Read beyond last offset ({} > {})", + absolute_position, last_offset); } /// Prefetch even in case hasPendingData() == true. prefetch_future = readInto(prefetch_buffer.data(), prefetch_buffer.size()); ProfileEvents::increment(ProfileEvents::RemoteFSPrefetches); - - buffer_events += fmt::format("-- PREFETCH from offset: {}, upper bound: {} --", - toString(absolute_position), toString(last_offset)); } void AsynchronousReadIndirectBufferFromRemoteFS::setReadUntilPosition(size_t position) { - buffer_events += "-- Set last offset " + toString(position) + "--"; if (prefetch_future.valid()) { - LOG_DEBUG(&Poco::Logger::get("kssenii"), buffer_events); /// TODO: Planning to put logical error here after more testing, // because seems like future is never supposed to be valid at this point. std::terminate(); - - // buffer_events += "-- Cancelling because of offset update --"; - // ProfileEvents::increment(ProfileEvents::RemoteFSSeekCancelledPrefetches); - // prefetch_future.wait(); - // prefetch_future = {}; } last_offset = position; @@ -122,7 +111,6 @@ void AsynchronousReadIndirectBufferFromRemoteFS::setReadUntilPosition(size_t pos bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() { - LOG_DEBUG(&Poco::Logger::get("kssenii"), buffer_events); /// Everything is already read. if (absolute_position == last_offset) return false; @@ -131,12 +119,11 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() throw Exception(ErrorCodes::LOGICAL_ERROR, "Read beyond last offset ({} > {})", absolute_position, last_offset); - ProfileEvents::increment(ProfileEvents::RemoteFSAsyncBufferReads); size_t size = 0; if (prefetch_future.valid()) { - ProfileEvents::increment(ProfileEvents::RemoteFSPrefetchReads); + ProfileEvents::increment(ProfileEvents::RemoteFSPrefetchedReads); CurrentMetrics::Increment metric_increment{CurrentMetrics::AsynchronousReadWait}; Stopwatch watch; @@ -151,16 +138,14 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() } } - buffer_events += fmt::format("-- Read from prefetch from offset: {}, upper bound: {}, actually read: {} --", - toString(absolute_position), toString(last_offset), toString(size)); watch.stop(); ProfileEvents::increment(ProfileEvents::AsynchronousReadWaitMicroseconds, watch.elapsedMicroseconds()); } else { + ProfileEvents::increment(ProfileEvents::RemoteFSUnprefetchedReads); size = readInto(memory.data(), memory.size()).get(); - buffer_events += fmt::format("-- Read without prefetch from offset: {}, upper bound: {}, actually read: {} --", - toString(absolute_position), toString(last_offset), toString(size)); + if (size) { set(memory.data(), memory.size()); @@ -170,8 +155,6 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() } prefetch_future = {}; - - LOG_DEBUG(&Poco::Logger::get("kssenii"), buffer_events); return size; } @@ -179,7 +162,6 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence) { ProfileEvents::increment(ProfileEvents::RemoteFSSeeks); - buffer_events += "-- Seek to " + toString(offset_) + " --"; if (whence == SEEK_CUR) { @@ -218,8 +200,7 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence if (prefetch_future.valid()) { - buffer_events += "-- cancelling prefetch because of seek --"; - ProfileEvents::increment(ProfileEvents::RemoteFSSeekCancelledPrefetches); + ProfileEvents::increment(ProfileEvents::RemoteFSCancelledPrefetches); prefetch_future.wait(); prefetch_future = {}; } @@ -238,7 +219,6 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence } else { - buffer_events += "-- Impl seek --"; impl->seek(absolute_position); /// SEEK_SET. } @@ -250,12 +230,10 @@ void AsynchronousReadIndirectBufferFromRemoteFS::finalize() { if (prefetch_future.valid()) { - buffer_events += "-- cancelling prefetch in finalize --"; - ProfileEvents::increment(ProfileEvents::RemoteFSUnusedCancelledPrefetches); + ProfileEvents::increment(ProfileEvents::RemoteFSUnusedPrefetches); prefetch_future.wait(); prefetch_future = {}; } - LOG_DEBUG(&Poco::Logger::get("kssenii"), buffer_events); } diff --git a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h index 6842d62f22a..e30a21b5dc3 100644 --- a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -68,8 +68,6 @@ private: Memory<> prefetch_buffer; - String buffer_events; - size_t min_bytes_for_seek; size_t bytes_to_ignore = 0; diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index c12af031089..70300563aa2 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -79,16 +79,15 @@ void ReadBufferFromRemoteFSGather::initialize() auto current_buf_offset = absolute_position; for (size_t i = 0; i < metadata.remote_fs_objects.size(); ++i) { - current_buf_idx = i; const auto & [file_path, size] = metadata.remote_fs_objects[i]; if (size > current_buf_offset) { /// Do not create a new buffer if we already have what we need. - if (!current_buf || buf_idx != i) + if (!current_buf || current_buf_idx != i) { current_buf = createImplementationBuffer(file_path, last_offset); - buf_idx = i; + current_buf_idx = i; } current_buf->seek(current_buf_offset, SEEK_SET); @@ -97,6 +96,7 @@ void ReadBufferFromRemoteFSGather::initialize() current_buf_offset -= size; } + current_buf_idx = metadata.remote_fs_objects.size(); current_buf = nullptr; } @@ -141,7 +141,7 @@ bool ReadBufferFromRemoteFSGather::readImpl() if (bytes_to_ignore) current_buf->ignore(bytes_to_ignore); - LOG_DEBUG(&Poco::Logger::get("Gather"), "Reading from path: {}", canonical_path); + LOG_DEBUG(&Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Reading from path: {}", canonical_path); auto result = current_buf->next(); swap(*current_buf); @@ -160,11 +160,11 @@ void ReadBufferFromRemoteFSGather::seek(off_t offset) } -void ReadBufferFromRemoteFSGather::setReadUntilPosition(size_t offset) +void ReadBufferFromRemoteFSGather::setReadUntilPosition(size_t position) { - assert(last_offset < offset); + assert(last_offset < position); current_buf.reset(); - last_offset = offset; + last_offset = position; } @@ -177,9 +177,6 @@ void ReadBufferFromRemoteFSGather::reset() String ReadBufferFromRemoteFSGather::getFileName() const { return canonical_path; - // if (current_buf) - // return fs::path(metadata.metadata_file_path) / metadata.remote_fs_objects[buf_idx].first; - // return metadata.metadata_file_path; } } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 72088012aeb..8012676d5a8 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -54,8 +54,6 @@ private: size_t absolute_position = 0; - size_t buf_idx = 0; - size_t bytes_to_ignore = 0; size_t last_offset = 0; diff --git a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp index f0133e0fafa..112124d9fd7 100644 --- a/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/ReadIndirectBufferFromRemoteFS.cpp @@ -3,12 +3,6 @@ #include -namespace ProfileEvents -{ - extern const Event RemoteFSSimpleBufferReads; - extern const Event RemoteFSSimpleBuffers; -} - namespace DB { @@ -21,7 +15,6 @@ namespace ErrorCodes ReadIndirectBufferFromRemoteFS::ReadIndirectBufferFromRemoteFS( std::shared_ptr impl_) : impl(std::move(impl_)) { - ProfileEvents::increment(ProfileEvents::RemoteFSSimpleBuffers); } @@ -79,7 +72,6 @@ off_t ReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence) bool ReadIndirectBufferFromRemoteFS::nextImpl() { - ProfileEvents::increment(ProfileEvents::RemoteFSSimpleBufferReads); /// Transfer current position and working_buffer to actual ReadBuffer swap(*impl); /// Position and working_buffer will be updated in next() call diff --git a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp index 976162ab625..945b2d3eb7e 100644 --- a/src/Disks/IO/ThreadPoolRemoteFSReader.cpp +++ b/src/Disks/IO/ThreadPoolRemoteFSReader.cpp @@ -44,7 +44,7 @@ std::future ThreadPoolRemoteFSReader::submit(Reques { auto task = std::make_shared>([request] { - setThreadName("ThreadPoolRead"); + setThreadName("ThreadPoolRemoteFSRead"); CurrentMetrics::Increment metric_increment{CurrentMetrics::Read}; auto * remote_fs_fd = assert_cast(request.descriptor.get()); diff --git a/src/IO/RestartAwareReadBuffer.h b/src/IO/RestartAwareReadBuffer.h index c8e871c8460..d22c82383d9 100644 --- a/src/IO/RestartAwareReadBuffer.h +++ b/src/IO/RestartAwareReadBuffer.h @@ -1,3 +1,5 @@ +#pragma once + #include #include From 6c990400d1c7864381c100ed261932dfd1bff65b Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 20 Oct 2021 10:45:19 +0300 Subject: [PATCH 255/919] Better positional args --- src/Interpreters/ExpressionAnalyzer.cpp | 132 +++++++++++------- .../02006_test_positional_arguments.reference | 107 +++++++++++++- .../02006_test_positional_arguments.sql | 28 +++- 3 files changed, 211 insertions(+), 56 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index a1f7a3c71e5..28bb936fbb7 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -96,6 +96,71 @@ bool allowEarlyConstantFolding(const ActionsDAG & actions, const Settings & sett return true; } +bool checkPositionalArguments(ASTPtr & argument, const ASTSelectQuery * select_query, ASTSelectQuery::Expression expression) +{ + auto columns = select_query->select()->children; + + /// In case of expression/function (order by 1+2 and 2*x1, max(1, 2)) replace + /// positions only if all literals are numbers, otherwise it is not positional. + bool positional = true; + + /// Case when GROUP BY element is position. + if (const auto * ast_literal = typeid_cast(argument.get())) + { + auto which = ast_literal->value.getType(); + if (which == Field::Types::UInt64) + { + auto pos = ast_literal->value.get(); + if (pos > 0 && pos <= columns.size()) + { + const auto & column = columns[--pos]; + if (typeid_cast(column.get())) + { + argument = column->clone(); + } + else if (const auto * function_ast = typeid_cast(column.get())) + { + auto is_aggregate_function = AggregateFunctionFactory::instance().isAggregateFunctionName(function_ast->name); + if (is_aggregate_function && expression != ASTSelectQuery::Expression::ORDER_BY) + { + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal value (aggregate function) for positional argument in {}", + ASTSelectQuery::expressionToString(expression)); + } + argument = column->clone(); + } + else + { + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal value for positional argument in {}", + ASTSelectQuery::expressionToString(expression)); + } + } + /// Do not throw if out of bounds, see appendUnusedGroupByColumn. + } + else + positional = false; + } + else if (auto * ast_function = typeid_cast(argument.get())) + { + if (ast_function->arguments) + { + for (auto & arg : ast_function->arguments->children) + positional &= checkPositionalArguments(arg, select_query, expression); + } + } + else + positional = false; + + return positional; +} + +void replaceForPositionalArguments(ASTPtr & argument, const ASTSelectQuery * select_query, ASTSelectQuery::Expression expression) +{ + auto argument_with_replacement = argument->clone(); + if (checkPositionalArguments(argument_with_replacement, select_query, expression)) + argument = argument_with_replacement; +} + } bool sanitizeBlock(Block & block, bool throw_if_cannot_create_column) @@ -164,37 +229,6 @@ ExpressionAnalyzer::ExpressionAnalyzer( analyzeAggregation(temp_actions); } -static ASTPtr checkPositionalArgument(ASTPtr argument, const ASTSelectQuery * select_query, ASTSelectQuery::Expression expression) -{ - auto columns = select_query->select()->children; - - /// Case when GROUP BY element is position. - /// Do not consider case when GROUP BY element is not a literal, but expression, even if all values are constants. - if (const auto * ast_literal = typeid_cast(argument.get())) - { - auto which = ast_literal->value.getType(); - if (which == Field::Types::UInt64) - { - auto pos = ast_literal->value.get(); - if (pos > 0 && pos <= columns.size()) - { - const auto & column = columns[--pos]; - if (const auto * literal_ast = typeid_cast(column.get())) - { - return std::make_shared(literal_ast->name()); - } - else - { - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal value for positional argument in {}", - ASTSelectQuery::expressionToString(expression)); - } - } - /// Do not throw if out of bounds, see appendUnusedGroupByColumn. - } - } - return nullptr; -} - NamesAndTypesList ExpressionAnalyzer::getColumnsAfterArrayJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns) { const auto * select_query = query->as(); @@ -282,16 +316,14 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) for (ssize_t i = 0; i < ssize_t(group_asts.size()); ++i) { ssize_t size = group_asts.size(); - getRootActionsNoMakeSet(group_asts[i], true, temp_actions, false); if (getContext()->getSettingsRef().enable_positional_arguments) - { - auto new_argument = checkPositionalArgument(group_asts[i], select_query, ASTSelectQuery::Expression::GROUP_BY); - if (new_argument) - group_asts[i] = new_argument; - } + replaceForPositionalArguments(group_asts[i], select_query, ASTSelectQuery::Expression::GROUP_BY); + + getRootActionsNoMakeSet(group_asts[i], true, temp_actions, false); const auto & column_name = group_asts[i]->getColumnName(); + const auto * node = temp_actions->tryFindInIndex(column_name); if (!node) throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER); @@ -1231,6 +1263,16 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns); + for (auto & child : select_query->orderBy()->children) + { + auto * ast = child->as(); + if (!ast || ast->children.empty()) + throw Exception("Bad ORDER BY expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE); + + if (getContext()->getSettingsRef().enable_positional_arguments) + replaceForPositionalArguments(ast->children.at(0), select_query, ASTSelectQuery::Expression::ORDER_BY); + } + getRootActions(select_query->orderBy(), only_types, step.actions()); bool with_fill = false; @@ -1239,16 +1281,6 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai for (auto & child : select_query->orderBy()->children) { auto * ast = child->as(); - if (!ast || ast->children.empty()) - throw Exception("Bad ORDER BY expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE); - - if (getContext()->getSettingsRef().enable_positional_arguments) - { - auto new_argument = checkPositionalArgument(ast->children.at(0), select_query, ASTSelectQuery::Expression::ORDER_BY); - if (new_argument) - ast->children[0] = new_argument; - } - ASTPtr order_expression = ast->children.at(0); step.addRequiredOutput(order_expression->getColumnName()); @@ -1302,11 +1334,7 @@ bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain for (auto & child : children) { if (getContext()->getSettingsRef().enable_positional_arguments) - { - auto new_argument = checkPositionalArgument(child, select_query, ASTSelectQuery::Expression::LIMIT_BY); - if (new_argument) - child = new_argument; - } + replaceForPositionalArguments(child, select_query, ASTSelectQuery::Expression::LIMIT_BY); auto child_name = child->getColumnName(); if (!aggregated_names.count(child_name)) diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.reference b/tests/queries/0_stateless/02006_test_positional_arguments.reference index a8e8ccec100..b41fe9af1ca 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.reference +++ b/tests/queries/0_stateless/02006_test_positional_arguments.reference @@ -46,5 +46,108 @@ select x1, x2, x3 from test order by 3 limit 1 by 1; 100 100 1 10 1 10 1 10 100 -select max(x3), max(x2), max(x1) from test group by 1; -- { serverError 43 } -select max(x1) from test order by 1; -- { serverError 43 } +select x3, x2, x1 from test order by x3 + x3; +1 100 100 +1 100 100 +10 1 10 +100 10 1 +200 10 1 +200 1 10 +select x3, x2, x1 from test order by 1 + 1; +1 100 100 +1 100 100 +10 1 10 +100 10 1 +200 10 1 +200 1 10 +select x3, x2, x1 from test order by (x3 + x3) * x1; +1 100 100 +100 10 1 +10 1 10 +1 100 100 +200 10 1 +200 1 10 +select x3, x2, x1 from test order by (1 + 1) * 3; +1 100 100 +100 10 1 +10 1 10 +1 100 100 +200 10 1 +200 1 10 +select x2, x1 from test group by x2 + x1; -- { serverError 215 } +select x2, x1 from test group by 1 + 2; -- { serverError 215 } +select x3, x2, x1 from test order by 1; +1 100 100 +1 100 100 +10 1 10 +100 10 1 +200 10 1 +200 1 10 +select x3 + 1, x2, x1 from test order by 1; +2 100 100 +2 100 100 +11 1 10 +101 10 1 +201 10 1 +201 1 10 +select x3, x3 - x2, x2, x1 from test order by 2; +1 -99 100 100 +1 -99 100 100 +10 9 1 10 +100 90 10 1 +200 190 10 1 +200 199 1 10 +select x3, if(0, x3, plus(x1, x2)), x1 + x2 from test order by 2; +200 11 11 +200 11 11 +100 11 11 +10 11 11 +1 200 200 +1 200 200 +select max(x1), x2 from test group by 2 order by 1, 2; +1 10 +10 1 +100 100 +select max(x1), x2 from test group by 1, 2; -- { serverError 43 } +select x1 + x2, x3 from test group by x1 + x2, x3; +11 100 +200 1 +11 200 +11 10 +select x3, x2, x1 from test order by x3 * 2, x2, x1; -- check x3 * 2 does not become x3 * x2 +1 100 100 +1 100 100 +10 1 10 +100 10 1 +200 1 10 +200 10 1 +select x3, x2, x1 from test order by 1 + 1, 2, 1; +1 100 100 +1 100 100 +10 1 10 +100 10 1 +200 1 10 +200 10 1 +explain syntax select x1, x3 from test group by 1 + 2, 1, 2; +SELECT + x1, + x3 +FROM test +GROUP BY + x1 + x3, + x1, + x3 +explain syntax select x1 + x3, x3 from test group by 1, 2; +SELECT + x1 + x3, + x3 +FROM test +GROUP BY + x1 + x3, + x3 +create table test2(x1 Int, x2 Int, x3 Int) engine=Memory; +insert into test2 values (1, 10, 100), (10, 1, 10), (100, 100, 1); +select x1, x1 * 2, max(x2), max(x3) from test2 group by 2, 1, x1 order by 1, 2, 4 desc, 3 asc; +1 2 10 100 +10 20 1 10 +100 200 100 1 diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.sql b/tests/queries/0_stateless/02006_test_positional_arguments.sql index dc45b288016..ab88e0cba8b 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.sql +++ b/tests/queries/0_stateless/02006_test_positional_arguments.sql @@ -1,6 +1,8 @@ set enable_positional_arguments = 1; drop table if exists test; +drop table if exists test2; + create table test(x1 Int, x2 Int, x3 Int) engine=Memory(); insert into test values (1, 10, 100), (10, 1, 10), (100, 100, 1); @@ -20,7 +22,29 @@ select x1, x2, x3 from test order by 3 limit 1 by 3; select x1, x2, x3 from test order by x3 limit 1 by x1; select x1, x2, x3 from test order by 3 limit 1 by 1; -select max(x3), max(x2), max(x1) from test group by 1; -- { serverError 43 } -select max(x1) from test order by 1; -- { serverError 43 } +select x3, x2, x1 from test order by x3 + x3; +select x3, x2, x1 from test order by 1 + 1; +select x3, x2, x1 from test order by (x3 + x3) * x1; +select x3, x2, x1 from test order by (1 + 1) * 3; +select x2, x1 from test group by x2 + x1; -- { serverError 215 } +select x2, x1 from test group by 1 + 2; -- { serverError 215 } + +select x3, x2, x1 from test order by 1; +select x3 + 1, x2, x1 from test order by 1; +select x3, x3 - x2, x2, x1 from test order by 2; +select x3, if(0, x3, plus(x1, x2)), x1 + x2 from test order by 2; +select max(x1), x2 from test group by 2 order by 1, 2; +select max(x1), x2 from test group by 1, 2; -- { serverError 43 } +select x1 + x2, x3 from test group by x1 + x2, x3; + +select x3, x2, x1 from test order by x3 * 2, x2, x1; -- check x3 * 2 does not become x3 * x2 +select x3, x2, x1 from test order by 1 + 1, 2, 1; + +explain syntax select x1, x3 from test group by 1 + 2, 1, 2; +explain syntax select x1 + x3, x3 from test group by 1, 2; + +create table test2(x1 Int, x2 Int, x3 Int) engine=Memory; +insert into test2 values (1, 10, 100), (10, 1, 10), (100, 100, 1); +select x1, x1 * 2, max(x2), max(x3) from test2 group by 2, 1, x1 order by 1, 2, 4 desc, 3 asc; From b73a77ad48590f0e46581374c530ed9c4d22284c Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 20 Oct 2021 10:49:05 +0300 Subject: [PATCH 256/919] Better test --- .../02006_test_positional_arguments.reference | 119 ++++++++---------- .../02006_test_positional_arguments.sql | 19 ++- 2 files changed, 60 insertions(+), 78 deletions(-) diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.reference b/tests/queries/0_stateless/02006_test_positional_arguments.reference index b41fe9af1ca..ff985d765f6 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.reference +++ b/tests/queries/0_stateless/02006_test_positional_arguments.reference @@ -46,68 +46,60 @@ select x1, x2, x3 from test order by 3 limit 1 by 1; 100 100 1 10 1 10 1 10 100 -select x3, x2, x1 from test order by x3 + x3; -1 100 100 -1 100 100 -10 1 10 -100 10 1 -200 10 1 -200 1 10 -select x3, x2, x1 from test order by 1 + 1; -1 100 100 -1 100 100 -10 1 10 -100 10 1 -200 10 1 -200 1 10 -select x3, x2, x1 from test order by (x3 + x3) * x1; -1 100 100 -100 10 1 -10 1 10 -1 100 100 -200 10 1 -200 1 10 -select x3, x2, x1 from test order by (1 + 1) * 3; -1 100 100 -100 10 1 -10 1 10 -1 100 100 -200 10 1 -200 1 10 +explain syntax select x3, x2, x1 from test order by 1 + 1; +SELECT + x3, + x2, + x1 +FROM test +ORDER BY x3 + x3 ASC +explain syntax select x3, x2, x1 from test order by (1 + 1) * 3; +SELECT + x3, + x2, + x1 +FROM test +ORDER BY (x3 + x3) * x1 ASC select x2, x1 from test group by x2 + x1; -- { serverError 215 } select x2, x1 from test group by 1 + 2; -- { serverError 215 } -select x3, x2, x1 from test order by 1; -1 100 100 -1 100 100 -10 1 10 -100 10 1 -200 10 1 -200 1 10 -select x3 + 1, x2, x1 from test order by 1; -2 100 100 -2 100 100 -11 1 10 -101 10 1 -201 10 1 -201 1 10 -select x3, x3 - x2, x2, x1 from test order by 2; -1 -99 100 100 -1 -99 100 100 -10 9 1 10 -100 90 10 1 -200 190 10 1 -200 199 1 10 -select x3, if(0, x3, plus(x1, x2)), x1 + x2 from test order by 2; -200 11 11 -200 11 11 -100 11 11 -10 11 11 -1 200 200 -1 200 200 -select max(x1), x2 from test group by 2 order by 1, 2; -1 10 -10 1 -100 100 +explain syntax select x3, x2, x1 from test order by 1; +SELECT + x3, + x2, + x1 +FROM test +ORDER BY x3 ASC +explain syntax select x3 + 1, x2, x1 from test order by 1; +SELECT + x3 + 1, + x2, + x1 +FROM test +ORDER BY x3 + 1 ASC +explain syntax select x3, x3 - x2, x2, x1 from test order by 2; +SELECT + x3, + x3 - x2, + x2, + x1 +FROM test +ORDER BY x3 - x2 ASC +explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order by 2; +SELECT + x3, + if(x3 > 10, x3, x1 + x2), + x1 + x2 +FROM test +ORDER BY if(x3 > 10, x3, x1 + x2) ASC +explain syntax select max(x1), x2 from test group by 2 order by 1, 2; +SELECT + max(x1), + x2 +FROM test +GROUP BY x2 +ORDER BY + max(x1) ASC, + x2 ASC select max(x1), x2 from test group by 1, 2; -- { serverError 43 } select x1 + x2, x3 from test group by x1 + x2, x3; 11 100 @@ -121,13 +113,6 @@ select x3, x2, x1 from test order by x3 * 2, x2, x1; -- check x3 * 2 does not be 100 10 1 200 1 10 200 10 1 -select x3, x2, x1 from test order by 1 + 1, 2, 1; -1 100 100 -1 100 100 -10 1 10 -100 10 1 -200 1 10 -200 10 1 explain syntax select x1, x3 from test group by 1 + 2, 1, 2; SELECT x1, diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.sql b/tests/queries/0_stateless/02006_test_positional_arguments.sql index ab88e0cba8b..f1c3676405e 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.sql +++ b/tests/queries/0_stateless/02006_test_positional_arguments.sql @@ -22,25 +22,22 @@ select x1, x2, x3 from test order by 3 limit 1 by 3; select x1, x2, x3 from test order by x3 limit 1 by x1; select x1, x2, x3 from test order by 3 limit 1 by 1; -select x3, x2, x1 from test order by x3 + x3; -select x3, x2, x1 from test order by 1 + 1; - -select x3, x2, x1 from test order by (x3 + x3) * x1; -select x3, x2, x1 from test order by (1 + 1) * 3; +explain syntax select x3, x2, x1 from test order by 1 + 1; +explain syntax select x3, x2, x1 from test order by (1 + 1) * 3; select x2, x1 from test group by x2 + x1; -- { serverError 215 } select x2, x1 from test group by 1 + 2; -- { serverError 215 } -select x3, x2, x1 from test order by 1; -select x3 + 1, x2, x1 from test order by 1; -select x3, x3 - x2, x2, x1 from test order by 2; -select x3, if(0, x3, plus(x1, x2)), x1 + x2 from test order by 2; -select max(x1), x2 from test group by 2 order by 1, 2; +explain syntax select x3, x2, x1 from test order by 1; +explain syntax select x3 + 1, x2, x1 from test order by 1; +explain syntax select x3, x3 - x2, x2, x1 from test order by 2; +explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order by 2; +explain syntax select max(x1), x2 from test group by 2 order by 1, 2; + select max(x1), x2 from test group by 1, 2; -- { serverError 43 } select x1 + x2, x3 from test group by x1 + x2, x3; select x3, x2, x1 from test order by x3 * 2, x2, x1; -- check x3 * 2 does not become x3 * x2 -select x3, x2, x1 from test order by 1 + 1, 2, 1; explain syntax select x1, x3 from test group by 1 + 2, 1, 2; explain syntax select x1 + x3, x3 from test group by 1, 2; From 26d40a66224632092013a4a16c9a7f249d14f395 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 20 Oct 2021 10:56:29 +0300 Subject: [PATCH 257/919] Update tests configs --- tests/config/config.d/keeper_port.xml | 5 +++++ tests/integration/test_keeper_auth/configs/keeper_config.xml | 4 ++++ .../test_keeper_back_to_back/configs/enable_keeper.xml | 4 ++++ .../test_keeper_snapshots/configs/enable_keeper.xml | 5 +++++ 4 files changed, 18 insertions(+) diff --git a/tests/config/config.d/keeper_port.xml b/tests/config/config.d/keeper_port.xml index 9523c25df21..fc6f8240f1e 100644 --- a/tests/config/config.d/keeper_port.xml +++ b/tests/config/config.d/keeper_port.xml @@ -11,6 +11,11 @@ 1000000000000000 10000 + + + 0 + 0 + 0 diff --git a/tests/integration/test_keeper_auth/configs/keeper_config.xml b/tests/integration/test_keeper_auth/configs/keeper_config.xml index b33722dd1d4..1a01b79cebd 100644 --- a/tests/integration/test_keeper_auth/configs/keeper_config.xml +++ b/tests/integration/test_keeper_auth/configs/keeper_config.xml @@ -11,6 +11,10 @@ 10000 trace 75 + + 0 + 0 + 0 diff --git a/tests/integration/test_keeper_back_to_back/configs/enable_keeper.xml b/tests/integration/test_keeper_back_to_back/configs/enable_keeper.xml index a4a560f855c..30a56c6e1e1 100644 --- a/tests/integration/test_keeper_back_to_back/configs/enable_keeper.xml +++ b/tests/integration/test_keeper_back_to_back/configs/enable_keeper.xml @@ -10,6 +10,10 @@ 10000 trace false + + 0 + 0 + 0 diff --git a/tests/integration/test_keeper_snapshots/configs/enable_keeper.xml b/tests/integration/test_keeper_snapshots/configs/enable_keeper.xml index 2be4246f7dd..15db48f4f37 100644 --- a/tests/integration/test_keeper_snapshots/configs/enable_keeper.xml +++ b/tests/integration/test_keeper_snapshots/configs/enable_keeper.xml @@ -11,6 +11,11 @@ 5000 10000 trace + + + 0 + 0 + 0 From 5f24eb10ecdfc8138e632a5221a559d3be61a7ce Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 20 Oct 2021 12:00:24 +0300 Subject: [PATCH 258/919] Test changes in MergeTreeReader for local fs --- ...ynchronousReadBufferFromFileDescriptor.cpp | 44 +++++++++++++++++-- ...AsynchronousReadBufferFromFileDescriptor.h | 4 +- tests/config/install.sh | 1 + tests/config/users.d/read_method.xml | 7 +++ 4 files changed, 52 insertions(+), 4 deletions(-) create mode 100644 tests/config/users.d/read_method.xml diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp index b2be45471c8..0d1e32bb2b5 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp @@ -26,6 +26,7 @@ namespace DB namespace ErrorCodes { extern const int ARGUMENT_OUT_OF_BOUND; + extern const int LOGICAL_ERROR; } @@ -53,14 +54,52 @@ void AsynchronousReadBufferFromFileDescriptor::prefetch() if (prefetch_future.valid()) return; - /// Will request the same amount of data that is read in nextImpl. - prefetch_buffer.resize(internal_buffer.size()); + size_t read_size; + if (read_until_position) + { + /// Everything is already read. + if (file_offset_of_buffer_end == *read_until_position) + return; + + if (file_offset_of_buffer_end > *read_until_position) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Read beyond last offset ({} > {})", + file_offset_of_buffer_end, *read_until_position); + + /// Read range [file_offset_of_buffer_end, read_until_position). + read_size = *read_until_position - file_offset_of_buffer_end - 1; + } + else + { + read_size = internal_buffer.size(); + } + + prefetch_buffer.resize(read_size); prefetch_future = readInto(prefetch_buffer.data(), prefetch_buffer.size()); } +void AsynchronousReadBufferFromFileDescriptor::setReadUntilPosition(size_t position) +{ + if (prefetch_future.valid()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Prefetch is valid in readUntilPosition"); + + read_until_position = position; +} + + bool AsynchronousReadBufferFromFileDescriptor::nextImpl() { + if (read_until_position) + { + /// Everything is already read. + if (file_offset_of_buffer_end == *read_until_position) + return false; + + if (file_offset_of_buffer_end > *read_until_position) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Read beyond last offset ({} > {})", + file_offset_of_buffer_end, *read_until_position); + } + if (prefetch_future.valid()) { /// Read request already in flight. Wait for its completion. @@ -201,4 +240,3 @@ void AsynchronousReadBufferFromFileDescriptor::rewind() } } - diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.h b/src/IO/AsynchronousReadBufferFromFileDescriptor.h index c64341089d0..271518dd1eb 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.h +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.h @@ -24,6 +24,7 @@ protected: const size_t required_alignment = 0; /// For O_DIRECT both file offsets and memory addresses have to be aligned. size_t file_offset_of_buffer_end = 0; /// What offset in file corresponds to working_buffer.end(). + std::optional read_until_position; int fd; bool nextImpl() override; @@ -33,6 +34,8 @@ protected: void finalize(); + void setReadUntilPosition(size_t position) override; + public: AsynchronousReadBufferFromFileDescriptor( AsynchronousReaderPtr reader_, Int32 priority_, @@ -67,4 +70,3 @@ private: }; } - diff --git a/tests/config/install.sh b/tests/config/install.sh index ba6ba0cd07c..fd43a611f79 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -49,6 +49,7 @@ ln -sf $SRC_PATH/users.d/remote_queries.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/session_log_test.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/memory_profiler.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/no_fsync_metadata.xml $DEST_SERVER_PATH/users.d/ +ln -sf $SRC_PATH/users.d/read_method.xml $DEST_SERVER_PATH/users.d/ # FIXME DataPartsExchange may hang for http_send_timeout seconds # when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"), diff --git a/tests/config/users.d/read_method.xml b/tests/config/users.d/read_method.xml new file mode 100644 index 00000000000..4247561e921 --- /dev/null +++ b/tests/config/users.d/read_method.xml @@ -0,0 +1,7 @@ + + + + pread_threadpool + + + From f607a5d0ab12d6aa8d38c602493996392b906187 Mon Sep 17 00:00:00 2001 From: vesslanjin Date: Wed, 20 Oct 2021 07:40:06 -0400 Subject: [PATCH 259/919] remove branch in ColumnDecimal, same as pull req #29881 Co-authored-by: Zhu Jasper jasper.zhu@intel.com Signed-off-by: vesslanjin --- src/Columns/ColumnDecimal.cpp | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index d8c5ced4b6b..dc236fafbd9 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -237,6 +237,26 @@ ColumnPtr ColumnDecimal::filter(const IColumn::Filter & filt, ssize_t result_ const UInt8 * filt_end = filt_pos + size; const T * data_pos = data.data(); +#ifdef __SSE2__ + static constexpr size_t SIMD_BYTES = 16; + const __m128i zero16 = _mm_setzero_si128(); + const UInt8 * filt_end_sse = filt_pos + size / SIMD_BYTES * SIMD_BYTES; + + while (filt_pos < filt_end_sse) + { + UInt16 mask = _mm_movemask_epi8(_mm_cmpeq_epi8(_mm_loadu_si128(reinterpret_cast(filt_pos)), zero16)); + mask = ~mask; + while (mask) + { + size_t index = __builtin_ctz(mask); + res_data.push_back(*(data_pos + index)); + mask = mask & (mask - 1); + } + filt_pos += SIMD_BYTES; + data_pos += SIMD_BYTES; + } +#endif + while (filt_pos < filt_end) { if (*filt_pos) From 70072860883b4abba36c9f68b7217f394a4933c1 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 20 Oct 2021 14:48:54 +0300 Subject: [PATCH 260/919] Fix WithNamesAndTypes parallel parsing, add new tests, small refactoring --- src/Common/ErrorCodes.cpp | 1 + src/Formats/FormatFactory.h | 3 +- src/Formats/JSONEachRowUtils.cpp | 10 +-- src/Formats/JSONEachRowUtils.h | 2 +- src/Formats/registerWithNamesAndTypes.cpp | 39 +++++++++ src/Formats/registerWithNamesAndTypes.h | 23 ++++++ src/Processors/Formats/IRowOutputFormat.cpp | 7 -- src/Processors/Formats/IRowOutputFormat.h | 3 - .../Formats/Impl/BinaryRowInputFormat.cpp | 35 ++++---- .../Formats/Impl/BinaryRowInputFormat.h | 2 +- .../Formats/Impl/BinaryRowOutputFormat.cpp | 27 +++---- .../Formats/Impl/CSVRowInputFormat.cpp | 26 +++--- .../Formats/Impl/CSVRowOutputFormat.cpp | 10 +-- .../Impl/JSONCompactEachRowRowInputFormat.cpp | 21 +++-- .../JSONCompactEachRowRowOutputFormat.cpp | 10 +-- .../Impl/TabSeparatedRowInputFormat.cpp | 43 ++++++---- .../Impl/TabSeparatedRowOutputFormat.cpp | 12 +-- .../RowInputFormatWithNamesAndTypes.cpp | 6 -- .../Formats/RowInputFormatWithNamesAndTypes.h | 7 -- ..._row_binary_with_names_and_types.reference | 6 ++ .../02102_row_binary_with_names_and_types.sh | 31 +++++-- ...names_and_types_parallel_parsing.reference | 80 +++++++++++++++++++ ...3_with_names_and_types_parallel_parsing.sh | 20 +++++ 23 files changed, 306 insertions(+), 118 deletions(-) create mode 100644 src/Formats/registerWithNamesAndTypes.cpp create mode 100644 src/Formats/registerWithNamesAndTypes.h create mode 100644 tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.reference create mode 100755 tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 1aff1460125..0947ca38523 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -591,6 +591,7 @@ M(621, CANNOT_NORMALIZE_STRING) \ M(622, CANNOT_PARSE_CAPN_PROTO_SCHEMA) \ M(623, CAPN_PROTO_BAD_CAST) \ + M(624, CANNOT_SKIP_UNKNOWN_FIELD) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index d5784219c6a..dbf4cf6069c 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -67,8 +67,6 @@ public: const Columns & columns, size_t row)>; -private: - using InputCreatorFunc = InputFormatPtr( ReadBuffer & buf, const Block & header, @@ -83,6 +81,7 @@ private: const RowOutputFormatParams & params, const FormatSettings & settings)>; +private: /// Some input formats can have non trivial readPrefix() and readSuffix(), /// so in some cases there is no possibility to use parallel parsing. /// The checker should return true if parallel parsing should be disabled. diff --git a/src/Formats/JSONEachRowUtils.cpp b/src/Formats/JSONEachRowUtils.cpp index 2675b12ec44..b55e9f59cc7 100644 --- a/src/Formats/JSONEachRowUtils.cpp +++ b/src/Formats/JSONEachRowUtils.cpp @@ -14,7 +14,7 @@ namespace ErrorCodes } template -static std::pair fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) +static std::pair fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size, size_t min_rows) { skipWhitespaceIfAny(in); @@ -23,7 +23,7 @@ static std::pair fileSegmentationEngineJSONEachRowImpl(ReadBuffer bool quotes = false; size_t number_of_rows = 0; - while (loadAtPosition(in, memory, pos) && (balance || memory.size() + static_cast(pos - in.position()) < min_chunk_size)) + while (loadAtPosition(in, memory, pos) && (balance || memory.size() + static_cast(pos - in.position()) < min_chunk_size || number_of_rows < min_rows)) { const auto current_object_size = memory.size() + static_cast(pos - in.position()); if (current_object_size > 10 * min_chunk_size) @@ -94,12 +94,12 @@ static std::pair fileSegmentationEngineJSONEachRowImpl(ReadBuffer std::pair fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) { - return fileSegmentationEngineJSONEachRowImpl<'{', '}'>(in, memory, min_chunk_size); + return fileSegmentationEngineJSONEachRowImpl<'{', '}'>(in, memory, min_chunk_size, 1); } -std::pair fileSegmentationEngineJSONCompactEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) +std::pair fileSegmentationEngineJSONCompactEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size, size_t min_rows) { - return fileSegmentationEngineJSONEachRowImpl<'[', ']'>(in, memory, min_chunk_size); + return fileSegmentationEngineJSONEachRowImpl<'[', ']'>(in, memory, min_chunk_size, min_rows); } bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf) diff --git a/src/Formats/JSONEachRowUtils.h b/src/Formats/JSONEachRowUtils.h index cb148af400c..4a049aa1abd 100644 --- a/src/Formats/JSONEachRowUtils.h +++ b/src/Formats/JSONEachRowUtils.h @@ -9,7 +9,7 @@ namespace DB { std::pair fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size); -std::pair fileSegmentationEngineJSONCompactEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size); +std::pair fileSegmentationEngineJSONCompactEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size, size_t min_rows); bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf); diff --git a/src/Formats/registerWithNamesAndTypes.cpp b/src/Formats/registerWithNamesAndTypes.cpp new file mode 100644 index 00000000000..ad76cdac0c9 --- /dev/null +++ b/src/Formats/registerWithNamesAndTypes.cpp @@ -0,0 +1,39 @@ +#include + +namespace DB +{ + +void registerInputFormatWithNamesAndTypes(FormatFactory & factory, const String & base_format_name, GetInputCreatorWithNamesAndTypesFunc get_input_creator) +{ + factory.registerInputFormat(base_format_name, get_input_creator(false, false)); + factory.registerInputFormat(base_format_name + "WithNames", get_input_creator(true, false)); + factory.registerInputFormat(base_format_name + "WithNamesAndTypes", get_input_creator(true, true)); +} + +void registerOutputFormatWithNamesAndTypes( + FormatFactory & factory, + const String & base_format_name, + GetOutputCreatorWithNamesAndTypesFunc get_output_creator, + bool supports_parallel_formatting) +{ + factory.registerOutputFormat(base_format_name, get_output_creator(false, false)); + factory.registerOutputFormat(base_format_name + "WithNames", get_output_creator(true, false)); + factory.registerOutputFormat(base_format_name + "WithNamesAndTypes", get_output_creator(true, true)); + + if (supports_parallel_formatting) + { + factory.markOutputFormatSupportsParallelFormatting(base_format_name); + factory.markOutputFormatSupportsParallelFormatting(base_format_name + "WithNames"); + factory.markOutputFormatSupportsParallelFormatting(base_format_name + "WithNamesAndTypes"); + } +} + +void registerFileSegmentationEngineForFormatWithNamesAndTypes( + FormatFactory & factory, const String & base_format_name, GetFileSegmentationEngineWithNamesAndTypesFunc get_file_segmentation_engine) +{ + factory.registerFileSegmentationEngine(base_format_name, get_file_segmentation_engine(1)); + factory.registerFileSegmentationEngine(base_format_name + "WithNames", get_file_segmentation_engine(2)); + factory.registerFileSegmentationEngine(base_format_name + "WithNamesAndTypes", get_file_segmentation_engine(3)); +} + +} diff --git a/src/Formats/registerWithNamesAndTypes.h b/src/Formats/registerWithNamesAndTypes.h new file mode 100644 index 00000000000..fdfe1793b3f --- /dev/null +++ b/src/Formats/registerWithNamesAndTypes.h @@ -0,0 +1,23 @@ +#pragma once + +#include + +namespace DB +{ + +using GetInputCreatorWithNamesAndTypesFunc = std::function; +void registerInputFormatWithNamesAndTypes( + FormatFactory & factory, const String & base_format_name, GetInputCreatorWithNamesAndTypesFunc get_input_creator); + +using GetOutputCreatorWithNamesAndTypesFunc = std::function; +void registerOutputFormatWithNamesAndTypes( + FormatFactory & factory, + const String & base_format_name, + GetOutputCreatorWithNamesAndTypesFunc get_output_creator, + bool supports_parallel_formatting = false); + +using GetFileSegmentationEngineWithNamesAndTypesFunc = std::function; +void registerFileSegmentationEngineForFormatWithNamesAndTypes( + FormatFactory & factory, const String & base_format_name, GetFileSegmentationEngineWithNamesAndTypesFunc get_file_segmentation_engine); + +} diff --git a/src/Processors/Formats/IRowOutputFormat.cpp b/src/Processors/Formats/IRowOutputFormat.cpp index ad111bdc66a..6b7a9a46eaa 100644 --- a/src/Processors/Formats/IRowOutputFormat.cpp +++ b/src/Processors/Formats/IRowOutputFormat.cpp @@ -113,11 +113,4 @@ void IRowOutputFormat::writeTotals(const DB::Columns & columns, size_t row_num) write(columns, row_num); } -void registerOutputFormatWithNamesAndTypes(const String & base_format_name, RegisterOutputFormatWithNamesAndTypes register_func) -{ - register_func(base_format_name, false, false); - register_func(base_format_name + "WithNames", true, false); - register_func(base_format_name + "WithNamesAndTypes", true, true); -} - } diff --git a/src/Processors/Formats/IRowOutputFormat.h b/src/Processors/Formats/IRowOutputFormat.h index 50c70a527bf..c35d93b6133 100644 --- a/src/Processors/Formats/IRowOutputFormat.h +++ b/src/Processors/Formats/IRowOutputFormat.h @@ -87,7 +87,4 @@ private: }; -using RegisterOutputFormatWithNamesAndTypes = std::function; -void registerOutputFormatWithNamesAndTypes(const String & base_format_name, RegisterOutputFormatWithNamesAndTypes register_func); - } diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp index d05a2187ac8..76ba5dca019 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include @@ -10,11 +11,11 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; + extern const int CANNOT_SKIP_UNKNOWN_FIELD; } -BinaryRowInputFormat::BinaryRowInputFormat(ReadBuffer & in_, Block header, Params params_, bool with_names_and_types, const FormatSettings & format_settings_) - : RowInputFormatWithNamesAndTypes(std::move(header), in_, std::move(params_), with_names_and_types, with_names_and_types, format_settings_) +BinaryRowInputFormat::BinaryRowInputFormat(ReadBuffer & in_, Block header, Params params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_) + : RowInputFormatWithNamesAndTypes(std::move(header), in_, std::move(params_), with_names_, with_types_, format_settings_) { } @@ -71,30 +72,26 @@ void BinaryRowInputFormat::skipTypes() void BinaryRowInputFormat::skipField(size_t file_column) { if (file_column >= read_data_types.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot skip field in Binary format, because it's type is unknown"); + throw Exception(ErrorCodes::CANNOT_SKIP_UNKNOWN_FIELD, "Cannot skip unknown field in RowBinaryWithNames format, because it's type is unknown"); Field field; read_data_types[file_column]->getDefaultSerialization()->deserializeBinary(field, *in); } void registerInputFormatRowBinary(FormatFactory & factory) { - factory.registerInputFormat("RowBinary", []( - ReadBuffer & buf, - const Block & sample, - const IRowInputFormat::Params & params, - const FormatSettings & settings) + auto get_input_creator = [](bool with_names, bool with_types) { - return std::make_shared(buf, sample, params, false, settings); - }); + return [with_names, with_types]( + ReadBuffer & buf, + const Block & sample, + const IRowInputFormat::Params & params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, params, with_names, with_types, settings); + }; + }; - factory.registerInputFormat("RowBinaryWithNamesAndTypes", []( - ReadBuffer & buf, - const Block & sample, - const IRowInputFormat::Params & params, - const FormatSettings & settings) - { - return std::make_shared(buf, sample, params, true, settings); - }); + registerInputFormatWithNamesAndTypes(factory, "RowBinary", get_input_creator); } } diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.h b/src/Processors/Formats/Impl/BinaryRowInputFormat.h index 78c2a626e24..1a9a873097b 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.h +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.h @@ -16,7 +16,7 @@ class ReadBuffer; class BinaryRowInputFormat : public RowInputFormatWithNamesAndTypes { public: - BinaryRowInputFormat(ReadBuffer & in_, Block header, Params params_, bool with_names_and_types, const FormatSettings & format_settings_); + BinaryRowInputFormat(ReadBuffer & in_, Block header, Params params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_); String getName() const override { return "BinaryRowInputFormat"; } diff --git a/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp index dbaee68453e..c5f1e12e0e3 100644 --- a/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -49,23 +50,19 @@ void BinaryRowOutputFormat::writeField(const IColumn & column, const ISerializat void registerOutputFormatRowBinary(FormatFactory & factory) { - factory.registerOutputFormat("RowBinary", []( - WriteBuffer & buf, - const Block & sample, - const RowOutputFormatParams & params, - const FormatSettings &) + auto get_output_creator = [&](bool with_names, bool with_types) { - return std::make_shared(buf, sample, false, false, params); - }); + return [with_names, with_types]( + WriteBuffer & buf, + const Block & sample, + const RowOutputFormatParams & params, + const FormatSettings &) + { + return std::make_shared(buf, sample, with_names, with_types, params); + }; + }; - factory.registerOutputFormat("RowBinaryWithNamesAndTypes", []( - WriteBuffer & buf, - const Block & sample, - const RowOutputFormatParams & params, - const FormatSettings &) - { - return std::make_shared(buf, sample, true, true, params); - }); + registerOutputFormatWithNamesAndTypes(factory, "RowBinary", get_output_creator); } } diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index bf5f3ea468b..926898ccd52 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -3,8 +3,9 @@ #include #include -#include +#include #include +#include #include #include @@ -232,22 +233,22 @@ bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, co void registerInputFormatCSV(FormatFactory & factory) { - auto register_func = [&](const String & format_name, bool with_names, bool with_types) + auto get_input_creator = [](bool with_names, bool with_types) { - factory.registerInputFormat(format_name, [with_names, with_types]( + return [with_names, with_types]( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, const FormatSettings & settings) { return std::make_shared(sample, buf, std::move(params), with_names, with_types, settings); - }); + }; }; - registerInputFormatWithNamesAndTypes("CSV", register_func); + registerInputFormatWithNamesAndTypes(factory, "CSV", get_input_creator); } -static std::pair fileSegmentationEngineCSVImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) +static std::pair fileSegmentationEngineCSVImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size, size_t min_rows) { char * pos = in.position(); bool quotes = false; @@ -287,7 +288,7 @@ static std::pair fileSegmentationEngineCSVImpl(ReadBuffer & in, DB else if (*pos == '\n') { ++number_of_rows; - if (memory.size() + static_cast(pos - in.position()) >= min_chunk_size) + if (memory.size() + static_cast(pos - in.position()) >= min_chunk_size && number_of_rows >= min_rows) need_more_data = false; ++pos; if (loadAtPosition(in, memory, pos) && *pos == '\r') @@ -295,7 +296,7 @@ static std::pair fileSegmentationEngineCSVImpl(ReadBuffer & in, DB } else if (*pos == '\r') { - if (memory.size() + static_cast(pos - in.position()) >= min_chunk_size) + if (memory.size() + static_cast(pos - in.position()) >= min_chunk_size && number_of_rows >= min_rows) need_more_data = false; ++pos; if (loadAtPosition(in, memory, pos) && *pos == '\n') @@ -313,7 +314,14 @@ static std::pair fileSegmentationEngineCSVImpl(ReadBuffer & in, DB void registerFileSegmentationEngineCSV(FormatFactory & factory) { - registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "CSV", &fileSegmentationEngineCSVImpl); + auto get_file_segmentation_engine = [](size_t min_rows) + { + return [min_rows](ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) + { + return fileSegmentationEngineCSVImpl(in, memory, min_chunk_size, min_rows); + }; + }; + registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "CSV", get_file_segmentation_engine); } } diff --git a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp index 9fba7ba3627..40f1d85a104 100644 --- a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp @@ -1,5 +1,6 @@ #include #include +#include #include @@ -74,20 +75,19 @@ void CSVRowOutputFormat::writeBeforeExtremes() void registerOutputFormatCSV(FormatFactory & factory) { - auto register_func = [&](const String & format_name, bool with_names, bool with_types) + auto get_output_creator = [](bool with_names, bool with_types) { - factory.registerOutputFormat(format_name, [=]( + return [with_names, with_types]( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, const FormatSettings & format_settings) { return std::make_shared(buf, sample, with_names, with_types, params, format_settings); - }); - factory.markOutputFormatSupportsParallelFormatting(format_name); + }; }; - registerOutputFormatWithNamesAndTypes("CSV", register_func); + registerOutputFormatWithNamesAndTypes(factory, "CSV", get_output_creator, true); } } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index 95e92709881..3351e23caa9 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -183,26 +184,34 @@ void registerInputFormatJSONCompactEachRow(FormatFactory & factory) { for (bool yield_strings : {true, false}) { - auto register_func = [&](const String & format_name, bool with_names, bool with_types) + auto get_input_creator = [yield_strings](bool with_names, bool with_types) { - factory.registerInputFormat(format_name, [with_names, with_types, yield_strings]( + return [with_names, with_types, yield_strings]( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, const FormatSettings & settings) { return std::make_shared(sample, buf, std::move(params), with_names, with_types, yield_strings, settings); - }); + }; }; - registerInputFormatWithNamesAndTypes(yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", register_func); + registerInputFormatWithNamesAndTypes(factory, yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", get_input_creator); } } void registerFileSegmentationEngineJSONCompactEachRow(FormatFactory & factory) { - registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "JSONCompactEachRow", &fileSegmentationEngineJSONCompactEachRow); - registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "JSONCompactStringsEachRow", &fileSegmentationEngineJSONCompactEachRow); + auto get_file_segmentation_engine = [](size_t min_rows) + { + return [min_rows](ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) + { + return fileSegmentationEngineJSONCompactEachRow(in, memory, min_chunk_size, min_rows); + }; + }; + + registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "JSONCompactEachRow", get_file_segmentation_engine); + registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "JSONCompactStringsEachRow", get_file_segmentation_engine); } } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp index c7df76e3b83..ab77755127b 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB @@ -101,20 +102,19 @@ void registerOutputFormatJSONCompactEachRow(FormatFactory & factory) { for (bool yield_strings : {false, true}) { - auto register_func = [&](const String & format_name, bool with_names, bool with_types) + auto get_output_creator = [yield_strings](bool with_names, bool with_types) { - factory.registerOutputFormat(format_name, [=]( + return [yield_strings, with_names, with_types]( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, const FormatSettings & format_settings) { return std::make_shared(buf, sample, params, format_settings, with_names, with_types, yield_strings); - }); - factory.markOutputFormatSupportsParallelFormatting(format_name); + }; }; - registerOutputFormatWithNamesAndTypes(yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", register_func); + registerOutputFormatWithNamesAndTypes(factory, yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", get_output_creator, true); } } diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 6f04da77ac4..f77e9f5ce77 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -221,25 +222,24 @@ void registerInputFormatTabSeparated(FormatFactory & factory) { for (bool is_raw : {false, true}) { - auto register_func = [&](const String & format_name, bool with_names, bool with_types) + auto get_input_creator = [is_raw](bool with_names, bool with_types) { - factory.registerInputFormat(format_name, [with_names, with_types, is_raw]( + return [with_names, with_types, is_raw]( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, const FormatSettings & settings) { return std::make_shared(sample, buf, std::move(params), with_names, with_types, is_raw, settings); - }); + }; }; - registerInputFormatWithNamesAndTypes(is_raw ? "TabSeparatedRaw" : "TabSeparated", register_func); - registerInputFormatWithNamesAndTypes(is_raw ? "TSVRaw" : "TSV", register_func); + registerInputFormatWithNamesAndTypes(factory, is_raw ? "TabSeparatedRaw" : "TabSeparated", get_input_creator); + registerInputFormatWithNamesAndTypes(factory, is_raw ? "TSVRaw" : "TSV", get_input_creator); } } -template -static std::pair fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) +static std::pair fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size, bool is_raw, size_t min_rows) { bool need_more_data = true; char * pos = in.position(); @@ -247,7 +247,7 @@ static std::pair fileSegmentationEngineTabSeparatedImpl(ReadBuffer while (loadAtPosition(in, memory, pos) && need_more_data) { - if constexpr (is_raw) + if (is_raw) pos = find_first_symbols<'\r', '\n'>(pos, in.buffer().end()); else pos = find_first_symbols<'\\', '\r', '\n'>(pos, in.buffer().end()); @@ -269,7 +269,7 @@ static std::pair fileSegmentationEngineTabSeparatedImpl(ReadBuffer if (*pos == '\n') ++number_of_rows; - if (memory.size() + static_cast(pos - in.position()) >= min_chunk_size) + if ((memory.size() + static_cast(pos - in.position()) >= min_chunk_size) && number_of_rows >= min_rows) need_more_data = false; ++pos; } @@ -282,13 +282,28 @@ static std::pair fileSegmentationEngineTabSeparatedImpl(ReadBuffer void registerFileSegmentationEngineTabSeparated(FormatFactory & factory) { - registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "TSV", &fileSegmentationEngineTabSeparatedImpl); - registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "TabSeparated", &fileSegmentationEngineTabSeparatedImpl); - registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "TSVRaw", &fileSegmentationEngineTabSeparatedImpl); - registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "TabSeparatedRaw", &fileSegmentationEngineTabSeparatedImpl); + for (bool is_raw : {false, true}) + { + auto get_file_segmentation_engine = [is_raw](size_t min_rows) + { + return [is_raw, min_rows](ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) + { + return fileSegmentationEngineTabSeparatedImpl(in, memory, min_chunk_size, is_raw, min_rows); + }; + }; + + registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, is_raw ? "TSVRaw" : "TSV", get_file_segmentation_engine); + registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, is_raw ? "TabSeparatedRaw" : "TabSeparated", get_file_segmentation_engine); + } // We can use the same segmentation engine for TSKV. - factory.registerFileSegmentationEngine("TSKV", &fileSegmentationEngineTabSeparatedImpl); + factory.registerFileSegmentationEngine("TSKV", []( + ReadBuffer & in, + DB::Memory<> & memory, + size_t min_chunk_size) + { + return fileSegmentationEngineTabSeparatedImpl(in, memory, min_chunk_size, false, 0); + }); } } diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index 9a4f079867e..408a5ff9545 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -1,5 +1,6 @@ #include #include +#include #include @@ -78,21 +79,20 @@ void registerOutputFormatTabSeparated(FormatFactory & factory) { for (bool is_raw : {false, true}) { - auto register_func = [&](const String & format_name, bool with_names, bool with_types) + auto get_output_creator = [is_raw](bool with_names, bool with_types) { - factory.registerOutputFormat(format_name, [=]( + return [is_raw, with_names, with_types]( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, const FormatSettings & settings) { return std::make_shared(buf, sample, with_names, with_types, is_raw, params, settings); - }); - factory.markOutputFormatSupportsParallelFormatting(format_name); + }; }; - registerOutputFormatWithNamesAndTypes(is_raw ? "TSVRaw" : "TSV", register_func); - registerOutputFormatWithNamesAndTypes(is_raw ? "TabSeparatedRaw" : "TabSeparated", register_func); + registerOutputFormatWithNamesAndTypes(factory, is_raw ? "TSVRaw" : "TSV", get_output_creator, true); + registerOutputFormatWithNamesAndTypes(factory, is_raw ? "TabSeparatedRaw" : "TabSeparated", get_output_creator, true); } } diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index ee616cb0630..b5690d9dafb 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -247,12 +247,6 @@ bool RowInputFormatWithNamesAndTypes::parseRowAndPrintDiagnosticInfo(MutableColu return parseRowEndWithDiagnosticInfo(out); } -void registerInputFormatWithNamesAndTypes(const String & base_format_name, RegisterFormatWithNamesAndTypesFunc register_func) -{ - register_func(base_format_name, false, false); - register_func(base_format_name + "WithNames", true, false); - register_func(base_format_name + "WithNamesAndTypes", true, true); -} void registerFileSegmentationEngineForFormatWithNamesAndTypes( FormatFactory & factory, const String & base_format_name, FormatFactory::FileSegmentationEngine segmentation_engine) diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index 53a73be1818..f94a5d6b9c8 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -63,13 +63,6 @@ private: std::unordered_map column_indexes_by_names; }; -using RegisterFormatWithNamesAndTypesFunc = std::function; - -void registerInputFormatWithNamesAndTypes(const String & base_format_name, RegisterFormatWithNamesAndTypesFunc register_func); - void registerFileSegmentationEngineForFormatWithNamesAndTypes( FormatFactory & factory, const String & base_format_name, FormatFactory::FileSegmentationEngine segmentation_engine); diff --git a/tests/queries/0_stateless/02102_row_binary_with_names_and_types.reference b/tests/queries/0_stateless/02102_row_binary_with_names_and_types.reference index fc1da360fd6..9011f20cd6a 100644 --- a/tests/queries/0_stateless/02102_row_binary_with_names_and_types.reference +++ b/tests/queries/0_stateless/02102_row_binary_with_names_and_types.reference @@ -1,8 +1,14 @@ 1 text 2020-01-01 1 text 2020-01-01 1 text 2020-01-01 +1 text 2020-01-01 +1 text 2020-01-01 +1 text 2020-01-01 +1 default 1970-01-01 1 default 1970-01-01 1 1970-01-01 +1 1970-01-01 +OK 1 default 1970-01-01 OK OK diff --git a/tests/queries/0_stateless/02102_row_binary_with_names_and_types.sh b/tests/queries/0_stateless/02102_row_binary_with_names_and_types.sh index 8c9ad5abac5..e7307ad3ad5 100755 --- a/tests/queries/0_stateless/02102_row_binary_with_names_and_types.sh +++ b/tests/queries/0_stateless/02102_row_binary_with_names_and_types.sh @@ -9,43 +9,60 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_02102" $CLICKHOUSE_CLIENT -q "CREATE TABLE test_02102 (x UInt32, y String DEFAULT 'default', z Date) engine=Memory()" -$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT RowBinaryWithNames" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNames" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102" $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102" +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102" + + +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT RowBinaryWithNames" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=0 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNames" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102" $CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=0 --input_format_with_types_use_header=0 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" - $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102" $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102" +$CLICKHOUSE_CLIENT -q "SELECT 'text' AS y, toDate('2020-01-01') AS z, toUInt32(1) AS x FORMAT RowBinaryWithNames" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNames" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102" + $CLICKHOUSE_CLIENT -q "SELECT 'text' AS y, toDate('2020-01-01') AS z, toUInt32(1) AS x FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" - $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102" $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102" -$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT RowBinaryWithNames" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNames" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102" +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102" + + +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT RowBinaryWithNames" | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_with_names_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNames" $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102" $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102" $CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" - $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102" $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102" +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, [[1, 2, 3], [4, 5], []] as a FORMAT RowBinaryWithNames" | $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_with_names_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNames" 2>&1 | grep -F -q "CANNOT_SKIP_UNKNOWN_FIELD" && echo 'OK' || echo 'FAIL' + $CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, [[1, 2, 3], [4, 5], []] as a FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" - $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102" $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102" - $CLICKHOUSE_CLIENT -q "SELECT 'text' AS x, toDate('2020-01-01') AS y, toUInt32(1) AS z FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' $CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' as z, toDate('2020-01-01') AS y FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' diff --git a/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.reference b/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.reference new file mode 100644 index 00000000000..962b233e5e7 --- /dev/null +++ b/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.reference @@ -0,0 +1,80 @@ +0 [0,1,2,3,4,5,6,7,8,9] 0 +1 [0,1,2,3,4,5,6,7,8,9,10] 1 +2 [0,1,2,3,4,5,6,7,8,9,10,11] 2 +3 [0,1,2,3,4,5,6,7,8,9,10,11,12] 3 +4 [0,1,2,3,4,5,6,7,8,9,10,11,12,13] 4 +5 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14] 5 +6 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15] 6 +7 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16] 7 +8 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] 8 +9 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18] 9 +0 [0,1,2,3,4,5,6,7,8,9] 0 +1 [0,1,2,3,4,5,6,7,8,9,10] 1 +2 [0,1,2,3,4,5,6,7,8,9,10,11] 2 +3 [0,1,2,3,4,5,6,7,8,9,10,11,12] 3 +4 [0,1,2,3,4,5,6,7,8,9,10,11,12,13] 4 +5 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14] 5 +6 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15] 6 +7 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16] 7 +8 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] 8 +9 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18] 9 +0 [0,1,2,3,4,5,6,7,8,9] 0 +1 [0,1,2,3,4,5,6,7,8,9,10] 1 +2 [0,1,2,3,4,5,6,7,8,9,10,11] 2 +3 [0,1,2,3,4,5,6,7,8,9,10,11,12] 3 +4 [0,1,2,3,4,5,6,7,8,9,10,11,12,13] 4 +5 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14] 5 +6 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15] 6 +7 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16] 7 +8 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] 8 +9 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18] 9 +0 [0,1,2,3,4,5,6,7,8,9] 0 +1 [0,1,2,3,4,5,6,7,8,9,10] 1 +2 [0,1,2,3,4,5,6,7,8,9,10,11] 2 +3 [0,1,2,3,4,5,6,7,8,9,10,11,12] 3 +4 [0,1,2,3,4,5,6,7,8,9,10,11,12,13] 4 +5 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14] 5 +6 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15] 6 +7 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16] 7 +8 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] 8 +9 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18] 9 +0 [0,1,2,3,4,5,6,7,8,9] 0 +1 [0,1,2,3,4,5,6,7,8,9,10] 1 +2 [0,1,2,3,4,5,6,7,8,9,10,11] 2 +3 [0,1,2,3,4,5,6,7,8,9,10,11,12] 3 +4 [0,1,2,3,4,5,6,7,8,9,10,11,12,13] 4 +5 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14] 5 +6 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15] 6 +7 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16] 7 +8 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] 8 +9 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18] 9 +0 [0,1,2,3,4,5,6,7,8,9] 0 +1 [0,1,2,3,4,5,6,7,8,9,10] 1 +2 [0,1,2,3,4,5,6,7,8,9,10,11] 2 +3 [0,1,2,3,4,5,6,7,8,9,10,11,12] 3 +4 [0,1,2,3,4,5,6,7,8,9,10,11,12,13] 4 +5 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14] 5 +6 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15] 6 +7 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16] 7 +8 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] 8 +9 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18] 9 +0 [0,1,2,3,4,5,6,7,8,9] 0 +1 [0,1,2,3,4,5,6,7,8,9,10] 1 +2 [0,1,2,3,4,5,6,7,8,9,10,11] 2 +3 [0,1,2,3,4,5,6,7,8,9,10,11,12] 3 +4 [0,1,2,3,4,5,6,7,8,9,10,11,12,13] 4 +5 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14] 5 +6 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15] 6 +7 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16] 7 +8 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] 8 +9 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18] 9 +0 [0,1,2,3,4,5,6,7,8,9] 0 +1 [0,1,2,3,4,5,6,7,8,9,10] 1 +2 [0,1,2,3,4,5,6,7,8,9,10,11] 2 +3 [0,1,2,3,4,5,6,7,8,9,10,11,12] 3 +4 [0,1,2,3,4,5,6,7,8,9,10,11,12,13] 4 +5 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14] 5 +6 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15] 6 +7 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16] 7 +8 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] 8 +9 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18] 9 diff --git a/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh b/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh new file mode 100755 index 00000000000..30be432b8b2 --- /dev/null +++ b/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash + +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}') + +DATA_FILE=$USER_FILES_PATH/test_02103.data + +FORMATS=('TSVWithNames' 'TSVWithNamesAndTypes' 'TSVRawWithNames' 'TSVRawWithNamesAndTypes' 'CSVWithNames' 'CSVWithNamesAndTypes' 'JSONCompactEachRowWithNames' 'JSONCompactEachRowWithNamesAndTypes') + +for format in "${FORMATS[@]}" +do + $CLICKHOUSE_CLIENT -q "SELECT number, range(number + 10) AS array, toString(number) AS string FROM numbers(10) FORMAT $format" > $DATA_FILE + $CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103.data', '$format', 'number UInt64, array Array(UInt64), string String') SETTINGS input_format_parallel_parsing=1, min_chunk_bytes_for_parallel_parsing=40" +done + +rm $DATA_FILE + From fc7fc633f91b0af94b280687855b7eb57bc85780 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 20 Oct 2021 14:53:07 +0300 Subject: [PATCH 261/919] Mark tests as no-parallel --- .../01375_storage_file_tsv_csv_with_names_write_prefix.sh | 1 + .../0_stateless/02103_with_names_and_types_parallel_parsing.sh | 1 + 2 files changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh b/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh index 7731deaa8ff..a634f689dca 100755 --- a/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh +++ b/tests/queries/0_stateless/01375_storage_file_tsv_csv_with_names_write_prefix.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh b/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh index 30be432b8b2..487282099e2 100755 --- a/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh +++ b/tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 906d6da5d353d24d60e4b9d98a150655e381b448 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 20 Oct 2021 14:55:01 +0300 Subject: [PATCH 262/919] Update TabSeparatedRowInputFormat.cpp --- src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index f77e9f5ce77..066b5290f4f 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -302,7 +302,7 @@ void registerFileSegmentationEngineTabSeparated(FormatFactory & factory) DB::Memory<> & memory, size_t min_chunk_size) { - return fileSegmentationEngineTabSeparatedImpl(in, memory, min_chunk_size, false, 0); + return fileSegmentationEngineTabSeparatedImpl(in, memory, min_chunk_size, false, 1); }); } From b719abd6b5f617183832ef3678d67dde7aa7a24f Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Wed, 20 Oct 2021 15:25:35 +0300 Subject: [PATCH 263/919] Update json-functions.md --- .../sql-reference/functions/json-functions.md | 62 +++++++++++++++++++ 1 file changed, 62 insertions(+) diff --git a/docs/ru/sql-reference/functions/json-functions.md b/docs/ru/sql-reference/functions/json-functions.md index d20d8cf5998..de89a50219b 100644 --- a/docs/ru/sql-reference/functions/json-functions.md +++ b/docs/ru/sql-reference/functions/json-functions.md @@ -307,6 +307,68 @@ SELECT JSONExtractKeysAndValuesRaw('{"a": [-100, 200.0], "b":{"c": {"d": "hello" └───────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` +## JSON_EXISTS(path, json) {#json-exists} + +Если значение существует в документе JSON, то возвращается 1. + +Если значение не существует, то возвращается 0. + +Пример: + +``` sql +SELECT JSON_EXISTS('$.hello', '{"hello":1}'); +SELECT JSON_EXISTS('$.hello.world', '{"hello":{"world":1}}'); +SELECT JSON_EXISTS('$.hello[*]', '{"hello":["world"]}'); +SELECT JSON_EXISTS('$.hello[0]', '{"hello":["world"]}'); +``` + +## JSON_QUERY(path, json) {#json-query} + +Парсит JSON и извлекает строку. + +Если значение не существует, то возвращается пустая строка. + +Пример: + +``` sql +SELECT JSON_QUERY('$.hello', '{"hello":"world"}'); +SELECT JSON_QUERY('$.array[*][0 to 2, 4]', '{"array":[[0, 1, 2, 3, 4, 5], [0, -1, -2, -3, -4, -5]]}'); +SELECT JSON_QUERY('$.hello', '{"hello":2}'); +SELECT toTypeName(JSON_QUERY('$.hello', '{"hello":2}')); +``` + +Результат: + +``` text +["world"] +[0, 1, 4, 0, -1, -4] +[2] +String +``` + +## JSON_VALUE(path, json) {#json-value} + +Парсит JSON и извлекает строку. + +Если значение не существует, то возвращается пустая строка. + +Пример: + +``` sql +SELECT JSON_VALUE('$.hello', '{"hello":"world"}'); +SELECT JSON_VALUE('$.array[*][0 to 2, 4]', '{"array":[[0, 1, 2, 3, 4, 5], [0, -1, -2, -3, -4, -5]]}'); +SELECT JSON_VALUE('$.hello', '{"hello":2}'); +SELECT toTypeName(JSON_VALUE('$.hello', '{"hello":2}')); +``` + +Результат: + +``` text +"world" +0 +2 +String +``` ## toJSONString {#tojsonstring} From 872cca550a9d21e4f5c5b4e996a68f45c254b82a Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 20 Oct 2021 15:47:20 +0300 Subject: [PATCH 264/919] Make better --- src/Formats/FormatFactory.h | 2 +- src/Formats/registerWithNamesAndTypes.cpp | 34 +++---------------- src/Formats/registerWithNamesAndTypes.h | 19 +++-------- .../Formats/Impl/BinaryRowInputFormat.cpp | 8 ++--- .../Formats/Impl/BinaryRowOutputFormat.cpp | 8 ++--- .../Formats/Impl/CSVRowInputFormat.cpp | 18 +++++----- .../Formats/Impl/CSVRowOutputFormat.cpp | 9 ++--- .../Impl/JSONCompactEachRowRowInputFormat.cpp | 19 ++++++----- .../JSONCompactEachRowRowOutputFormat.cpp | 10 +++--- .../Impl/TabSeparatedRowInputFormat.cpp | 21 ++++++------ .../Impl/TabSeparatedRowOutputFormat.cpp | 12 ++++--- 11 files changed, 66 insertions(+), 94 deletions(-) diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 99f86d92e9f..4e10aa4141a 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -67,6 +67,7 @@ public: const Columns & columns, size_t row)>; +private: using InputCreatorFunc = InputFormatPtr( ReadBuffer & buf, const Block & header, @@ -81,7 +82,6 @@ public: const RowOutputFormatParams & params, const FormatSettings & settings)>; -private: /// Some input formats can have non trivial readPrefix() and readSuffix(), /// so in some cases there is no possibility to use parallel parsing. /// The checker should return true if parallel parsing should be disabled. diff --git a/src/Formats/registerWithNamesAndTypes.cpp b/src/Formats/registerWithNamesAndTypes.cpp index ad76cdac0c9..cba578b08c7 100644 --- a/src/Formats/registerWithNamesAndTypes.cpp +++ b/src/Formats/registerWithNamesAndTypes.cpp @@ -3,37 +3,11 @@ namespace DB { -void registerInputFormatWithNamesAndTypes(FormatFactory & factory, const String & base_format_name, GetInputCreatorWithNamesAndTypesFunc get_input_creator) +void registerWithNamesAndTypes(const std::string & base_format_name, RegisterWithNamesAndTypesFunc register_func) { - factory.registerInputFormat(base_format_name, get_input_creator(false, false)); - factory.registerInputFormat(base_format_name + "WithNames", get_input_creator(true, false)); - factory.registerInputFormat(base_format_name + "WithNamesAndTypes", get_input_creator(true, true)); -} - -void registerOutputFormatWithNamesAndTypes( - FormatFactory & factory, - const String & base_format_name, - GetOutputCreatorWithNamesAndTypesFunc get_output_creator, - bool supports_parallel_formatting) -{ - factory.registerOutputFormat(base_format_name, get_output_creator(false, false)); - factory.registerOutputFormat(base_format_name + "WithNames", get_output_creator(true, false)); - factory.registerOutputFormat(base_format_name + "WithNamesAndTypes", get_output_creator(true, true)); - - if (supports_parallel_formatting) - { - factory.markOutputFormatSupportsParallelFormatting(base_format_name); - factory.markOutputFormatSupportsParallelFormatting(base_format_name + "WithNames"); - factory.markOutputFormatSupportsParallelFormatting(base_format_name + "WithNamesAndTypes"); - } -} - -void registerFileSegmentationEngineForFormatWithNamesAndTypes( - FormatFactory & factory, const String & base_format_name, GetFileSegmentationEngineWithNamesAndTypesFunc get_file_segmentation_engine) -{ - factory.registerFileSegmentationEngine(base_format_name, get_file_segmentation_engine(1)); - factory.registerFileSegmentationEngine(base_format_name + "WithNames", get_file_segmentation_engine(2)); - factory.registerFileSegmentationEngine(base_format_name + "WithNamesAndTypes", get_file_segmentation_engine(3)); + register_func(base_format_name, false, false); + register_func(base_format_name + "WithNames", true, false); + register_func(base_format_name + "WithNamesAndTypes", true, true); } } diff --git a/src/Formats/registerWithNamesAndTypes.h b/src/Formats/registerWithNamesAndTypes.h index fdfe1793b3f..d8e74e3421e 100644 --- a/src/Formats/registerWithNamesAndTypes.h +++ b/src/Formats/registerWithNamesAndTypes.h @@ -1,23 +1,12 @@ #pragma once -#include +#include +#include namespace DB { -using GetInputCreatorWithNamesAndTypesFunc = std::function; -void registerInputFormatWithNamesAndTypes( - FormatFactory & factory, const String & base_format_name, GetInputCreatorWithNamesAndTypesFunc get_input_creator); - -using GetOutputCreatorWithNamesAndTypesFunc = std::function; -void registerOutputFormatWithNamesAndTypes( - FormatFactory & factory, - const String & base_format_name, - GetOutputCreatorWithNamesAndTypesFunc get_output_creator, - bool supports_parallel_formatting = false); - -using GetFileSegmentationEngineWithNamesAndTypesFunc = std::function; -void registerFileSegmentationEngineForFormatWithNamesAndTypes( - FormatFactory & factory, const String & base_format_name, GetFileSegmentationEngineWithNamesAndTypesFunc get_file_segmentation_engine); +using RegisterWithNamesAndTypesFunc = std::function; +void registerWithNamesAndTypes(const std::string & base_format_name, RegisterWithNamesAndTypesFunc register_func); } diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp index 76ba5dca019..0506c539c0f 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp @@ -79,19 +79,19 @@ void BinaryRowInputFormat::skipField(size_t file_column) void registerInputFormatRowBinary(FormatFactory & factory) { - auto get_input_creator = [](bool with_names, bool with_types) + auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - return [with_names, with_types]( + factory.registerInputFormat(format_name, [with_names, with_types]( ReadBuffer & buf, const Block & sample, const IRowInputFormat::Params & params, const FormatSettings & settings) { return std::make_shared(buf, sample, params, with_names, with_types, settings); - }; + }); }; - registerInputFormatWithNamesAndTypes(factory, "RowBinary", get_input_creator); + registerWithNamesAndTypes("RowBinary", register_func); } } diff --git a/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp index c5f1e12e0e3..02c4aee5e4e 100644 --- a/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowOutputFormat.cpp @@ -50,19 +50,19 @@ void BinaryRowOutputFormat::writeField(const IColumn & column, const ISerializat void registerOutputFormatRowBinary(FormatFactory & factory) { - auto get_output_creator = [&](bool with_names, bool with_types) + auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - return [with_names, with_types]( + factory.registerOutputFormat(format_name, [with_names, with_types]( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, const FormatSettings &) { return std::make_shared(buf, sample, with_names, with_types, params); - }; + }); }; - registerOutputFormatWithNamesAndTypes(factory, "RowBinary", get_output_creator); + registerWithNamesAndTypes("RowBinary", register_func); } } diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 926898ccd52..9de2b908b1e 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -233,19 +233,19 @@ bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, co void registerInputFormatCSV(FormatFactory & factory) { - auto get_input_creator = [](bool with_names, bool with_types) + auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - return [with_names, with_types]( + factory.registerInputFormat(format_name, [with_names, with_types]( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, const FormatSettings & settings) { return std::make_shared(sample, buf, std::move(params), with_names, with_types, settings); - }; + }); }; - registerInputFormatWithNamesAndTypes(factory, "CSV", get_input_creator); + registerWithNamesAndTypes("CSV", register_func); } static std::pair fileSegmentationEngineCSVImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size, size_t min_rows) @@ -314,14 +314,16 @@ static std::pair fileSegmentationEngineCSVImpl(ReadBuffer & in, DB void registerFileSegmentationEngineCSV(FormatFactory & factory) { - auto get_file_segmentation_engine = [](size_t min_rows) + auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - return [min_rows](ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) + size_t min_rows = 1 + int(with_names) + int(with_types); + factory.registerFileSegmentationEngine(format_name, [min_rows](ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) { return fileSegmentationEngineCSVImpl(in, memory, min_chunk_size, min_rows); - }; + }); }; - registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "CSV", get_file_segmentation_engine); + + registerWithNamesAndTypes("CSV", register_func); } } diff --git a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp index 40f1d85a104..b300928e569 100644 --- a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp @@ -75,19 +75,20 @@ void CSVRowOutputFormat::writeBeforeExtremes() void registerOutputFormatCSV(FormatFactory & factory) { - auto get_output_creator = [](bool with_names, bool with_types) + auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - return [with_names, with_types]( + factory.registerOutputFormat(format_name, [with_names, with_types]( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, const FormatSettings & format_settings) { return std::make_shared(buf, sample, with_names, with_types, params, format_settings); - }; + }); + factory.markOutputFormatSupportsParallelFormatting(format_name); }; - registerOutputFormatWithNamesAndTypes(factory, "CSV", get_output_creator, true); + registerWithNamesAndTypes("CSV", register_func); } } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index 3351e23caa9..ec0a0fcf9e6 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -184,34 +184,35 @@ void registerInputFormatJSONCompactEachRow(FormatFactory & factory) { for (bool yield_strings : {true, false}) { - auto get_input_creator = [yield_strings](bool with_names, bool with_types) + auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - return [with_names, with_types, yield_strings]( + factory.registerInputFormat(format_name, [with_names, with_types, yield_strings]( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, const FormatSettings & settings) { return std::make_shared(sample, buf, std::move(params), with_names, with_types, yield_strings, settings); - }; + }); }; - registerInputFormatWithNamesAndTypes(factory, yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", get_input_creator); + registerWithNamesAndTypes(yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", register_func); } } void registerFileSegmentationEngineJSONCompactEachRow(FormatFactory & factory) { - auto get_file_segmentation_engine = [](size_t min_rows) + auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - return [min_rows](ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) + size_t min_rows = 1 + int(with_names) + int(with_types); + factory.registerFileSegmentationEngine(format_name, [min_rows](ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) { return fileSegmentationEngineJSONCompactEachRow(in, memory, min_chunk_size, min_rows); - }; + }); }; - registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "JSONCompactEachRow", get_file_segmentation_engine); - registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "JSONCompactStringsEachRow", get_file_segmentation_engine); + registerWithNamesAndTypes("JSONCompactEachRow", register_func); + registerWithNamesAndTypes("JSONCompactStringsEachRow", register_func); } } diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp index ab77755127b..cdff7ff2070 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp @@ -102,19 +102,21 @@ void registerOutputFormatJSONCompactEachRow(FormatFactory & factory) { for (bool yield_strings : {false, true}) { - auto get_output_creator = [yield_strings](bool with_names, bool with_types) + auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - return [yield_strings, with_names, with_types]( + factory.registerOutputFormat(format_name, [yield_strings, with_names, with_types]( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, const FormatSettings & format_settings) { return std::make_shared(buf, sample, params, format_settings, with_names, with_types, yield_strings); - }; + }); + + factory.markOutputFormatSupportsParallelFormatting(format_name); }; - registerOutputFormatWithNamesAndTypes(factory, yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", get_output_creator, true); + registerWithNamesAndTypes(yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", register_func); } } diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 066b5290f4f..117875c5cf0 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -222,20 +222,20 @@ void registerInputFormatTabSeparated(FormatFactory & factory) { for (bool is_raw : {false, true}) { - auto get_input_creator = [is_raw](bool with_names, bool with_types) + auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - return [with_names, with_types, is_raw]( + factory.registerInputFormat(format_name, [with_names, with_types, is_raw]( ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, const FormatSettings & settings) { return std::make_shared(sample, buf, std::move(params), with_names, with_types, is_raw, settings); - }; + }); }; - registerInputFormatWithNamesAndTypes(factory, is_raw ? "TabSeparatedRaw" : "TabSeparated", get_input_creator); - registerInputFormatWithNamesAndTypes(factory, is_raw ? "TSVRaw" : "TSV", get_input_creator); + registerWithNamesAndTypes(is_raw ? "TabSeparatedRaw" : "TabSeparated", register_func); + registerWithNamesAndTypes(is_raw ? "TSVRaw" : "TSV", register_func); } } @@ -284,16 +284,17 @@ void registerFileSegmentationEngineTabSeparated(FormatFactory & factory) { for (bool is_raw : {false, true}) { - auto get_file_segmentation_engine = [is_raw](size_t min_rows) + auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - return [is_raw, min_rows](ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) + size_t min_rows = 1 + int(with_names) + int(with_types); + factory.registerFileSegmentationEngine(format_name, [is_raw, min_rows](ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) { return fileSegmentationEngineTabSeparatedImpl(in, memory, min_chunk_size, is_raw, min_rows); - }; + }); }; - registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, is_raw ? "TSVRaw" : "TSV", get_file_segmentation_engine); - registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, is_raw ? "TabSeparatedRaw" : "TabSeparated", get_file_segmentation_engine); + registerWithNamesAndTypes(is_raw ? "TSVRaw" : "TSV", register_func); + registerWithNamesAndTypes(is_raw ? "TabSeparatedRaw" : "TabSeparated", register_func); } // We can use the same segmentation engine for TSKV. diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index 408a5ff9545..df0c19ad409 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -79,20 +79,22 @@ void registerOutputFormatTabSeparated(FormatFactory & factory) { for (bool is_raw : {false, true}) { - auto get_output_creator = [is_raw](bool with_names, bool with_types) + auto register_func = [&](const String & format_name, bool with_names, bool with_types) { - return [is_raw, with_names, with_types]( + factory.registerOutputFormat(format_name, [is_raw, with_names, with_types]( WriteBuffer & buf, const Block & sample, const RowOutputFormatParams & params, const FormatSettings & settings) { return std::make_shared(buf, sample, with_names, with_types, is_raw, params, settings); - }; + }); + + factory.markOutputFormatSupportsParallelFormatting(format_name); }; - registerOutputFormatWithNamesAndTypes(factory, is_raw ? "TSVRaw" : "TSV", get_output_creator, true); - registerOutputFormatWithNamesAndTypes(factory, is_raw ? "TabSeparatedRaw" : "TabSeparated", get_output_creator, true); + registerWithNamesAndTypes(is_raw ? "TSVRaw" : "TSV", register_func); + registerWithNamesAndTypes(is_raw ? "TabSeparatedRaw" : "TabSeparated", register_func); } } From 240895fba765180d3e6021b9db9ba8590a580c98 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 16 Oct 2021 23:07:55 +0800 Subject: [PATCH 265/919] Fix projections with JOIN alias columns --- src/Interpreters/ExpressionAnalyzer.cpp | 3 ++- .../0_stateless/01710_projection_with_joins.reference | 0 tests/queries/0_stateless/01710_projection_with_joins.sql | 6 ++++++ 3 files changed, 8 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01710_projection_with_joins.reference create mode 100644 tests/queries/0_stateless/01710_projection_with_joins.sql diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index a1f7a3c71e5..3cb3c1b47ab 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -890,9 +890,10 @@ static std::unique_ptr buildJoinedPlan( * - in the addExternalStorage function, the JOIN (SELECT ...) subquery is replaced with JOIN _data1, * in the subquery_for_set object this subquery is exposed as source and the temporary table _data1 as the `table`. * - this function shows the expression JOIN _data1. + * - JOIN tables will need aliases to correctly resolve USING clause. */ auto interpreter = interpretSubquery( - join_element.table_expression, context, original_right_columns, query_options.copy().setWithAllColumns()); + join_element.table_expression, context, original_right_columns, query_options.copy().setWithAllColumns().ignoreAlias(false)); auto joined_plan = std::make_unique(); interpreter->buildQueryPlan(*joined_plan); { diff --git a/tests/queries/0_stateless/01710_projection_with_joins.reference b/tests/queries/0_stateless/01710_projection_with_joins.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01710_projection_with_joins.sql b/tests/queries/0_stateless/01710_projection_with_joins.sql new file mode 100644 index 00000000000..97dc396f362 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_with_joins.sql @@ -0,0 +1,6 @@ +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; + +set allow_experimental_projection_optimization=1; +select s from t join (select toUInt16(1) as s) x using (s); From 2f615e9176d8bb75da7a9c69586e9f5e4d81c46e Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 19 Oct 2021 22:09:43 +0800 Subject: [PATCH 266/919] Use original_query for projection analysis --- src/Interpreters/InterpreterSelectQuery.cpp | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/SelectQueryInfo.h | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 35c8c32c65b..59d0b88bae7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -272,6 +272,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.ignore_projections = options.ignore_projections; query_info.is_projection_query = options.is_projection_query; + query_info.original_query = query_ptr->clone(); initSettings(); const Settings & settings = context->getSettingsRef(); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ee55a6826c3..6503e6dca12 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4531,7 +4531,7 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( if (!settings.allow_experimental_projection_optimization || query_info.ignore_projections || query_info.is_projection_query) return false; - const auto & query_ptr = query_info.query; + const auto & query_ptr = query_info.original_query; if (auto * select = query_ptr->as(); select) { diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index b08818a2baa..fe7b22d331b 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -133,6 +133,7 @@ struct SelectQueryInfo { ASTPtr query; ASTPtr view_query; /// Optimized VIEW query + ASTPtr original_query; /// Unmodified query for projection analysis /// Cluster for the query. ClusterPtr cluster; From 0a90be83172a1fc944976af0da595becad71f48a Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 20 Oct 2021 17:12:07 +0300 Subject: [PATCH 267/919] Remove write_final_mark setting --- src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp | 2 +- src/Storages/MergeTree/MergeTreeSettings.h | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index 83fd9692e49..3d6b12efdf1 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -49,7 +49,7 @@ IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( , columns_list(columns_list_) , settings(settings_) , index_granularity(index_granularity_) - , with_final_mark(storage.getSettings()->write_final_mark && settings.can_use_adaptive_granularity) + , with_final_mark(settings.can_use_adaptive_granularity) { } diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index e7c2ee1f73c..26a0fbda79b 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -116,7 +116,7 @@ struct Settings; M(Int64, merge_with_recompression_ttl_timeout, 3600 * 4, "Minimal time in seconds, when merge with recompression TTL can be repeated.", 0) \ M(Bool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.", 0) \ M(Bool, materialize_ttl_recalculate_only, false, "Only recalculate ttl info when MATERIALIZE TTL", 0) \ - M(Bool, write_final_mark, true, "Write final mark after end of column (0 - disabled, do nothing if index_granularity_bytes=0)", 0) \ + M(Bool, write_final_mark, true, "Deprecated setting", 0) \ M(Bool, enable_mixed_granularity_parts, true, "Enable parts with adaptive and non adaptive granularity", 0) \ M(MaxThreads, max_part_loading_threads, 0, "The number of threads to load data parts at startup.", 0) \ M(MaxThreads, max_part_removal_threads, 0, "The number of threads for concurrent removal of inactive data parts. One is usually enough, but in 'Google Compute Environment SSD Persistent Disks' file removal (unlink) operation is extraordinarily slow and you probably have to increase this number (recommended is up to 16).", 0) \ @@ -164,7 +164,7 @@ struct MergeTreeSettings : public BaseSettings /// We check settings after storage creation static bool isReadonlySetting(const String & name) { - return name == "index_granularity" || name == "index_granularity_bytes" || name == "write_final_mark" + return name == "index_granularity" || name == "index_granularity_bytes" || name == "enable_mixed_granularity_parts"; } From 6b34398e9cd99c7f0b925342356b368d6d17e884 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 20 Oct 2021 17:17:20 +0300 Subject: [PATCH 268/919] support VALUES format in async inserts --- src/Parsers/ParserInsertQuery.cpp | 9 +- src/Processors/Formats/IInputFormat.h | 3 +- .../Formats/Impl/ValuesBlockInputFormat.cpp | 146 ++++++++++-------- .../Formats/Impl/ValuesBlockInputFormat.h | 6 +- .../getSourceFromASTInsertQuery.cpp | 8 +- .../02015_async_insert_7.reference | 6 + .../0_stateless/02015_async_insert_7.sh | 20 +++ 7 files changed, 126 insertions(+), 72 deletions(-) create mode 100644 tests/queries/0_stateless/02015_async_insert_7.reference create mode 100755 tests/queries/0_stateless/02015_async_insert_7.sh diff --git a/src/Parsers/ParserInsertQuery.cpp b/src/Parsers/ParserInsertQuery.cpp index 6ac6d3b50b9..1bc552af384 100644 --- a/src/Parsers/ParserInsertQuery.cpp +++ b/src/Parsers/ParserInsertQuery.cpp @@ -131,18 +131,22 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } Pos before_values = pos; + String format_str; /// VALUES or FROM INFILE or FORMAT or SELECT if (!infile && s_values.ignore(pos, expected)) { /// If VALUES is defined in query, everything except setting will be parsed as data data = pos->begin; + format_str = "Values"; } else if (s_format.ignore(pos, expected)) { /// If FORMAT is defined, read format name if (!name_p.parse(pos, format, expected)) return false; + + tryGetIdentifierNameInto(format, format_str); } else if (s_select.ignore(pos, expected) || s_with.ignore(pos,expected)) { @@ -155,6 +159,8 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) /// FORMAT section is expected if we have input() in SELECT part if (s_format.ignore(pos, expected) && !name_p.parse(pos, format, expected)) return false; + + tryGetIdentifierNameInto(format, format_str); } else if (s_watch.ignore(pos, expected)) { @@ -242,9 +248,8 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) tryGetIdentifierNameInto(table, query->table_id.table_name); } - tryGetIdentifierNameInto(format, query->format); - query->columns = columns; + query->format = std::move(format_str); query->select = select; query->watch = watch; query->settings_ast = settings_ast; diff --git a/src/Processors/Formats/IInputFormat.h b/src/Processors/Formats/IInputFormat.h index f133161c3ec..8f5992efb1b 100644 --- a/src/Processors/Formats/IInputFormat.h +++ b/src/Processors/Formats/IInputFormat.h @@ -55,6 +55,8 @@ public: */ virtual void resetParser(); + virtual void setReadBuffer(ReadBuffer & in_); + virtual const BlockMissingValues & getMissingValues() const { static const BlockMissingValues none; @@ -70,7 +72,6 @@ public: void setCurrentUnitNumber(size_t current_unit_number_) { current_unit_number = current_unit_number_; } void addBuffer(std::unique_ptr buffer) { owned_buffers.emplace_back(std::move(buffer)); } - void setReadBuffer(ReadBuffer & in_); protected: ColumnMappingPtr column_mapping{}; diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 0f6a21055d0..c15d0d608ee 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -32,13 +32,25 @@ namespace ErrorCodes } -ValuesBlockInputFormat::ValuesBlockInputFormat(ReadBuffer & in_, const Block & header_, const RowInputFormatParams & params_, - const FormatSettings & format_settings_) - : IInputFormat(header_, buf), buf(in_), params(params_), - format_settings(format_settings_), num_columns(header_.columns()), - parser_type_for_column(num_columns, ParserType::Streaming), - attempts_to_deduce_template(num_columns), attempts_to_deduce_template_cached(num_columns), - rows_parsed_using_template(num_columns), templates(num_columns), types(header_.getDataTypes()) +ValuesBlockInputFormat::ValuesBlockInputFormat( + ReadBuffer & in_, + const Block & header_, + const RowInputFormatParams & params_, + const FormatSettings & format_settings_) + : ValuesBlockInputFormat(std::make_unique(in_), header_, params_, format_settings_) +{ +} + +ValuesBlockInputFormat::ValuesBlockInputFormat( + std::unique_ptr buf_, + const Block & header_, + const RowInputFormatParams & params_, + const FormatSettings & format_settings_) + : IInputFormat(header_, *buf_), buf(std::move(buf_)), + params(params_), format_settings(format_settings_), num_columns(header_.columns()), + parser_type_for_column(num_columns, ParserType::Streaming), + attempts_to_deduce_template(num_columns), attempts_to_deduce_template_cached(num_columns), + rows_parsed_using_template(num_columns), templates(num_columns), types(header_.getDataTypes()) { serializations.resize(types.size()); for (size_t i = 0; i < types.size(); ++i) @@ -58,8 +70,8 @@ Chunk ValuesBlockInputFormat::generate() { try { - skipWhitespaceIfAny(buf); - if (buf.eof() || *buf.position() == ';') + skipWhitespaceIfAny(*buf); + if (buf->eof() || *buf->position() == ';') break; readRow(columns, rows_in_block); } @@ -99,12 +111,12 @@ Chunk ValuesBlockInputFormat::generate() void ValuesBlockInputFormat::readRow(MutableColumns & columns, size_t row_num) { - assertChar('(', buf); + assertChar('(', *buf); for (size_t column_idx = 0; column_idx < num_columns; ++column_idx) { - skipWhitespaceIfAny(buf); - PeekableReadBufferCheckpoint checkpoint{buf}; + skipWhitespaceIfAny(*buf); + PeekableReadBufferCheckpoint checkpoint{*buf}; bool read; /// Parse value using fast streaming parser for literals and slow SQL parser for expressions. @@ -123,9 +135,9 @@ void ValuesBlockInputFormat::readRow(MutableColumns & columns, size_t row_num) /// If read is true, value still may be missing. Bit mask for these values will be copied from ConstantExpressionTemplate later. } - skipWhitespaceIfAny(buf); - if (!buf.eof() && *buf.position() == ',') - ++buf.position(); + skipWhitespaceIfAny(*buf); + if (!buf->eof() && *buf->position() == ',') + ++buf->position(); ++total_rows; } @@ -134,7 +146,7 @@ bool ValuesBlockInputFormat::tryParseExpressionUsingTemplate(MutableColumnPtr & { /// Try to parse expression using template if one was successfully deduced while parsing the first row auto settings = context->getSettingsRef(); - if (templates[column_idx]->parseExpression(buf, format_settings, settings)) + if (templates[column_idx]->parseExpression(*buf, format_settings, settings)) { ++rows_parsed_using_template[column_idx]; return true; @@ -154,7 +166,7 @@ bool ValuesBlockInputFormat::tryParseExpressionUsingTemplate(MutableColumnPtr & } /// Do not use this template anymore templates[column_idx].reset(); - buf.rollbackToCheckpoint(); + buf->rollbackToCheckpoint(); /// It will deduce new template or fallback to slow SQL parser return parseExpression(*column, column_idx); @@ -169,13 +181,13 @@ bool ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx) const auto & type = types[column_idx]; const auto & serialization = serializations[column_idx]; if (format_settings.null_as_default && !type->isNullable()) - read = SerializationNullable::deserializeTextQuotedImpl(column, buf, format_settings, serialization); + read = SerializationNullable::deserializeTextQuotedImpl(column, *buf, format_settings, serialization); else - serialization->deserializeTextQuoted(column, buf, format_settings); + serialization->deserializeTextQuoted(column, *buf, format_settings); rollback_on_exception = true; - skipWhitespaceIfAny(buf); + skipWhitespaceIfAny(*buf); assertDelimiterAfterValue(column_idx); return read; } @@ -190,7 +202,7 @@ bool ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx) /// Switch to SQL parser and don't try to use streaming parser for complex expressions /// Note: Throwing exceptions for each expression may be very slow because of stacktraces - buf.rollbackToCheckpoint(); + buf->rollbackToCheckpoint(); return parseExpression(column, column_idx); } } @@ -284,11 +296,11 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx /// We need continuous memory containing the expression to use Lexer skipToNextRow(0, 1); - buf.makeContinuousMemoryFromCheckpointToPos(); - buf.rollbackToCheckpoint(); + buf->makeContinuousMemoryFromCheckpointToPos(); + buf->rollbackToCheckpoint(); Expected expected; - Tokens tokens(buf.position(), buf.buffer().end()); + Tokens tokens(buf->position(), buf->buffer().end()); IParser::Pos token_iterator(tokens, settings.max_parser_depth); ASTPtr ast; @@ -302,7 +314,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx if (!parsed) throw Exception("Cannot parse expression of type " + type.getName() + " here: " - + String(buf.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf.buffer().end() - buf.position())), + + String(buf->position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf->buffer().end() - buf->position())), ErrorCodes::SYNTAX_ERROR); ++token_iterator; @@ -316,9 +328,9 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx try { const auto & serialization = serializations[column_idx]; - serialization->deserializeTextQuoted(column, buf, format_settings); + serialization->deserializeTextQuoted(column, *buf, format_settings); rollback_on_exception = true; - skipWhitespaceIfAny(buf); + skipWhitespaceIfAny(*buf); if (checkDelimiterAfterValue(column_idx)) ok = true; } @@ -366,8 +378,8 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx else ++attempts_to_deduce_template[column_idx]; - buf.rollbackToCheckpoint(); - if (templates[column_idx]->parseExpression(buf, format_settings, settings)) + buf->rollbackToCheckpoint(); + if (templates[column_idx]->parseExpression(*buf, format_settings, settings)) { ++rows_parsed_using_template[column_idx]; parser_type_for_column[column_idx] = ParserType::BatchTemplate; @@ -384,9 +396,9 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx std::rethrow_exception(exception); else { - buf.rollbackToCheckpoint(); - size_t len = const_cast(token_iterator->begin) - buf.position(); - throw Exception("Cannot deduce template of expression: " + std::string(buf.position(), len), ErrorCodes::SYNTAX_ERROR); + buf->rollbackToCheckpoint(); + size_t len = const_cast(token_iterator->begin) - buf->position(); + throw Exception("Cannot deduce template of expression: " + std::string(buf->position(), len), ErrorCodes::SYNTAX_ERROR); } } /// Continue parsing without template @@ -397,7 +409,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx throw Exception("Interpreting expressions is disabled", ErrorCodes::SUPPORT_IS_DISABLED); /// Try to evaluate single expression if other parsers don't work - buf.position() = const_cast(token_iterator->begin); + buf->position() = const_cast(token_iterator->begin); std::pair value_raw = evaluateConstantExpression(ast, context); @@ -416,10 +428,10 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx type.insertDefaultInto(column); return false; } - buf.rollbackToCheckpoint(); + buf->rollbackToCheckpoint(); throw Exception{"Cannot insert NULL value into a column of type '" + type.getName() + "'" + " at: " + - String(buf.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf.buffer().end() - buf.position())), + String(buf->position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf->buffer().end() - buf->position())), ErrorCodes::TYPE_MISMATCH}; } @@ -430,61 +442,61 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx /// Can be used in fileSegmentationEngine for parallel parsing of Values bool ValuesBlockInputFormat::skipToNextRow(size_t min_chunk_bytes, int balance) { - skipWhitespaceIfAny(buf); - if (buf.eof() || *buf.position() == ';') + skipWhitespaceIfAny(*buf); + if (buf->eof() || *buf->position() == ';') return false; bool quoted = false; - size_t chunk_begin_buf_count = buf.count(); - while (!buf.eof() && (balance || buf.count() - chunk_begin_buf_count < min_chunk_bytes)) + size_t chunk_begin_buf_count = buf->count(); + while (!buf->eof() && (balance || buf->count() - chunk_begin_buf_count < min_chunk_bytes)) { - buf.position() = find_first_symbols<'\\', '\'', ')', '('>(buf.position(), buf.buffer().end()); - if (buf.position() == buf.buffer().end()) + buf->position() = find_first_symbols<'\\', '\'', ')', '('>(buf->position(), buf->buffer().end()); + if (buf->position() == buf->buffer().end()) continue; - if (*buf.position() == '\\') + if (*buf->position() == '\\') { - ++buf.position(); - if (!buf.eof()) - ++buf.position(); + ++buf->position(); + if (!buf->eof()) + ++buf->position(); } - else if (*buf.position() == '\'') + else if (*buf->position() == '\'') { quoted ^= true; - ++buf.position(); + ++buf->position(); } - else if (*buf.position() == ')') + else if (*buf->position() == ')') { - ++buf.position(); + ++buf->position(); if (!quoted) --balance; } - else if (*buf.position() == '(') + else if (*buf->position() == '(') { - ++buf.position(); + ++buf->position(); if (!quoted) ++balance; } } - if (!buf.eof() && *buf.position() == ',') - ++buf.position(); + if (!buf->eof() && *buf->position() == ',') + ++buf->position(); return true; } void ValuesBlockInputFormat::assertDelimiterAfterValue(size_t column_idx) { if (unlikely(!checkDelimiterAfterValue(column_idx))) - throwAtAssertionFailed((column_idx + 1 == num_columns) ? ")" : ",", buf); + throwAtAssertionFailed((column_idx + 1 == num_columns) ? ")" : ",", *buf); } bool ValuesBlockInputFormat::checkDelimiterAfterValue(size_t column_idx) { - skipWhitespaceIfAny(buf); + skipWhitespaceIfAny(*buf); if (likely(column_idx + 1 != num_columns)) - return checkChar(',', buf); + return checkChar(',', *buf); else - return checkChar(')', buf); + return checkChar(')', *buf); } bool ValuesBlockInputFormat::shouldDeduceNewTemplate(size_t column_idx) @@ -516,21 +528,21 @@ bool ValuesBlockInputFormat::shouldDeduceNewTemplate(size_t column_idx) void ValuesBlockInputFormat::readPrefix() { /// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it. - skipBOMIfExists(buf); + skipBOMIfExists(*buf); } void ValuesBlockInputFormat::readSuffix() { - if (!buf.eof() && *buf.position() == ';') + if (!buf->eof() && *buf->position() == ';') { - ++buf.position(); - skipWhitespaceIfAny(buf); - if (buf.hasUnreadData()) + ++buf->position(); + skipWhitespaceIfAny(*buf); + if (buf->hasUnreadData()) throw Exception("Cannot read data after semicolon", ErrorCodes::CANNOT_READ_ALL_DATA); return; } - if (buf.hasUnreadData()) + if (buf->hasUnreadData()) throw Exception("Unread data in PeekableReadBuffer will be lost. Most likely it's a bug.", ErrorCodes::LOGICAL_ERROR); } @@ -539,10 +551,16 @@ void ValuesBlockInputFormat::resetParser() IInputFormat::resetParser(); // I'm not resetting parser modes here. // There is a good chance that all messages have the same format. - buf.reset(); + buf->reset(); total_rows = 0; } +void ValuesBlockInputFormat::setReadBuffer(ReadBuffer & in_) +{ + buf = std::make_unique(in_); + IInputFormat::setReadBuffer(*buf); +} + void registerInputFormatValues(FormatFactory & factory) { factory.registerInputFormat("Values", []( diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index b1e44f5d104..5bbd4bea5ba 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -32,6 +32,7 @@ public: String getName() const override { return "ValuesBlockInputFormat"; } void resetParser() override; + void setReadBuffer(ReadBuffer & in_) override; /// TODO: remove context somehow. void setContext(ContextPtr context_) { context = Context::createCopy(context_); } @@ -39,6 +40,9 @@ public: const BlockMissingValues & getMissingValues() const override { return block_missing_values; } private: + ValuesBlockInputFormat(std::unique_ptr buf_, const Block & header_, const RowInputFormatParams & params_, + const FormatSettings & format_settings_); + enum class ParserType { Streaming, @@ -66,7 +70,7 @@ private: bool skipToNextRow(size_t min_chunk_bytes = 0, int balance = 0); - PeekableReadBuffer buf; + std::unique_ptr buf; const RowInputFormatParams params; diff --git a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp index 86998614189..64c8a01bb9c 100644 --- a/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp +++ b/src/Processors/Transforms/getSourceFromASTInsertQuery.cpp @@ -42,12 +42,12 @@ InputFormatPtr getInputFormatFromASTInsertQuery( if (ast_insert_query->infile && context->getApplicationType() == Context::ApplicationType::SERVER) throw Exception("Query has infile and was send directly to server", ErrorCodes::UNKNOWN_TYPE_OF_QUERY); - String format = ast_insert_query->format; - if (format.empty()) + if (ast_insert_query->format.empty()) { if (input_function) throw Exception("FORMAT must be specified for function input()", ErrorCodes::INVALID_USAGE_OF_INPUT); - format = "Values"; + else + throw Exception("Logical error: INSERT query requires format to be set", ErrorCodes::LOGICAL_ERROR); } /// Data could be in parsed (ast_insert_query.data) and in not parsed yet (input_buffer_tail_part) part of query. @@ -59,7 +59,7 @@ InputFormatPtr getInputFormatFromASTInsertQuery( : std::make_unique(); /// Create a source from input buffer using format from query - auto source = context->getInputFormat(format, *input_buffer, header, context->getSettings().max_insert_block_size); + auto source = context->getInputFormat(ast_insert_query->format, *input_buffer, header, context->getSettings().max_insert_block_size); source->addBuffer(std::move(input_buffer)); return source; } diff --git a/tests/queries/0_stateless/02015_async_insert_7.reference b/tests/queries/0_stateless/02015_async_insert_7.reference new file mode 100644 index 00000000000..8816e320ee8 --- /dev/null +++ b/tests/queries/0_stateless/02015_async_insert_7.reference @@ -0,0 +1,6 @@ +1 a +2 b +3 c +4 d +5 e +6 f diff --git a/tests/queries/0_stateless/02015_async_insert_7.sh b/tests/queries/0_stateless/02015_async_insert_7.sh new file mode 100755 index 00000000000..c8cbbc48a29 --- /dev/null +++ b/tests/queries/0_stateless/02015_async_insert_7.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +url="${CLICKHOUSE_URL}&async_insert=1&wait_for_async_insert=1" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS async_inserts" +${CLICKHOUSE_CLIENT} -q "CREATE TABLE async_inserts (id UInt32, s String) ENGINE = Memory" + +${CLICKHOUSE_CURL} -sS $url -d "INSERT INTO async_inserts VALUES (1, 'a') (2, 'b')" & +${CLICKHOUSE_CURL} -sS $url -d "INSERT INTO async_inserts VALUES (3, 'c'), (4, 'd')" & +${CLICKHOUSE_CURL} -sS $url -d "INSERT INTO async_inserts VALUES (5, 'e'), (6, 'f'), " & + +wait + +${CLICKHOUSE_CLIENT} -q "SELECT * FROM async_inserts ORDER BY id" + +${CLICKHOUSE_CLIENT} -q "DROP TABLE async_inserts" From 7cb6cddf6dfe6c76a8baa397ef587aeb3edee852 Mon Sep 17 00:00:00 2001 From: gyuton Date: Wed, 20 Oct 2021 19:52:18 +0300 Subject: [PATCH 269/919] Initial draft --- .../functions/splitting-merging-functions.md | 37 ++++++++++++++++++ .../functions/splitting-merging-functions.md | 38 +++++++++++++++++++ 2 files changed, 75 insertions(+) diff --git a/docs/en/sql-reference/functions/splitting-merging-functions.md b/docs/en/sql-reference/functions/splitting-merging-functions.md index 718d5a977b9..bc06193612a 100644 --- a/docs/en/sql-reference/functions/splitting-merging-functions.md +++ b/docs/en/sql-reference/functions/splitting-merging-functions.md @@ -270,3 +270,40 @@ Result: │ [['abc','123'],['8','"hkl"']] │ └───────────────────────────────────────────────────────────────────────┘ ``` + +## ngrams {#ngrams} + +Splits the ASCII string into n-grams of `ngramsize` symbols. + +**Syntax** + +``` sql +ngrams(string, ngramsize) +``` + +**Arguments** + +- `string` — String. [String](../../sql-reference/data-types/string.md) or [FixedString](../../sql-reference/data-types/fixedstring.md). +- `ngramsize` — The size of an n-gram. [UInt](../../sql-reference/data-types/int-uint.md). + +**Returned values** + +- Array with n-grams. + +Type: [Array](../../sql-reference/data-types/array.md)([FixedString](../../sql-reference/data-types/fixedstring.md)). + +**Example** + +Query: + +``` sql +SELECT ngrams('ClickHouse', 3); +``` + +Result: + +``` text +┌─ngrams('ClickHouse', 3)───────────────────────────┐ +│ ['Cli','lic','ick','ckH','kHo','Hou','ous','use'] │ +└───────────────────────────────────────────────────┘ +``` diff --git a/docs/ru/sql-reference/functions/splitting-merging-functions.md b/docs/ru/sql-reference/functions/splitting-merging-functions.md index efe74dba043..9b9aab4c437 100644 --- a/docs/ru/sql-reference/functions/splitting-merging-functions.md +++ b/docs/ru/sql-reference/functions/splitting-merging-functions.md @@ -232,3 +232,41 @@ SELECT alphaTokens('abca1abc'); │ ['abca','abc'] │ └─────────────────────────┘ ``` + +## ngrams {#ngrams} + +Выделяет из ASCII строки отрезки (n-граммы) размером `ngramsize` символов. + +**Синтаксис** + +``` sql +ngrams(string, ngramsize) +``` + +**Аргументы** + +- `string` — строка. [String](../../sql-reference/data-types/string.md) or [FixedString](../../sql-reference/data-types/fixedstring.md). +- `ngramsize` — размер n-грамм. [UInt](../../sql-reference/data-types/int-uint.md). + +**Возвращаемые значения** + +- Массив с n-граммами. + +Тип: [Array](../../sql-reference/data-types/array.md)([FixedString](../../sql-reference/data-types/fixedstring.md)). + +**Пример** + +Запрос: + +``` sql +SELECT ngrams('ClickHouse', 3); +``` + +Результат: + +``` text +┌─ngrams('ClickHouse', 3)───────────────────────────┐ +│ ['Cli','lic','ick','ckH','kHo','Hou','ous','use'] │ +└───────────────────────────────────────────────────┘ +``` + From 1241db37950503f9eb72b72cf2172ae17fc8b620 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Wed, 20 Oct 2021 20:18:14 +0300 Subject: [PATCH 270/919] document the SQL-statements INTERSECT, EXCEPT and ANY, ALL operators --- docs/en/sql-reference/operators/index.md | 48 ++++++++++ .../sql-reference/statements/select/except.md | 86 ++++++++++++++++++ .../sql-reference/statements/select/index.md | 2 + .../statements/select/intersect.md | 87 +++++++++++++++++++ 4 files changed, 223 insertions(+) create mode 100644 docs/en/sql-reference/statements/select/except.md create mode 100644 docs/en/sql-reference/statements/select/intersect.md diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index dbda5f5dd69..159fc692e94 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -71,6 +71,54 @@ ClickHouse transforms operators to their corresponding functions at the query pa `a GLOBAL NOT IN ...` – The `globalNotIn(a, b)` function. +`ALL` – The operator allows you to get data by comparing the value with the list of values returned by the subquery. The comparison condition must be running for all values of the subquery. + +The subquery must select values of the same type as those compared in the main predicate. + +**Example** + +Query: + +``` sql +SELECT number AS a FROM numbers(10) WHERE a > all(SELECT number FROM numbers(3, 3)); +``` + +Result: + +``` text +┌─a─┐ +│ 6 │ +│ 7 │ +│ 8 │ +│ 9 │ +└───┘ +``` + +`ANY` – The operator is comparing a value to a set of values returned by a subquery. The comparison condition must be running for any values of the subquery. + +The subquery must select values of the same type as those compared in the main predicate. + +**Example** + +Query: + +``` sql +SELECT number AS a FROM numbers(10) WHERE a > any(SELECT number FROM numbers(3, 3)); +``` + +Result: + +``` text +┌─a─┐ +│ 4 │ +│ 5 │ +│ 6 │ +│ 7 │ +│ 8 │ +│ 9 │ +└───┘ +``` + ## Operators for Working with Dates and Times {#operators-datetime} ### EXTRACT {#operator-extract} diff --git a/docs/en/sql-reference/statements/select/except.md b/docs/en/sql-reference/statements/select/except.md new file mode 100644 index 00000000000..ffbc0fae3da --- /dev/null +++ b/docs/en/sql-reference/statements/select/except.md @@ -0,0 +1,86 @@ +--- +toc_title: EXCEPT +--- + +# EXCEPT Clause {#except-clause} + +The `EXCEPT` clause returns only those rows that result from the first query without the second. The queries must match the number of columns, order, and type. The result of `EXCEPT` can contain duplicate rows. + +Multiple `EXCEPT` statements execute left to right if brackets are not specified. The `EXCEPT` operator has the same priority as the `UNION` clause and lower priority as the `INTERSECT` clause. + +``` sql +SELECT column1 [, column2 ] +FROM table1 +[WHERE condition] + +EXCEPT + +SELECT column1 [, column2 ] +FROM table2 +[WHERE condition] + +``` +The condition could be any expression based on your requirements. + +**Example** + +Query: + +``` sql +SELECT number FROM numbers(1,10) EXCEPT SELECT number FROM numbers(3,6); +``` + +Result: + +``` text +┌─number─┐ +│ 1 │ +│ 2 │ +│ 9 │ +│ 10 │ +└────────┘ +``` + +Query: + +``` sql +SELECT number FROM numbers(1,10) WHERE number > 5 EXCEPT SELECT number FROM numbers(3,6) WHERE number < 7; +``` + +Result: + +``` text +┌─number─┐ +│ 7 │ +│ 8 │ +│ 9 │ +│ 10 │ +└────────┘ +``` + +Query: + +``` sql +CREATE TABLE t1(one String, two String, three String) ENGINE=Memory(); +CREATE TABLE t2(four String, five String, six String) ENGINE=Memory(); + +INSERT INTO t1 VALUES ('q', 'm', 'b'), ('s', 'd', 'f'), ('l', 'p', 'o'), ('s', 'd', 'f'), ('s', 'd', 'f'), ('k', 't', 'd'), ('l', 'p', 'o'); +INSERT INTO t2 VALUES ('q', 'm', 'b'), ('b', 'd', 'k'), ('s', 'y', 't'), ('s', 'd', 'f'), ('m', 'f', 'o'), ('k', 'k', 'd'); + +SELECT * FROM t1 EXCEPT SELECT * FROM t2; +``` + +Result: + +``` text +┌─one─┬─two─┬─three─┐ +│ l │ p │ o │ +│ k │ t │ d │ +│ l │ p │ o │ +└─────┴─────┴───────┘ +``` + +**See Also** + +- [UNION](union.md#union-clause) +- [INTERSECT](except.md#except-clause) \ No newline at end of file diff --git a/docs/en/sql-reference/statements/select/index.md b/docs/en/sql-reference/statements/select/index.md index 0427764475a..39d12f880e4 100644 --- a/docs/en/sql-reference/statements/select/index.md +++ b/docs/en/sql-reference/statements/select/index.md @@ -49,6 +49,8 @@ Specifics of each optional clause are covered in separate sections, which are li - [LIMIT clause](../../../sql-reference/statements/select/limit.md) - [OFFSET clause](../../../sql-reference/statements/select/offset.md) - [UNION clause](../../../sql-reference/statements/select/union.md) +- [INTERSECT clause](../../../sql-reference/statements/select/intersect.md) +- [EXCEPT clause](../../../sql-reference/statements/select/except.md) - [INTO OUTFILE clause](../../../sql-reference/statements/select/into-outfile.md) - [FORMAT clause](../../../sql-reference/statements/select/format.md) diff --git a/docs/en/sql-reference/statements/select/intersect.md b/docs/en/sql-reference/statements/select/intersect.md new file mode 100644 index 00000000000..89446c68e62 --- /dev/null +++ b/docs/en/sql-reference/statements/select/intersect.md @@ -0,0 +1,87 @@ +--- +toc_title: INTERSECT +--- + +# INTERSECT Clause {#intersect-clause} + +The `INTERSECT` clause returns only those rows that result from first and second queries. The queries must match the number of columns, order, type, and names. The result of `INTERSECT` can contain duplicate rows. + +Multiple `INTERSECT` statements execute left to right if brackets are not specified. The `INTERSECT` operator has a higher priority than the `UNION` and `EXCEPT` clause. + + +``` sql +SELECT column1 [, column2 ] +FROM table1 +[WHERE condition] + +INTERSECT + +SELECT column1 [, column2 ] +FROM table2 +[WHERE condition] + +``` +The condition could be any expression based on your requirements. + +**Example** + +Query: + +``` sql +SELECT number FROM numbers(1,10) INTERSECT SELECT number FROM numbers(3,6); +``` + +Result: + +``` text +┌─number─┐ +│ 3 │ +│ 4 │ +│ 5 │ +│ 6 │ +│ 7 │ +│ 8 │ +└────────┘ +``` + +Query: + +``` sql +SELECT number FROM numbers(1,10) WHERE number > 5 INTERSECT SELECT number FROM numbers(3,6) WHERE number < 7; +``` + +Result: + +``` text +┌─number─┐ +│ 6 │ +└────────┘ +``` + +Query: + +``` sql +CREATE TABLE t1(one String, two String, three String) ENGINE=Memory(); +CREATE TABLE t2(four String, five String, six String) ENGINE=Memory(); + +INSERT INTO t1 VALUES ('q', 'm', 'b'), ('s', 'd', 'f'), ('l', 'p', 'o'), ('s', 'd', 'f'), ('s', 'd', 'f'), ('k', 't', 'd'), ('l', 'p', 'o'); +INSERT INTO t2 VALUES ('q', 'm', 'b'), ('b', 'd', 'k'), ('s', 'y', 't'), ('s', 'd', 'f'), ('m', 'f', 'o'), ('k', 'k', 'd'); + +SELECT * FROM t1 INTERSECT SELECT * FROM t2; +``` + +Result: + +``` text +┌─one─┬─two─┬─three─┐ +│ q │ m │ b │ +│ s │ d │ f │ +│ s │ d │ f │ +│ s │ d │ f │ +└─────┴─────┴───────┘ +``` + +**See Also** + +- [UNION](union.md#union-clause) +- [EXCEPT](except.md#except-clause) \ No newline at end of file From b0752b43ac9529b4f744af72e32806c4240824b4 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 20 Oct 2021 20:20:14 +0300 Subject: [PATCH 271/919] Add the getOSKernelVersion function MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Задокументировал английскую версию функции getOSKernelVersion. --- .../functions/other-functions.md | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 6864ba7705b..a7f81ab8325 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2427,3 +2427,39 @@ Type: [UInt32](../../sql-reference/data-types/int-uint.md). **See Also** - [shardNum()](#shard-num) function example also contains `shardCount()` function call. + +## getOSKernelVersion {#getoskernelversion} + +Returns string with the current OS kernel version. + +**Syntax** + +``` sql +getOSKernelVersion() +``` + +**Arguments** + +- None. + +**Returned value** + +- The current OS kernel version. + +Type: [String](../../sql-reference/data-types/string.md). + +**Example** + +Query: + +``` sql +SELECT getOSKernelVersion(); +``` + +Result: + +``` text +┌─getOSKernelVersion()────┐ +│ Linux 4.15.0-55-generic │ +└─────────────────────────┘ +``` From e36f06887bcccc1bb2bfc29844415b279987da21 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Wed, 20 Oct 2021 20:43:07 +0300 Subject: [PATCH 272/919] update --- docs/en/sql-reference/operators/index.md | 2 +- docs/en/sql-reference/statements/select/intersect.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index 159fc692e94..7cc7a48cd77 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -94,7 +94,7 @@ Result: └───┘ ``` -`ANY` – The operator is comparing a value to a set of values returned by a subquery. The comparison condition must be running for any values of the subquery. +`ANY` – The operator is comparing a value to a set of values returned by a subquery. The comparison condition can be running for any values of the subquery. The subquery must select values of the same type as those compared in the main predicate. diff --git a/docs/en/sql-reference/statements/select/intersect.md b/docs/en/sql-reference/statements/select/intersect.md index 89446c68e62..95af5eb50a0 100644 --- a/docs/en/sql-reference/statements/select/intersect.md +++ b/docs/en/sql-reference/statements/select/intersect.md @@ -4,7 +4,7 @@ toc_title: INTERSECT # INTERSECT Clause {#intersect-clause} -The `INTERSECT` clause returns only those rows that result from first and second queries. The queries must match the number of columns, order, type, and names. The result of `INTERSECT` can contain duplicate rows. +The `INTERSECT` clause returns only those rows that result from first and second queries. The queries must match the number of columns, order, and type. The result of `INTERSECT` can contain duplicate rows. Multiple `INTERSECT` statements execute left to right if brackets are not specified. The `INTERSECT` operator has a higher priority than the `UNION` and `EXCEPT` clause. From 9d74ae16df157178ec7f1698d652de9cf4fa6f23 Mon Sep 17 00:00:00 2001 From: Alexey Date: Wed, 20 Oct 2021 20:01:57 +0000 Subject: [PATCH 273/919] en updated --- docs/en/operations/settings/settings.md | 9 ++++++--- .../statements/describe-table.md | 19 ++++++++++--------- 2 files changed, 16 insertions(+), 12 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ab4f3506630..ee16e820913 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3817,12 +3817,15 @@ Default value: `0`. ## describe_include_subcolumns {#describe_include_subcolumns} -Enables describing subcolumns for a [DESCRIBE](../../sql-reference/statements/describe-table.md) query. Subcolumns appear if [Tuple](../../sql-reference/data-types/tuple.md) or [Map](../../sql-reference/data-types/map.md#map-subcolumns) data types are used. +Enables describing subcolumns for a [DESCRIBE](../../sql-reference/statements/describe-table.md) query. Subcolumns appear in columns of [Tuple](../../sql-reference/data-types/tuple.md) or [Map](../../sql-reference/data-types/map.md#map-subcolumns) data types. Possible values: -- 0 — Subcolumns are not included in DESCRIBE queries. -- 1 — Subcolumns are included in DESCRIBE queries. +- 0 — Subcolumns are not included in `DESCRIBE` queries. +- 1 — Subcolumns are included in `DESCRIBE` queries. Default value: `0`. +**Example** + +See an example for the [DESCRIBE](../../sql-reference/statements/describe-table.md) statement. \ No newline at end of file diff --git a/docs/en/sql-reference/statements/describe-table.md b/docs/en/sql-reference/statements/describe-table.md index 8e53cbc982b..1fd1befed53 100644 --- a/docs/en/sql-reference/statements/describe-table.md +++ b/docs/en/sql-reference/statements/describe-table.md @@ -15,17 +15,18 @@ DESC|DESCRIBE TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] The `DESCRIBE` statement returns a row for each table column with the following [String](../../sql-reference/data-types/string.md) values: -- `name` — a column name. -- `type` — a column type. -- `default_type` — a clause that is used in the column [default expression](../../sql-reference/statements/create/table.md#create-default-values): `DEFAULT`, `MATERIALIZED` or `ALIAS`. If there is no default expression, then empty string is returned. +- `name` — A column name. +- `type` — A column type. +- `default_type` — A clause that is used in the column [default expression](../../sql-reference/statements/create/table.md#create-default-values): `DEFAULT`, `MATERIALIZED` or `ALIAS`. If there is no default expression, then empty string is returned. - `default_expression` — an expression specified after the `DEFAULT` clause. -- `comment` — a comment. -- `codec_expression` - a [codec](../../sql-reference/statements/create/table.md#codecs) that is applied to the column. -- `ttl_expression` - a [TTL](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl) expression. -- `is_subcolumn` - a flag that is set to `1` for internal subcolumns. It is included into an output if subcolumn description is enabled. +- `comment` — A [column comment](../../sql-reference/statements/alter/column.md#alter_comment-column). +- `codec_expression` - A [codec](../../sql-reference/statements/create/table.md#codecs) that is applied to the column. +- `ttl_expression` - A [TTL](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl) expression. +- `is_subcolumn` - A flag that equals `1` for internal subcolumns. It is included into the result if subcolumn description is enabled. -[Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns are described separately. The name of each nested column is prefixed with a parent column name and a dot. -To enable internal subcolumn description, use the [describe_include_subcolumns](../../operations/settings/settings.md#describe_include_subcolumns) setting. +All columns in a [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) data structure are described separately. The name of each column is prefixed with a parent column name and a dot. + +To include internal subcolumns into the result, use the [describe_include_subcolumns](../../operations/settings/settings.md#describe_include_subcolumns) setting. **Example** From e4e157688d2ac6cd36e80afdb302bca6b588f786 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 21 Oct 2021 01:57:43 +0300 Subject: [PATCH 274/919] Review fixes --- src/Disks/HDFS/DiskHDFS.cpp | 3 +- ...chronousReadIndirectBufferFromRemoteFS.cpp | 93 +++++++++++-------- ...ynchronousReadIndirectBufferFromRemoteFS.h | 8 +- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 17 +--- src/Disks/IO/ReadBufferFromRemoteFSGather.h | 14 +-- ...ynchronousReadBufferFromFileDescriptor.cpp | 46 +++++---- ...AsynchronousReadBufferFromFileDescriptor.h | 2 + src/IO/ReadBufferFromS3.cpp | 22 ++--- src/IO/ReadBufferFromS3.h | 4 +- src/Storages/HDFS/ReadBufferFromHDFS.cpp | 27 +++++- src/Storages/HDFS/ReadBufferFromHDFS.h | 2 +- src/Storages/MergeTree/IMergeTreeReader.h | 3 +- .../MergeTree/MergeTreeIndexReader.cpp | 10 +- .../MergeTree/MergeTreeRangeReader.cpp | 20 +--- src/Storages/MergeTree/MergeTreeReadPool.cpp | 16 +--- src/Storages/MergeTree/MergeTreeReadPool.h | 2 +- 16 files changed, 150 insertions(+), 139 deletions(-) diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index 2867fc5de34..b1f4db28f4f 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -80,8 +80,7 @@ std::unique_ptr DiskHDFS::readFile(const String & path, if (read_settings.remote_fs_method == RemoteFSReadMethod::read_threadpool) { auto reader = getThreadPoolReader(); - auto buf = std::make_unique(reader, read_settings.priority, std::move(hdfs_impl)); - return std::make_unique(std::move(buf), settings->min_bytes_for_seek); + return std::make_unique(reader, read_settings.priority, std::move(hdfs_impl)); } else { diff --git a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp index 8c41cd3df39..eec9c2ed157 100644 --- a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -56,13 +56,41 @@ String AsynchronousReadIndirectBufferFromRemoteFS::getFileName() const } +size_t AsynchronousReadIndirectBufferFromRemoteFS::getNumBytesToRead() +{ + size_t num_bytes_to_read; + + /// Position is set only for MergeTree tables. + if (read_until_position) + { + /// Everything is already read. + if (file_offset_of_buffer_end == *read_until_position) + return 0; + + if (file_offset_of_buffer_end > *read_until_position) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Read beyond last offset ({} > {})", + file_offset_of_buffer_end, *read_until_position); + + /// Read range [file_offset_of_buffer_end, read_until_position). + num_bytes_to_read = *read_until_position - file_offset_of_buffer_end; + num_bytes_to_read = std::min(num_bytes_to_read, internal_buffer.size()); + } + else + { + num_bytes_to_read = internal_buffer.size(); + } + + return num_bytes_to_read; +} + + std::future AsynchronousReadIndirectBufferFromRemoteFS::readInto(char * data, size_t size) { IAsynchronousReader::Request request; request.descriptor = std::make_shared(impl); request.buf = data; request.size = size; - request.offset = absolute_position; + request.offset = file_offset_of_buffer_end; request.priority = priority; if (bytes_to_ignore) @@ -79,17 +107,12 @@ void AsynchronousReadIndirectBufferFromRemoteFS::prefetch() if (prefetch_future.valid()) return; - /// Everything is already read. - if (absolute_position == last_offset) + auto num_bytes_to_read = getNumBytesToRead(); + if (!num_bytes_to_read) return; - if (absolute_position > last_offset) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Read beyond last offset ({} > {})", - absolute_position, last_offset); - } - /// Prefetch even in case hasPendingData() == true. + prefetch_buffer.resize(num_bytes_to_read); prefetch_future = readInto(prefetch_buffer.data(), prefetch_buffer.size()); ProfileEvents::increment(ProfileEvents::RemoteFSPrefetches); } @@ -98,27 +121,15 @@ void AsynchronousReadIndirectBufferFromRemoteFS::prefetch() void AsynchronousReadIndirectBufferFromRemoteFS::setReadUntilPosition(size_t position) { if (prefetch_future.valid()) - { - /// TODO: Planning to put logical error here after more testing, - // because seems like future is never supposed to be valid at this point. - std::terminate(); - } + throw Exception(ErrorCodes::LOGICAL_ERROR, "Prefetch is valid in readUntilPosition"); - last_offset = position; + read_until_position = position; impl->setReadUntilPosition(position); } bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() { - /// Everything is already read. - if (absolute_position == last_offset) - return false; - - if (absolute_position > last_offset) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Read beyond last offset ({} > {})", - absolute_position, last_offset); - size_t size = 0; if (prefetch_future.valid()) @@ -134,7 +145,7 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() memory.swap(prefetch_buffer); set(memory.data(), memory.size()); working_buffer.resize(size); - absolute_position += size; + file_offset_of_buffer_end += size; } } @@ -143,14 +154,18 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() } else { + auto num_bytes_to_read = getNumBytesToRead(); + if (!num_bytes_to_read) /// Nothing to read. + return false; + ProfileEvents::increment(ProfileEvents::RemoteFSUnprefetchedReads); - size = readInto(memory.data(), memory.size()).get(); + size = readInto(memory.data(), num_bytes_to_read).get(); if (size) { set(memory.data(), memory.size()); working_buffer.resize(size); - absolute_position += size; + file_offset_of_buffer_end += size; } } @@ -166,24 +181,24 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence if (whence == SEEK_CUR) { /// If position within current working buffer - shift pos. - if (!working_buffer.empty() && static_cast(getPosition() + offset_) < absolute_position) + if (!working_buffer.empty() && static_cast(getPosition() + offset_) < file_offset_of_buffer_end) { pos += offset_; return getPosition(); } else { - absolute_position += offset_; + file_offset_of_buffer_end += offset_; } } else if (whence == SEEK_SET) { /// If position is within current working buffer - shift pos. if (!working_buffer.empty() - && static_cast(offset_) >= absolute_position - working_buffer.size() - && size_t(offset_) < absolute_position) + && static_cast(offset_) >= file_offset_of_buffer_end - working_buffer.size() + && size_t(offset_) < file_offset_of_buffer_end) { - pos = working_buffer.end() - (absolute_position - offset_); + pos = working_buffer.end() - (file_offset_of_buffer_end - offset_); assert(pos >= working_buffer.begin()); assert(pos <= working_buffer.end()); @@ -192,7 +207,7 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence } else { - absolute_position = offset_; + file_offset_of_buffer_end = offset_; } } else @@ -207,22 +222,22 @@ off_t AsynchronousReadIndirectBufferFromRemoteFS::seek(off_t offset_, int whence pos = working_buffer.end(); - /// Note: we read in range [absolute_position, last_offset). - if (absolute_position < last_offset - && static_cast(absolute_position) >= getPosition() - && static_cast(absolute_position) < getPosition() + static_cast(min_bytes_for_seek)) + /// Note: we read in range [file_offset_of_buffer_end, read_until_position). + if (file_offset_of_buffer_end < read_until_position + && static_cast(file_offset_of_buffer_end) >= getPosition() + && static_cast(file_offset_of_buffer_end) < getPosition() + static_cast(min_bytes_for_seek)) { /** * Lazy ignore. Save number of bytes to ignore and ignore it either for prefetch buffer or current buffer. */ - bytes_to_ignore = absolute_position - getPosition(); + bytes_to_ignore = file_offset_of_buffer_end - getPosition(); } else { - impl->seek(absolute_position); /// SEEK_SET. + impl->reset(); } - return absolute_position; + return file_offset_of_buffer_end; } diff --git a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h index e30a21b5dc3..8b25aa94a61 100644 --- a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -41,7 +41,7 @@ public: off_t seek(off_t offset_, int whence) override; - off_t getPosition() override { return absolute_position - available(); } + off_t getPosition() override { return file_offset_of_buffer_end - available(); } String getFileName() const override; @@ -54,6 +54,8 @@ private: void finalize(); + size_t getNumBytesToRead(); + std::future readInto(char * data, size_t size); AsynchronousReaderPtr reader; @@ -64,7 +66,7 @@ private: std::future prefetch_future; - size_t absolute_position = 0; + size_t file_offset_of_buffer_end = 0; Memory<> prefetch_buffer; @@ -72,7 +74,7 @@ private: size_t bytes_to_ignore = 0; - size_t last_offset = 0; + std::optional read_until_position = 0; }; } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 70300563aa2..b2484a7928e 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -86,7 +86,7 @@ void ReadBufferFromRemoteFSGather::initialize() /// Do not create a new buffer if we already have what we need. if (!current_buf || current_buf_idx != i) { - current_buf = createImplementationBuffer(file_path, last_offset); + current_buf = createImplementationBuffer(file_path, read_until_position); current_buf_idx = i; } @@ -123,7 +123,7 @@ bool ReadBufferFromRemoteFSGather::nextImpl() ++current_buf_idx; const auto & current_path = metadata.remote_fs_objects[current_buf_idx].first; - current_buf = createImplementationBuffer(current_path, last_offset); + current_buf = createImplementationBuffer(current_path, read_until_position); return readImpl(); } @@ -141,7 +141,6 @@ bool ReadBufferFromRemoteFSGather::readImpl() if (bytes_to_ignore) current_buf->ignore(bytes_to_ignore); - LOG_DEBUG(&Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Reading from path: {}", canonical_path); auto result = current_buf->next(); swap(*current_buf); @@ -153,18 +152,10 @@ bool ReadBufferFromRemoteFSGather::readImpl() } -void ReadBufferFromRemoteFSGather::seek(off_t offset) -{ - current_buf.reset(); - absolute_position = offset; -} - - void ReadBufferFromRemoteFSGather::setReadUntilPosition(size_t position) { - assert(last_offset < position); - current_buf.reset(); - last_offset = position; + read_until_position = position; + reset(); } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 8012676d5a8..a3ca8a081f5 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -19,6 +19,10 @@ class S3Client; namespace DB { +/** + * Remote disk might need to split one clickhouse file into multiple files in remote fs. + * This class works like a proxy to allow transition from one file into multiple. + */ class ReadBufferFromRemoteFSGather : public ReadBuffer { friend class ReadIndirectBufferFromRemoteFS; @@ -30,8 +34,6 @@ public: void reset(); - void seek(off_t offset); /// SEEK_SET only. - void setReadUntilPosition(size_t position) override; size_t readInto(char * data, size_t size, size_t offset, size_t ignore = 0); @@ -56,7 +58,7 @@ private: size_t bytes_to_ignore = 0; - size_t last_offset = 0; + size_t read_until_position = 0; String canonical_path; }; @@ -84,7 +86,7 @@ public: { } - SeekableReadBufferPtr createImplementationBuffer(const String & path, size_t last_offset) const override; + SeekableReadBufferPtr createImplementationBuffer(const String & path, size_t read_until_position) const override; private: std::shared_ptr client_ptr; @@ -114,7 +116,7 @@ public: { } - SeekableReadBufferPtr createImplementationBuffer(const String & path, size_t last_offset) const override; + SeekableReadBufferPtr createImplementationBuffer(const String & path, size_t read_until_position) const override; private: String uri; @@ -144,7 +146,7 @@ public: hdfs_uri = hdfs_uri_.substr(0, begin_of_path); } - SeekableReadBufferPtr createImplementationBuffer(const String & path, size_t last_offset) const override; + SeekableReadBufferPtr createImplementationBuffer(const String & path, size_t read_until_position) const override; private: const Poco::Util::AbstractConfiguration & config; diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp index 0d1e32bb2b5..8dbdc85a330 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp @@ -49,31 +49,44 @@ std::future AsynchronousReadBufferFromFileDescripto } -void AsynchronousReadBufferFromFileDescriptor::prefetch() +size_t AsynchronousReadBufferFromFileDescriptor::getNumBytesToRead() { - if (prefetch_future.valid()) - return; + size_t num_bytes_to_read; - size_t read_size; + /// Position is set only for MergeTree tables. if (read_until_position) { /// Everything is already read. if (file_offset_of_buffer_end == *read_until_position) - return; + return 0; if (file_offset_of_buffer_end > *read_until_position) throw Exception(ErrorCodes::LOGICAL_ERROR, "Read beyond last offset ({} > {})", file_offset_of_buffer_end, *read_until_position); /// Read range [file_offset_of_buffer_end, read_until_position). - read_size = *read_until_position - file_offset_of_buffer_end - 1; + num_bytes_to_read = *read_until_position - file_offset_of_buffer_end; + num_bytes_to_read = std::min(num_bytes_to_read, internal_buffer.size()); } else { - read_size = internal_buffer.size(); + num_bytes_to_read = internal_buffer.size(); } - prefetch_buffer.resize(read_size); + return num_bytes_to_read; +} + + +void AsynchronousReadBufferFromFileDescriptor::prefetch() +{ + if (prefetch_future.valid()) + return; + + auto num_bytes_to_read = getNumBytesToRead(); + if (!num_bytes_to_read) + return; + + prefetch_buffer.resize(num_bytes_to_read); prefetch_future = readInto(prefetch_buffer.data(), prefetch_buffer.size()); } @@ -89,17 +102,6 @@ void AsynchronousReadBufferFromFileDescriptor::setReadUntilPosition(size_t posit bool AsynchronousReadBufferFromFileDescriptor::nextImpl() { - if (read_until_position) - { - /// Everything is already read. - if (file_offset_of_buffer_end == *read_until_position) - return false; - - if (file_offset_of_buffer_end > *read_until_position) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Read beyond last offset ({} > {})", - file_offset_of_buffer_end, *read_until_position); - } - if (prefetch_future.valid()) { /// Read request already in flight. Wait for its completion. @@ -127,9 +129,13 @@ bool AsynchronousReadBufferFromFileDescriptor::nextImpl() } else { + auto num_bytes_to_read = getNumBytesToRead(); + if (!num_bytes_to_read) /// Nothing to read. + return false; + /// No pending request. Do synchronous read. - auto size = readInto(memory.data(), memory.size()).get(); + auto size = readInto(memory.data(), num_bytes_to_read).get(); file_offset_of_buffer_end += size; if (size) diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.h b/src/IO/AsynchronousReadBufferFromFileDescriptor.h index 271518dd1eb..12e34cf784e 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.h +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.h @@ -67,6 +67,8 @@ public: private: std::future readInto(char * data, size_t size); + + size_t getNumBytesToRead(); }; } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index bf578373535..5c2b9aeff44 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -35,7 +35,7 @@ namespace ErrorCodes ReadBufferFromS3::ReadBufferFromS3( std::shared_ptr client_ptr_, const String & bucket_, const String & key_, - UInt64 max_single_read_retries_, const ReadSettings & settings_, bool use_external_buffer_, size_t last_offset_) + UInt64 max_single_read_retries_, const ReadSettings & settings_, bool use_external_buffer_, size_t read_until_position_) : SeekableReadBuffer(nullptr, 0) , client_ptr(std::move(client_ptr_)) , bucket(bucket_) @@ -43,19 +43,19 @@ ReadBufferFromS3::ReadBufferFromS3( , max_single_read_retries(max_single_read_retries_) , read_settings(settings_) , use_external_buffer(use_external_buffer_) - , last_offset(last_offset_) + , read_until_position(read_until_position_) { } bool ReadBufferFromS3::nextImpl() { - if (last_offset) + if (read_until_position) { - if (last_offset == offset) + if (read_until_position == offset) return false; - if (last_offset < offset) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, last_offset - 1); + if (read_until_position < offset) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, read_until_position - 1); } bool next_result = false; @@ -171,13 +171,13 @@ std::unique_ptr ReadBufferFromS3::initialize() req.SetBucket(bucket); req.SetKey(key); - if (last_offset) + if (read_until_position) { - if (offset >= last_offset) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, last_offset - 1); + if (offset >= read_until_position) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, read_until_position - 1); - req.SetRange(fmt::format("bytes={}-{}", offset, last_offset - 1)); - LOG_DEBUG(log, "Read S3 object. Bucket: {}, Key: {}, Range: {}-{}", bucket, key, offset, last_offset - 1); + req.SetRange(fmt::format("bytes={}-{}", offset, read_until_position - 1)); + LOG_DEBUG(log, "Read S3 object. Bucket: {}, Key: {}, Range: {}-{}", bucket, key, offset, read_until_position - 1); } else { diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index fae3938bf72..2b30a2bf51b 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -46,7 +46,7 @@ public: UInt64 max_single_read_retries_, const ReadSettings & settings_, bool use_external_buffer = false, - size_t last_offset_ = 0); + size_t read_until_position_ = 0); size_t right = 0; bool nextImpl() override; @@ -59,7 +59,7 @@ private: ReadSettings read_settings; bool use_external_buffer; - off_t last_offset; + off_t read_until_position = 0; }; } diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index f24705d7f65..2fc72c58b64 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -15,6 +15,7 @@ namespace ErrorCodes extern const int CANNOT_OPEN_FILE; extern const int CANNOT_SEEK_THROUGH_FILE; extern const int SEEK_POSITION_OUT_OF_BOUND; + extern const int LOGICAL_ERROR; } ReadBufferFromHDFS::~ReadBufferFromHDFS() = default; @@ -33,16 +34,18 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory(buf_size_) , hdfs_uri(hdfs_uri_) , hdfs_file_path(hdfs_file_path_) , builder(createHDFSBuilder(hdfs_uri_, config_)) + , read_until_position(read_until_position_) { std::lock_guard lock(hdfs_init_mutex); @@ -79,7 +82,23 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory {})", offset, read_until_position - 1); + + num_bytes_to_read = read_until_position - offset; + } + else + { + num_bytes_to_read = internal_buffer.size(); + } + + int bytes_read = hdfsRead(fs.get(), fin, internal_buffer.begin(), num_bytes_to_read); if (bytes_read < 0) throw Exception(ErrorCodes::NETWORK_ERROR, "Fail to read from HDFS: {}, file path: {}. Error: {}", @@ -125,9 +144,9 @@ ReadBufferFromHDFS::ReadBufferFromHDFS( const String & hdfs_uri_, const String & hdfs_file_path_, const Poco::Util::AbstractConfiguration & config_, - size_t buf_size_, size_t) + size_t buf_size_, size_t read_until_position_) : SeekableReadBuffer(nullptr, 0) - , impl(std::make_unique(hdfs_uri_, hdfs_file_path_, config_, buf_size_)) + , impl(std::make_unique(hdfs_uri_, hdfs_file_path_, config_, buf_size_, read_until_position_)) { } diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.h b/src/Storages/HDFS/ReadBufferFromHDFS.h index 38c8047ba93..d2cc0fcefb4 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.h +++ b/src/Storages/HDFS/ReadBufferFromHDFS.h @@ -29,7 +29,7 @@ public: ReadBufferFromHDFS(const String & hdfs_uri_, const String & hdfs_file_path_, const Poco::Util::AbstractConfiguration & config_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE, - size_t last_offset = 0); + size_t read_until_position_ = 0); ~ReadBufferFromHDFS() override; diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index bcb51f2fce6..28334b9a8bb 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -30,7 +30,8 @@ public: const ValueSizeMap & avg_value_size_hints_ = ValueSizeMap{}); /// Return the number of rows has been read or zero if there is no columns to read. - /// If continue_reading is true, continue reading from last state, otherwise seek to from_mark + /// If continue_reading is true, continue reading from last state, otherwise seek to from_mark. + /// current_task_last mark is needed for asynchronous reading (mainly from remote fs). virtual size_t readRows(size_t from_mark, size_t current_task_last_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) = 0; diff --git a/src/Storages/MergeTree/MergeTreeIndexReader.cpp b/src/Storages/MergeTree/MergeTreeIndexReader.cpp index 2e83ac8d427..93302a4c900 100644 --- a/src/Storages/MergeTree/MergeTreeIndexReader.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexReader.cpp @@ -54,15 +54,9 @@ MergeTreeIndexReader::MergeTreeIndexReader( std::move(settings)); version = index_format.version; - auto current_task_last_mark_range = std::max_element(all_mark_ranges_.begin(), all_mark_ranges_.end(), - [&](const MarkRange & range1, const MarkRange & range2) - { - return range1.end < range2.end; - }); - size_t current_task_last_mark = 0; - if (current_task_last_mark_range != all_mark_ranges_.end()) - current_task_last_mark = current_task_last_mark_range->end; + for (const auto mark_range : all_mark_ranges_) + current_task_last_mark = std::max(current_task_last_mark, mark_range.end); stream->adjustForRange(0, current_task_last_mark); stream->seekToStart(); diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 6909b3d2575..833a9c3023b 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -754,15 +754,9 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t ReadResult result; result.columns.resize(merge_tree_reader->getColumns().size()); - auto current_task_last_mark_range = std::max_element(ranges.begin(), ranges.end(), - [&](const MarkRange & range1, const MarkRange & range2) - { - return range1.end < range2.end; - }); - size_t current_task_last_mark = 0; - if (current_task_last_mark_range != ranges.end()) - current_task_last_mark = current_task_last_mark_range->end; + for (const auto mark_range : ranges) + current_task_last_mark = std::max(current_task_last_mark, mark_range.end); /// Stream is lazy. result.num_added_rows is the number of rows added to block which is not equal to /// result.num_rows_read until call to stream.finalize(). Also result.num_added_rows may be less than @@ -821,15 +815,9 @@ Columns MergeTreeRangeReader::continueReadingChain(ReadResult & result, size_t & const auto & rows_per_granule = result.rowsPerGranule(); const auto & started_ranges = result.startedRanges(); - auto current_task_last_mark_range = std::max_element(started_ranges.begin(), started_ranges.end(), - [&](const ReadResult::RangeInfo & lhs, const ReadResult::RangeInfo & rhs) - { - return lhs.range.end < rhs.range.end; - }); - size_t current_task_last_mark = 0; - if (current_task_last_mark_range != started_ranges.end()) - current_task_last_mark = current_task_last_mark_range->range.end; + for (const auto mark_range : started_ranges) + current_task_last_mark = std::max(current_task_last_mark, mark_range.range.end); size_t next_range_to_start = 0; diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 326ad843e7e..ab798cae5ee 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -42,13 +42,7 @@ MergeTreeReadPool::MergeTreeReadPool( { /// parts don't contain duplicate MergeTreeDataPart's. const auto per_part_sum_marks = fillPerPartInfo(parts_ranges, check_columns_); - auto min_marks_for_concurrent_read = min_marks_for_concurrent_read_; - if (stored_on_remote_disk) - { - do_not_steal_tasks = true; - min_marks_for_concurrent_read = std::max(min_marks_for_concurrent_read, sum_marks_ / threads_); - } - fillPerThreadInfo(threads_, sum_marks_, per_part_sum_marks, parts_ranges, min_marks_for_concurrent_read); + fillPerThreadInfo(threads_, sum_marks_, per_part_sum_marks, parts_ranges, min_marks_for_concurrent_read_); } @@ -96,7 +90,7 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(const size_t min_marks_to_read, auto & marks_in_part = thread_tasks.sum_marks_in_parts.back(); size_t need_marks; - if (stored_on_remote_disk) /// For better performance with remote disks + if (is_part_on_remote_disk[part_idx]) /// For better performance with remote disks need_marks = marks_in_part; else /// Get whole part to read if it is small enough. need_marks = std::min(marks_in_part, min_marks_to_read); @@ -201,14 +195,12 @@ std::vector MergeTreeReadPool::fillPerPartInfo( { std::vector per_part_sum_marks; Block sample_block = metadata_snapshot->getSampleBlock(); + is_part_on_remote_disk.resize(parts.size()); for (const auto i : collections::range(0, parts.size())) { const auto & part = parts[i]; - - /// Turn off tasks stealing in case there is remote disk. - if (part.data_part->isStoredOnRemoteDisk()) - stored_on_remote_disk = true; + is_part_on_remote_disk[i] = part.data_part->isStoredOnRemoteDisk(); /// Read marks for every data part. size_t sum_marks = 0; diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index 6cdf249345f..cf5d031e683 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -137,7 +137,7 @@ private: Poco::Logger * log = &Poco::Logger::get("MergeTreeReadPool"); - bool stored_on_remote_disk = false; + std::vector is_part_on_remote_disk; }; using MergeTreeReadPoolPtr = std::shared_ptr; From 5845336acb8b1bbf511ace67e2bcc5dd6d82c8d4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 20 Oct 2021 23:19:36 +0000 Subject: [PATCH 275/919] Adjust tests --- ...ranularity_collapsing_merge_tree.reference | 8 +- ...ive_index_granularity_merge_tree.reference | 86 +++++++++---------- ...granularity_replacing_merge_tree.reference | 24 +++--- ..._versioned_collapsing_merge_tree.reference | 12 +-- .../01221_system_settings.reference | 2 +- .../01710_minmax_count_projection.sql | 2 +- 6 files changed, 67 insertions(+), 67 deletions(-) diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.reference b/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.reference index 3a176a17f5a..26be1c6ee8a 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.reference +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_collapsing_merge_tree.reference @@ -1,11 +1,11 @@ 4 -4 -8 +5 8 +9 ----- 4 -1 +2 0 0 6 -2 +3 diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.reference b/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.reference index 3e002b3899c..1a68af74960 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.reference +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_merge_tree.reference @@ -1,66 +1,66 @@ ----HORIZONTAL MERGE TESTS---- 4 +5 +8 +9 +----- 4 +3 8 -8 +5 ----- 4 2 8 -4 ------ -4 -1 -8 -2 ------ -4 -1 -8 -1 ------ -4 -1 -4 -1 -8 -1 -42 42 -100 aaaa -101 bbbb -102 cccc -103 dddd -2 -----VERTICAL MERGE TESTS---- -4 -4 -8 -8 +3 ----- 4 2 8 -4 ------ -4 -1 -8 2 ----- 4 -1 +2 +4 +2 8 2 ------ -4 -1 -4 -1 -8 -1 42 42 100 aaaa 101 bbbb 102 cccc 103 dddd 3 +----VERTICAL MERGE TESTS---- +4 +5 +8 +9 +----- +4 +3 +8 +5 +----- +4 +2 +8 +3 +----- +4 +2 +8 +3 +----- +4 +2 +4 +2 +8 +2 +42 42 +100 aaaa +101 bbbb +102 cccc +103 dddd +4 diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_replacing_merge_tree.reference b/tests/queries/0_stateless/00926_adaptive_index_granularity_replacing_merge_tree.reference index 5d5f3af28ab..b4ef1031186 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_replacing_merge_tree.reference +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_replacing_merge_tree.reference @@ -1,32 +1,32 @@ 4 +5 +8 +9 +----- 4 +3 8 -8 +5 ----- 4 2 8 -4 +3 ----- 4 -1 -8 2 +8 +3 ----- 4 -1 -8 2 ------ 4 -1 -4 -1 -8 2 +8 +3 42 42 100 aaaa 101 bbbb 102 cccc 103 dddd -3 +4 diff --git a/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.reference b/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.reference index f93aae0225a..9b2c420f3d9 100644 --- a/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.reference +++ b/tests/queries/0_stateless/00926_adaptive_index_granularity_versioned_collapsing_merge_tree.reference @@ -1,14 +1,14 @@ 4 -4 -8 +5 8 +9 ----- 4 -1 -0 -6 2 ------ +0 6 3 +----- +6 +4 0 diff --git a/tests/queries/0_stateless/01221_system_settings.reference b/tests/queries/0_stateless/01221_system_settings.reference index 108d48d6051..10f29ea75d1 100644 --- a/tests/queries/0_stateless/01221_system_settings.reference +++ b/tests/queries/0_stateless/01221_system_settings.reference @@ -1,4 +1,4 @@ send_timeout 300 0 \N \N 0 Seconds -storage_policy default 0 Name of storage disk policy String +write_final_mark 1 0 Deprecated setting Bool 1 1 diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.sql b/tests/queries/0_stateless/01710_minmax_count_projection.sql index 112487b219e..246caa2a962 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.sql +++ b/tests/queries/0_stateless/01710_minmax_count_projection.sql @@ -38,7 +38,7 @@ select min(j) from no_final_mark; select min(j) from has_final_mark; select min(j) from mixed_final_mark; -select min(j), max(j) from no_final_mark; -- {serverError TOO_MANY_ROWS} +-- select min(j), max(j) from no_final_mark; -- {serverError TOO_MANY_ROWS} select min(j), max(j) from has_final_mark; set max_rows_to_read = 5001; -- one normal part 5000 + one minmax_count_projection part 1 From a223526f451d2bdb32041a137ca9f5bbf31dcc28 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 20 Oct 2021 23:29:35 +0000 Subject: [PATCH 276/919] Fix --- src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 5ac4180ec27..2adb359eae5 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -216,7 +216,7 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( "SELECT attname AS name, format_type(atttypid, atttypmod) AS type, " "attnotnull AS not_null, attndims AS dims " "FROM pg_attribute " - "WHERE attrelid = {}::regclass " + "WHERE attrelid = (SELECT oid FROM pg_class WHERE relname = {})" "AND NOT attisdropped AND attnum > 0", quoteString(postgres_table_name)); table.columns = readNamesAndTypesList(tx, postgres_table_name, query, use_nulls, false); From bc4c2f236696b4b4c80db27624883724bc71def5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 20 Oct 2021 23:40:21 +0000 Subject: [PATCH 277/919] Fix clang-tidy --- 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 28bb936fbb7..68951f5ed68 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -140,7 +140,7 @@ bool checkPositionalArguments(ASTPtr & argument, const ASTSelectQuery * select_q else positional = false; } - else if (auto * ast_function = typeid_cast(argument.get())) + else if (const auto * ast_function = typeid_cast(argument.get())) { if (ast_function->arguments) { From e679e952c89e1af257a2fecd0a1e960ef1b67401 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 21 Oct 2021 10:16:11 +0800 Subject: [PATCH 278/919] Let's check without projection just in case --- tests/queries/0_stateless/01710_projection_with_joins.sql | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01710_projection_with_joins.sql b/tests/queries/0_stateless/01710_projection_with_joins.sql index 97dc396f362..fcd1c586fa3 100644 --- a/tests/queries/0_stateless/01710_projection_with_joins.sql +++ b/tests/queries/0_stateless/01710_projection_with_joins.sql @@ -2,5 +2,7 @@ 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; -set allow_experimental_projection_optimization=1; -select s from t join (select toUInt16(1) as s) x using (s); +select s from t join (select toUInt16(1) as s) x using (s) settings allow_experimental_projection_optimization = 1; +select s from t join (select toUInt16(1) as s) x using (s) settings allow_experimental_projection_optimization = 0; + +drop table t; From cd17643eb87ec7b49e351d063088d395cfd25a67 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 21 Oct 2021 09:22:42 +0300 Subject: [PATCH 279/919] Fix tests --- tests/integration/test_storage_kafka/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 6106966e5b7..14db82a09e9 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2852,6 +2852,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster): ], 'expected':'{"raw_message":"0502696407626C6F636B4E6F0476616C310476616C320476616C3305496E74363406537472696E6706537472696E6707466C6F617433320555496E743800000000000000000342414402414D0000003F01","error":"Cannot read all data. Bytes read: 9. Bytes expected: 65.: (at row 1)\\n"}', 'printable':False, + 'format_settings':'input_format_with_types_use_header=0', }, 'ORC': { 'data_sample': [ @@ -2879,6 +2880,9 @@ def test_kafka_formats_with_broken_message(kafka_cluster): if format_opts.get('printable', False) == False: raw_message = 'hex(_raw_message)' kafka_produce(kafka_cluster, topic_name, data_prefix + data_sample) + format_settings = '' + if format_opts.get('format_settings'): + format_settings = 'SETTINGS ' + format_opts.get('format_settings') instance.query(''' DROP TABLE IF EXISTS test.kafka_{format_name}; @@ -2904,9 +2908,9 @@ def test_kafka_formats_with_broken_message(kafka_cluster): DROP TABLE IF EXISTS test.kafka_errors_{format_name}_mv; CREATE MATERIALIZED VIEW test.kafka_errors_{format_name}_mv Engine=Log AS SELECT {raw_message} as raw_message, _error as error, _topic as topic, _partition as partition, _offset as offset FROM test.kafka_{format_name} - WHERE length(_error) > 0; + WHERE length(_error) > 0 {format_settings}; '''.format(topic_name=topic_name, format_name=format_name, raw_message=raw_message, - extra_settings=format_opts.get('extra_settings') or '')) + extra_settings=format_opts.get('extra_settings') or '', format_settings=format_settings) for format_name, format_opts in list(all_formats.items()): logging.debug('Checking {format_name}') From 650a79a90771d64b6e70ac8708316923a5fa60aa Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 21 Oct 2021 10:08:08 +0300 Subject: [PATCH 280/919] SQLUserDefinedFunctions composition fix --- src/Interpreters/UserDefinedSQLFunctionVisitor.cpp | 5 ++++- src/Parsers/ASTFunction.cpp | 8 ++++++-- ..._sql_user_defined_functions_composition.reference | 2 ++ .../02103_sql_user_defined_functions_composition.sql | 12 ++++++++++++ 4 files changed, 24 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02103_sql_user_defined_functions_composition.reference create mode 100644 tests/queries/0_stateless/02103_sql_user_defined_functions_composition.sql diff --git a/src/Interpreters/UserDefinedSQLFunctionVisitor.cpp b/src/Interpreters/UserDefinedSQLFunctionVisitor.cpp index 8df6932b153..d9ac53097ab 100644 --- a/src/Interpreters/UserDefinedSQLFunctionVisitor.cpp +++ b/src/Interpreters/UserDefinedSQLFunctionVisitor.cpp @@ -18,7 +18,7 @@ namespace ErrorCodes extern const int UNSUPPORTED_METHOD; } -void UserDefinedSQLFunctionMatcher::visit(ASTPtr & ast, Data &) +void UserDefinedSQLFunctionMatcher::visit(ASTPtr & ast, Data & data) { auto * function = ast->as(); if (!function) @@ -27,7 +27,10 @@ void UserDefinedSQLFunctionMatcher::visit(ASTPtr & ast, Data &) auto result = tryToReplaceFunction(*function); if (result) + { ast = result; + visit(ast, data); + } } bool UserDefinedSQLFunctionMatcher::needChildVisit(const ASTPtr &, const ASTPtr &) diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 3c78c4060d6..093baca375c 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -440,6 +440,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format if (!written && 0 == strcmp(name.c_str(), "lambda")) { + /// Special case: zero elements tuple in lhs of lambda is printed as (). /// Special case: one-element tuple in lhs of lambda is printed as its element. if (frame.need_parens) @@ -449,9 +450,12 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format if (first_arg_func && first_arg_func->name == "tuple" && first_arg_func->arguments - && first_arg_func->arguments->children.size() == 1) + && (first_arg_func->arguments->children.size() == 1 || first_arg_func->arguments->children.size() == 0)) { - first_arg_func->arguments->children[0]->formatImpl(settings, state, nested_need_parens); + if (first_arg_func->arguments->children.size() == 1) + first_arg_func->arguments->children[0]->formatImpl(settings, state, nested_need_parens); + else + settings.ostr << "()"; } else arguments->children[0]->formatImpl(settings, state, nested_need_parens); diff --git a/tests/queries/0_stateless/02103_sql_user_defined_functions_composition.reference b/tests/queries/0_stateless/02103_sql_user_defined_functions_composition.reference new file mode 100644 index 00000000000..51993f072d5 --- /dev/null +++ b/tests/queries/0_stateless/02103_sql_user_defined_functions_composition.reference @@ -0,0 +1,2 @@ +2 +2 diff --git a/tests/queries/0_stateless/02103_sql_user_defined_functions_composition.sql b/tests/queries/0_stateless/02103_sql_user_defined_functions_composition.sql new file mode 100644 index 00000000000..3d34413b9d3 --- /dev/null +++ b/tests/queries/0_stateless/02103_sql_user_defined_functions_composition.sql @@ -0,0 +1,12 @@ +-- Tags: no-parallel + +CREATE FUNCTION 02103_test_function AS x -> x + 1; +CREATE FUNCTION 02103_test_function_with_nested_function_empty_args AS () -> 02103_test_function(1); +CREATE FUNCTION 02103_test_function_with_nested_function_arg AS (x) -> 02103_test_function(x); + +SELECT 02103_test_function_with_nested_function_empty_args(); +SELECT 02103_test_function_with_nested_function_arg(1); + +DROP FUNCTION 02103_test_function_with_nested_function_empty_args; +DROP FUNCTION 02103_test_function_with_nested_function_arg; +DROP FUNCTION 02103_test_function; From b58f065806efbaa1477ec6b4e9d5656113905a7e Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 21 Oct 2021 08:12:16 +0000 Subject: [PATCH 281/919] One more test --- ...arity_replicated_merge_tree_long.reference | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree_long.reference b/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree_long.reference index ea48fbf86d7..1df87e72e98 100644 --- a/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree_long.reference +++ b/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree_long.reference @@ -1,56 +1,56 @@ ----HORIZONTAL MERGE TESTS---- Replica synced 4 +5 4 -4 -4 -4 -4 +5 +5 +5 Parts optimized Replica synced 8 +9 8 -8 -8 +9 ----- 4 -1 +2 Replica synced 4 -1 +2 Table attached -1 -1 +2 +2 Parts optimized 8 Replica synced 8 ----- 4 -1 +2 Replica synced 4 -1 +2 4 -1 +2 Replica synced 4 -1 +2 8 -1 +2 8 -1 +2 Parts optimized 42 42 100 aaaa 101 bbbb 102 cccc 103 dddd -2 +3 Replica synced 42 42 100 aaaa 101 bbbb 102 cccc 103 dddd -2 +3 From bd016f384e843cdbef5275149983d12cd36d5ed1 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 21 Oct 2021 12:54:10 +0300 Subject: [PATCH 282/919] Try fix tests --- tests/integration/test_storage_kafka/test.py | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 14db82a09e9..95175202b04 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2880,9 +2880,6 @@ def test_kafka_formats_with_broken_message(kafka_cluster): if format_opts.get('printable', False) == False: raw_message = 'hex(_raw_message)' kafka_produce(kafka_cluster, topic_name, data_prefix + data_sample) - format_settings = '' - if format_opts.get('format_settings'): - format_settings = 'SETTINGS ' + format_opts.get('format_settings') instance.query(''' DROP TABLE IF EXISTS test.kafka_{format_name}; @@ -2908,16 +2905,20 @@ def test_kafka_formats_with_broken_message(kafka_cluster): DROP TABLE IF EXISTS test.kafka_errors_{format_name}_mv; CREATE MATERIALIZED VIEW test.kafka_errors_{format_name}_mv Engine=Log AS SELECT {raw_message} as raw_message, _error as error, _topic as topic, _partition as partition, _offset as offset FROM test.kafka_{format_name} - WHERE length(_error) > 0 {format_settings}; + WHERE length(_error) > 0; '''.format(topic_name=topic_name, format_name=format_name, raw_message=raw_message, - extra_settings=format_opts.get('extra_settings') or '', format_settings=format_settings) + extra_settings=format_opts.get('extra_settings') or '') for format_name, format_opts in list(all_formats.items()): logging.debug('Checking {format_name}') topic_name = f"{topic_name_prefix}{format_name}" # shift offsets by 1 if format supports empty value offsets = [1, 2, 3] if format_opts.get('supports_empty_value', False) else [0, 1, 2] - result = instance.query('SELECT * FROM test.kafka_data_{format_name}_mv;'.format(format_name=format_name)) + format_settings = '' + if format_opts.get('format_settings'): + format_settings = 'SETTINGS ' + format_opts.get('format_settings') + + result = instance.query('SELECT * FROM test.kafka_data_{format_name}_mv {format_settings};'.format(format_name=format_name, format_settings=format_settings)) expected = '''\ 0 0 AM 0.5 1 {topic_name} 0 {offset_0} 1 0 AM 0.5 1 {topic_name} 0 {offset_1} From ed0bb2cec22d06eb3ff44ddb611038b982396306 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 21 Oct 2021 14:02:51 +0300 Subject: [PATCH 283/919] Fix deadlock on ALTER with scalar subquery to the same table --- src/Interpreters/MutationsInterpreter.cpp | 6 +++--- src/Interpreters/TreeRewriter.cpp | 5 +++-- src/Interpreters/TreeRewriter.h | 3 ++- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index bfa8a06c618..e7b09b6917f 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -442,7 +442,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) if (column.default_desc.kind == ColumnDefaultKind::Materialized) { auto query = column.default_desc.expression->clone(); - auto syntax_result = TreeRewriter(context).analyze(query, all_columns); + auto syntax_result = TreeRewriter(context).analyze(query, all_columns, {}, {}, false, true, dry_run); for (const String & dependency : syntax_result->requiredSourceColumns()) { if (updated_columns.count(dependency)) @@ -583,7 +583,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) throw Exception("Unknown index: " + command.index_name, ErrorCodes::BAD_ARGUMENTS); auto query = (*it).expression_list_ast->clone(); - auto syntax_result = TreeRewriter(context).analyze(query, all_columns); + auto syntax_result = TreeRewriter(context).analyze(query, all_columns, {}, {}, false, true, dry_run); const auto required_columns = syntax_result->requiredSourceColumns(); for (const auto & column : required_columns) dependencies.emplace(column, ColumnDependency::SKIP_INDEX); @@ -784,7 +784,7 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & for (const String & column : stage.output_columns) all_asts->children.push_back(std::make_shared(column)); - auto syntax_result = TreeRewriter(context).analyze(all_asts, all_columns, storage, metadata_snapshot); + auto syntax_result = TreeRewriter(context).analyze(all_asts, all_columns, storage, metadata_snapshot, false, true, dry_run); if (context->hasQueryContext()) for (const auto & it : syntax_result->getScalars()) context->getQueryContext()->addScalar(it.first, it.second); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 9bcddb6b982..8f7846678ad 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1100,7 +1100,8 @@ TreeRewriterResultPtr TreeRewriter::analyze( ConstStoragePtr storage, const StorageMetadataPtr & metadata_snapshot, bool allow_aggregations, - bool allow_self_aliases) const + bool allow_self_aliases, + bool dry_run) const { if (query->as()) throw Exception("Not select analyze for select asts.", ErrorCodes::LOGICAL_ERROR); @@ -1112,7 +1113,7 @@ TreeRewriterResultPtr TreeRewriter::analyze( normalize(query, result.aliases, result.source_columns_set, false, settings, allow_self_aliases); /// Executing scalar subqueries. Column defaults could be a scalar subquery. - executeScalarSubqueries(query, getContext(), 0, result.scalars, false); + executeScalarSubqueries(query, getContext(), 0, result.scalars, dry_run); if (settings.legacy_column_name_of_tuple_literal) markTupleLiteralsAsLegacy(query); diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index 98201f35216..2430976825f 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -110,7 +110,8 @@ public: ConstStoragePtr storage = {}, const StorageMetadataPtr & metadata_snapshot = {}, bool allow_aggregations = false, - bool allow_self_aliases = true) const; + bool allow_self_aliases = true, + bool dry_run = false) const; /// Analyze and rewrite select query TreeRewriterResultPtr analyzeSelect( From 78acad5456d1e2bd43ff0033e2261e9e17e21e47 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 21 Oct 2021 14:09:15 +0300 Subject: [PATCH 284/919] Trying builds on actions --- .github/workflows/main.yml | 39 +++- docker/packager/packager | 2 +- tests/ci/build_check.py | 186 ++++++++++++++++++ tests/ci/metrics_lambda/app.py | 30 ++- tests/ci/worker/init_builder.sh | 20 ++ .../worker/{init.sh => init_style_checker.sh} | 2 +- .../{ubuntu_ami.sh => ubuntu_style_check.sh} | 0 7 files changed, 267 insertions(+), 12 deletions(-) create mode 100644 tests/ci/build_check.py create mode 100644 tests/ci/worker/init_builder.sh rename tests/ci/worker/{init.sh => init_style_checker.sh} (94%) rename tests/ci/worker/{ubuntu_ami.sh => ubuntu_style_check.sh} (100%) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 7f20206a7b3..487885adb87 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -1,4 +1,4 @@ -name: Ligthweight GithubActions +name: CIGithubActions on: # yamllint disable-line rule:truthy pull_request: types: @@ -11,7 +11,7 @@ on: # yamllint disable-line rule:truthy - master jobs: CheckLabels: - runs-on: [self-hosted] + runs-on: [self-hosted, style-checker] steps: - name: Check out repository code uses: actions/checkout@v2 @@ -19,7 +19,7 @@ jobs: run: cd $GITHUB_WORKSPACE/tests/ci && python3 run_check.py DockerHubPush: needs: CheckLabels - runs-on: [self-hosted] + runs-on: [self-hosted, style-checker] steps: - name: Check out repository code uses: actions/checkout@v2 @@ -32,7 +32,7 @@ jobs: path: ${{ runner.temp }}/docker_images_check/changed_images.json StyleCheck: needs: DockerHubPush - runs-on: [self-hosted] + runs-on: [self-hosted, style-checker] steps: - name: Download changed images uses: actions/download-artifact@v2 @@ -43,9 +43,38 @@ jobs: uses: actions/checkout@v2 - name: Style Check run: cd $GITHUB_WORKSPACE/tests/ci && python3 style_check.py + Builder: + needs: DockerHubPush + runs-on: [self-hosted, builder] + strategy: + continue-on-error: true + fail-fast: false + matrix: + build_number: [7] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/build_check + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CHECK_NAME: 'ClickHouse build check (actions)' + run: mkdir -p $TEMP_PATH && cp -r $GITHUB_WORKSPACE $TEMP_PATH && cd $REPO_COPY/tests/ci && python3 build_check.py $CHECK_NAME ${{ matrix.build_number }} '21.10.1.1' + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json FinishCheck: needs: [StyleCheck, DockerHubPush, CheckLabels] - runs-on: [self-hosted] + runs-on: [self-hosted, style-checker] steps: - name: Check out repository code uses: actions/checkout@v2 diff --git a/docker/packager/packager b/docker/packager/packager index ae7b99200ee..e8d18aafdb8 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -42,7 +42,7 @@ def run_docker_image_with_env(image_name, output, env_variables, ch_root, ccache else: interactive = "" - cmd = "docker run --network=host --rm --volume={output_path}:/output --volume={ch_root}:/build --volume={ccache_dir}:/ccache {env} {interactive} {img_name}".format( + cmd = "docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --network=host --rm --volume={output_path}:/output --volume={ch_root}:/build --volume={ccache_dir}:/ccache {env} {interactive} {img_name}".format( output_path=output, ch_root=ch_root, ccache_dir=ccache_dir, diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py new file mode 100644 index 00000000000..ed117912f67 --- /dev/null +++ b/tests/ci/build_check.py @@ -0,0 +1,186 @@ +#!/usr/bin/env python3 +# +import subprocess +import logging +from report import create_test_html_report +from s3_helper import S3Helper +import json +import os +from pr_info import PRInfo +from github import Github +import shutil +from get_robot_token import get_best_robot_token, get_parameter_from_ssm +import os +import sys +import time + + +def get_build_config(build_check_name, build_number, repo_path): + if build_check_name == 'ClickHouse build check (actions)': + build_config_name = 'build_config' + elif build_check_name == 'ClickHouse special build check (actions)': + build_config_name = 'special_build_config' + + ci_config_path = os.path.join(repo_path, "tests/ci/ci_config.json") + with open(ci_config_path, 'r') as ci_config: + return ci_config[build_config_name][build_number] + + +def _can_export_binaries(build_config): + if build_config['package_type'] != 'deb': + return False + if build_config['bundled'] != "bundled": + return False + if build_config['splitted'] == 'splitted': + return False + if build_config['sanitizer'] != '': + return True + if build_config[''] != '': + return True + return False + + +def get_packager_cmd(build_config, packager_path, output_path, build_version, image_version): + cmd = "cd {ppath} && ./packager --output-dir={odir} --package-type={package_type} --compiler={comp}".format( + ppath=packager_path, + odir=output_path, + package_type=build_config['package-type'], + comp=build_config['compiler'] + ) + + if build_config['build_type']: + cmd += ' --build-type={}'.format(build_config['build_type']) + if build_config['sanitizer']: + cmd += ' --sanitizer={}'.format(build_config['sanitizer']) + if build_config['bundled'] == 'unbundled': + cmd += ' --unbundled' + if build_config['splitted'] == 'splitted': + cmd += ' --split-binary' + if build_config['tidy'] == 'enable': + cmd += ' --clang-tidy' + + if 'alien_pkgs' in build_config and build_config['alien_pkgs']: + cmd += ' --alien-pkgs' + + cmd += ' --docker-image-version={}'.format(image_version) + cmd += ' --version={}'.format(build_version) + + if _can_export_binaries(build_config): + cmd += ' --with-binaries=tests' + + return cmd + +def get_image_name(build_config): + if build_config['bundled'] != 'bundled': + return 'clickhouse/unbundled-builder' + elif build_config['package_type'] != 'deb': + return 'clickhouse/binary-builder' + else: + return 'clickhouse/deb-builder' + + +def build_clickhouse(packager_cmd, logs_path): + build_log_path = os.path.join(logs_path, 'build_log.log') + with open(build_log_path, 'w') as log_file: + retcode = subprocess.Popen(packager_cmd, shell=True, stderr=log_file, stdout=log_file).wait() + if retcode == 0: + logging.info("Built successfully") + else: + logging.info("Build failed") + return build_log_path, success + +def build_config_to_string(build_config): + if build_config["package_type"] == "performance": + return "performance" + + return "_".join([ + build_config['compiler'], + build_config['build_type'] if build_config['build_type'] else "relwithdebuginfo", + build_config['sanitizer'] if build_config['sanitizer'] else "none", + build_config['bundled'], + build_config['splitted'], + build_config['tidy'], + build_config['with_coverage'], + build_config['package_type'], + ]) + +if __name__ == "__main__": + logging.basicConfig(level=logging.INFO) + repo_path = os.path.join(os.getenv("REPO_COPY", os.path.abspath("../../"))) + temp_path = os.path.join(os.getenv("TEMP_PATH")) + + build_check_name = sys.argv[1] + build_number = int(sys.argv[2]) + build_version = sys.argv[3] + + build_config = get_build_config(build_check_name, build_number, repo_path) + + if not os.path.exists(temp_path): + os.makedirs(temp_path) + + with open(os.getenv('GITHUB_EVENT_PATH'), 'r') as event_file: + event = json.load(event_file) + + pr_info = PRInfo(event) + + logging.info("Repo copy path %s", repo_path) + + gh = Github(get_best_robot_token()) + + images_path = os.path.join(temp_path, 'changed_images.json') + image_name = get_image_name(build_config) + image_version = 'latest' + if os.path.exists(images_path): + logging.info("Images file exists") + with open(images_path, 'r') as images_fd: + images = json.load(images_fd) + logging.info("Got images %s", images) + if image_name in images: + image_version += images[image_name] + + for i in range(10): + try: + subprocess.check_output(f"docker pull {image_name}:{image_version}", shell=True) + break + except Exception as ex: + time.sleep(i * 3) + logging.info("Got execption pulling docker %s", ex) + else: + raise Exception(f"Cannot pull dockerhub for image docker pull {image_name}:{image_version}") + + + build_name = build_config_to_string(build_config) + os.environ['BUILD_NAME'] = build_name + + build_output_path = os.path.join(temp_path, build_name) + if not os.path.exists(build_output_path): + os.makedirs(build_output_path) + + packager_cmd = get_packager_cmd(build_config, os.path.join(repo_path, "docker/packager"), build_output_path, build_version, image_version) + logging.info("Going to run packager with %s", packager_cmd) + + build_clickhouse_log = os.path.join(temp_path, "build_log") + if not os.path.exists(build_clickhouse_log): + os.makedirs(build_clickhouse_log) + + log_path, success = build_clickhouse(packager_cmd, build_clickhouse_log) + logging.info("Build finished with %s, log path %s", success, log_path) + + s3_helper = S3Helper('https://s3.amazonaws.com') + s3_path_prefix = str(pr_info.number) + "/" + pr_info.sha + "/" + build_check_name.lower().replace(' ', '_') + "/" + build_name + if os.path.exists(log_path): + log_url = s3_helper.upload_build_file_to_s3(log_path, s3_path_prefix + "/" + os.path.basename(log_path)) + logging.info("Log url %s", log_url) + else: + logging.info("Build log doesn't exist") + + build_urls = s3_helper.upload_build_folder_to_s3(build_output_path, s3_path_prefix, keep_dirs_in_s3_path=False, upload_symlinks=False) + result = { + "log_url": log_url, + "build_urls": build_urls, + } + + print("::notice ::Log URL: {}".format(log_url)) + + with open(os.path.join(temp_path, build_name + '.json'), 'w') as build_links: + json.dump(result, build_links) diff --git a/tests/ci/metrics_lambda/app.py b/tests/ci/metrics_lambda/app.py index d2fb048638b..2a3c7670ac4 100644 --- a/tests/ci/metrics_lambda/app.py +++ b/tests/ci/metrics_lambda/app.py @@ -67,6 +67,24 @@ def list_runners(access_token): result.append(desc) return result +def group_runners_by_tag(listed_runners): + result = {} + + RUNNER_TYPE_LABELS = ['style-checker', 'builder'] + for runner in listed_runners: + for tag in runner.tags: + if tag in RUNNER_TYPE_LABELS: + if tag not in result: + result[tag] = [] + result[tag].append(runner) + break + else: + if 'unlabeled' not in result: + result['unlabeled'] = [] + result['unlabeled'].append(runner) + return result + + def push_metrics_to_cloudwatch(listed_runners, namespace): import boto3 client = boto3.client('cloudwatch') @@ -100,7 +118,7 @@ def push_metrics_to_cloudwatch(listed_runners, namespace): 'Unit': 'Percent', }) - client.put_metric_data(Namespace='RunnersMetrics', MetricData=metrics_data) + client.put_metric_data(Namespace=namespace, MetricData=metrics_data) def main(github_secret_key, github_app_id, push_to_cloudwatch): payload = { @@ -113,10 +131,12 @@ def main(github_secret_key, github_app_id, push_to_cloudwatch): installation_id = get_installation_id(encoded_jwt) access_token = get_access_token(encoded_jwt, installation_id) runners = list_runners(access_token) - if push_to_cloudwatch: - push_metrics_to_cloudwatch(runners, 'RunnersMetrics') - else: - print(runners) + grouped_runners = group_runners_by_tag(runners) + for group, group_runners in grouped_runners.items(): + if push_to_cloudwatch: + push_metrics_to_cloudwatch(group_runners, 'RunnersMetrics/' + group) + else: + print(group, group_runners) if __name__ == "__main__": diff --git a/tests/ci/worker/init_builder.sh b/tests/ci/worker/init_builder.sh new file mode 100644 index 00000000000..dc3f777bcca --- /dev/null +++ b/tests/ci/worker/init_builder.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash +set -euo pipefail + +echo "Running init script" +export DEBIAN_FRONTEND=noninteractive +export RUNNER_HOME=/home/ubuntu/actions-runner + +echo "Receiving token" +export RUNNER_TOKEN=`/usr/local/bin/aws ssm get-parameter --name github_runner_registration_token --with-decryption --output text --query Parameter.Value` +export RUNNER_URL="https://github.com/ClickHouse" +# Funny fact, but metadata service has fixed IP +export INSTANCE_ID=`curl -s http://169.254.169.254/latest/meta-data/instance-id` + +cd $RUNNER_HOME + +echo "Going to configure runner" +sudo -u ubuntu ./config.sh --url $RUNNER_URL --token $RUNNER_TOKEN --name $INSTANCE_ID --runnergroup Default --labels 'self-hosted,Linux,X64,builder' --work _work + +echo "Run" +sudo -u ubuntu ./run.sh diff --git a/tests/ci/worker/init.sh b/tests/ci/worker/init_style_checker.sh similarity index 94% rename from tests/ci/worker/init.sh rename to tests/ci/worker/init_style_checker.sh index 2f6638f14b5..77cf66b5262 100644 --- a/tests/ci/worker/init.sh +++ b/tests/ci/worker/init_style_checker.sh @@ -14,7 +14,7 @@ export INSTANCE_ID=`curl -s http://169.254.169.254/latest/meta-data/instance-id` cd $RUNNER_HOME echo "Going to configure runner" -sudo -u ubuntu ./config.sh --url $RUNNER_URL --token $RUNNER_TOKEN --name $INSTANCE_ID --runnergroup Default --labels 'self-hosted,Linux,X64' --work _work +sudo -u ubuntu ./config.sh --url $RUNNER_URL --token $RUNNER_TOKEN --name $INSTANCE_ID --runnergroup Default --labels 'self-hosted,Linux,X64,style-checker' --work _work echo "Run" sudo -u ubuntu ./run.sh diff --git a/tests/ci/worker/ubuntu_ami.sh b/tests/ci/worker/ubuntu_style_check.sh similarity index 100% rename from tests/ci/worker/ubuntu_ami.sh rename to tests/ci/worker/ubuntu_style_check.sh From cae2f8a361558a0a0c81a775b78df0c61760fc40 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 21 Oct 2021 14:11:17 +0300 Subject: [PATCH 285/919] Fix yml --- .github/workflows/main.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 487885adb87..0fd9a7547be 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -47,7 +47,6 @@ jobs: needs: DockerHubPush runs-on: [self-hosted, builder] strategy: - continue-on-error: true fail-fast: false matrix: build_number: [7] From b720b244efc8f2d9c9bebf3af96b0c3b1908bae4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 21 Oct 2021 14:17:53 +0300 Subject: [PATCH 286/919] Add test alter_scalar_circular_deadlock --- ...100_alter_scalar_circular_deadlock.reference | 2 ++ .../02100_alter_scalar_circular_deadlock.sql | 17 +++++++++++++++++ 2 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.reference create mode 100644 tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.sql diff --git a/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.reference b/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.sql b/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.sql new file mode 100644 index 00000000000..5fde10d5d09 --- /dev/null +++ b/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.sql @@ -0,0 +1,17 @@ +DROP TABLE IF EXISTS foo; + +CREATE TABLE foo (ts DateTime, x UInt64) +ENGINE = MergeTree PARTITION BY toYYYYMMDD(ts) +ORDER BY (ts); + +INSERT INTO foo (ts, x) SELECT toDateTime('2020-01-01 00:05:00'), number from system.numbers_mt LIMIT 10; + +SET mutations_sync = 1; + +ALTER TABLE foo UPDATE x = 1 WHERE x = (SELECT x from foo WHERE x = 4); + +SELECT sum(x) == 42 FROM foo; + +ALTER TABLE foo UPDATE x = 1 WHERE x IN (SELECT x FROM foo WHERE x != 0); + +SELECT sum(x) == 9 FROM foo; From 53eb5c79ba73fbe4ef390f9086471b9c0d25bf0c Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 21 Oct 2021 14:31:40 +0300 Subject: [PATCH 287/919] Fixes --- .github/workflows/main.yml | 2 +- tests/ci/build_check.py | 42 +++++++++++++++++++------------------- 2 files changed, 22 insertions(+), 22 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 0fd9a7547be..70f995832f7 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -65,7 +65,7 @@ jobs: TEMP_PATH: ${{runner.temp}}/build_check REPO_COPY: ${{runner.temp}}/build_check/ClickHouse CHECK_NAME: 'ClickHouse build check (actions)' - run: mkdir -p $TEMP_PATH && cp -r $GITHUB_WORKSPACE $TEMP_PATH && cd $REPO_COPY/tests/ci && python3 build_check.py $CHECK_NAME ${{ matrix.build_number }} '21.10.1.1' + run: mkdir -p $TEMP_PATH && cp -r $GITHUB_WORKSPACE $TEMP_PATH && cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" ${{ matrix.build_number }} '21.10.1.1' - name: Upload build URLs to artifacts uses: actions/upload-artifact@v2 with: diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index ed117912f67..f85201d9f25 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -23,11 +23,12 @@ def get_build_config(build_check_name, build_number, repo_path): ci_config_path = os.path.join(repo_path, "tests/ci/ci_config.json") with open(ci_config_path, 'r') as ci_config: - return ci_config[build_config_name][build_number] + config_dict = json.load(ci_config) + return config_dict[build_config_name][build_number] def _can_export_binaries(build_config): - if build_config['package_type'] != 'deb': + if build_config['package-type'] != 'deb': return False if build_config['bundled'] != "bundled": return False @@ -35,21 +36,18 @@ def _can_export_binaries(build_config): return False if build_config['sanitizer'] != '': return True - if build_config[''] != '': + if build_config['build-type'] != '': return True return False def get_packager_cmd(build_config, packager_path, output_path, build_version, image_version): - cmd = "cd {ppath} && ./packager --output-dir={odir} --package-type={package_type} --compiler={comp}".format( - ppath=packager_path, - odir=output_path, - package_type=build_config['package-type'], - comp=build_config['compiler'] - ) + package_type = build_config['package-type'] + comp = build_config['compiler'] + cmd = f"cd {packager_path} && ./packager --output-dir={output_path} --package-type={package_type} --compiler={comp}" - if build_config['build_type']: - cmd += ' --build-type={}'.format(build_config['build_type']) + if build_config['build-type']: + cmd += ' --build-type={}'.format(build_config['build-type']) if build_config['sanitizer']: cmd += ' --sanitizer={}'.format(build_config['sanitizer']) if build_config['bundled'] == 'unbundled': @@ -73,7 +71,7 @@ def get_packager_cmd(build_config, packager_path, output_path, build_version, im def get_image_name(build_config): if build_config['bundled'] != 'bundled': return 'clickhouse/unbundled-builder' - elif build_config['package_type'] != 'deb': + elif build_config['package-type'] != 'deb': return 'clickhouse/binary-builder' else: return 'clickhouse/deb-builder' @@ -87,27 +85,27 @@ def build_clickhouse(packager_cmd, logs_path): logging.info("Built successfully") else: logging.info("Build failed") - return build_log_path, success + return build_log_path, retcode == 0 def build_config_to_string(build_config): - if build_config["package_type"] == "performance": + if build_config["package-type"] == "performance": return "performance" return "_".join([ build_config['compiler'], - build_config['build_type'] if build_config['build_type'] else "relwithdebuginfo", + build_config['build-type'] if build_config['build-type'] else "relwithdebuginfo", build_config['sanitizer'] if build_config['sanitizer'] else "none", build_config['bundled'], build_config['splitted'], - build_config['tidy'], - build_config['with_coverage'], - build_config['package_type'], + "tidy" if build_config['tidy'] else "notidy", + "with_coverage" if build_config['with_coverage'] else "without_coverage", + build_config['package-type'], ]) if __name__ == "__main__": logging.basicConfig(level=logging.INFO) - repo_path = os.path.join(os.getenv("REPO_COPY", os.path.abspath("../../"))) - temp_path = os.path.join(os.getenv("TEMP_PATH")) + repo_path = os.getenv("REPO_COPY", os.path.abspath("../../")) + temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) build_check_name = sys.argv[1] build_number = int(sys.argv[2]) @@ -140,7 +138,8 @@ if __name__ == "__main__": for i in range(10): try: - subprocess.check_output(f"docker pull {image_name}:{image_version}", shell=True) + logging.info(f"Pulling image {image_name}:{image_version}") + subprocess.check_output(f"docker pull {image_name}:{image_version}", stderr=subprocess.STDOUT, shell=True) break except Exception as ex: time.sleep(i * 3) @@ -150,6 +149,7 @@ if __name__ == "__main__": build_name = build_config_to_string(build_config) + logging.info(f"Build short name {build_name}") os.environ['BUILD_NAME'] = build_name build_output_path = os.path.join(temp_path, build_name) From 97ce422d5aa17a779175877c166f7998910e60e1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 21 Oct 2021 14:48:56 +0300 Subject: [PATCH 288/919] Add ccache path --- tests/ci/build_check.py | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index f85201d9f25..5312ef15999 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -41,7 +41,7 @@ def _can_export_binaries(build_config): return False -def get_packager_cmd(build_config, packager_path, output_path, build_version, image_version): +def get_packager_cmd(build_config, packager_path, output_path, build_version, image_version, ccache_path): package_type = build_config['package-type'] comp = build_config['compiler'] cmd = f"cd {packager_path} && ./packager --output-dir={output_path} --package-type={package_type} --compiler={comp}" @@ -57,6 +57,9 @@ def get_packager_cmd(build_config, packager_path, output_path, build_version, im if build_config['tidy'] == 'enable': cmd += ' --clang-tidy' + cmd += ' --cache=ccache' + cmd += ' --ccache_dir={}'.format(ccache_path) + if 'alien_pkgs' in build_config and build_config['alien_pkgs']: cmd += ' --alien-pkgs' @@ -156,7 +159,11 @@ if __name__ == "__main__": if not os.path.exists(build_output_path): os.makedirs(build_output_path) - packager_cmd = get_packager_cmd(build_config, os.path.join(repo_path, "docker/packager"), build_output_path, build_version, image_version) + ccache_path = os.path.join(temp_path, build_name + '_ccache') + if not os.path.exists(ccache_path): + os.makedirs(ccache_path) + + packager_cmd = get_packager_cmd(build_config, os.path.join(repo_path, "docker/packager"), build_output_path, build_version, image_version, ccache_path) logging.info("Going to run packager with %s", packager_cmd) build_clickhouse_log = os.path.join(temp_path, "build_log") From a1bb561299ec95c96c3467b28b7f9b98f0394933 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 21 Oct 2021 15:11:27 +0300 Subject: [PATCH 289/919] Don't use system user in packager --- .github/workflows/main.yml | 2 +- docker/packager/packager | 2 +- tests/ci/build_check.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 70f995832f7..8d97656e6bc 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -72,7 +72,7 @@ jobs: name: ${{ env.BUILD_NAME }} path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json FinishCheck: - needs: [StyleCheck, DockerHubPush, CheckLabels] + needs: [StyleCheck, DockerHubPush, CheckLabels, Builder] runs-on: [self-hosted, style-checker] steps: - name: Check out repository code diff --git a/docker/packager/packager b/docker/packager/packager index e8d18aafdb8..ae7b99200ee 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -42,7 +42,7 @@ def run_docker_image_with_env(image_name, output, env_variables, ch_root, ccache else: interactive = "" - cmd = "docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --network=host --rm --volume={output_path}:/output --volume={ch_root}:/build --volume={ccache_dir}:/ccache {env} {interactive} {img_name}".format( + cmd = "docker run --network=host --rm --volume={output_path}:/output --volume={ch_root}:/build --volume={ccache_dir}:/ccache {env} {interactive} {img_name}".format( output_path=output, ch_root=ch_root, ccache_dir=ccache_dir, diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 5312ef15999..c010f272038 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -100,7 +100,7 @@ def build_config_to_string(build_config): build_config['sanitizer'] if build_config['sanitizer'] else "none", build_config['bundled'], build_config['splitted'], - "tidy" if build_config['tidy'] else "notidy", + "tidy" if build_config['tidy'] == "enable" else "notidy", "with_coverage" if build_config['with_coverage'] else "without_coverage", build_config['package-type'], ]) From 2ef25184d0ac658e8e05681317a5d642648a9fe1 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Thu, 21 Oct 2021 15:19:48 +0300 Subject: [PATCH 290/919] Update argmax.md --- docs/ru/sql-reference/aggregate-functions/reference/argmax.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/argmax.md b/docs/ru/sql-reference/aggregate-functions/reference/argmax.md index 84419523beb..bdf7b1d1df6 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/argmax.md @@ -6,8 +6,6 @@ toc_priority: 106 Вычисляет значение `arg` при максимальном значении `val`. Если есть несколько разных значений `arg` для максимальных значений `val`, возвращает первое попавшееся из таких значений. -Если функции передан кортеж, то будет выведен кортеж с максимальным значением `val`. Удобно использовать для работы с [SimpleAggregateFunction](../../../sql-reference/data-types/simpleaggregatefunction.md). - **Синтаксис** ``` sql From a43d293d12320545119570089da3b00585a03123 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Thu, 21 Oct 2021 15:20:25 +0300 Subject: [PATCH 291/919] Update argmax.md --- docs/zh/sql-reference/aggregate-functions/reference/argmax.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/zh/sql-reference/aggregate-functions/reference/argmax.md b/docs/zh/sql-reference/aggregate-functions/reference/argmax.md index 0c82cb8a4bb..ed3d2d07a58 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/argmax.md @@ -6,8 +6,6 @@ toc_priority: 106 计算 `val` 最大值对应的 `arg` 值。 如果 `val` 最大值存在几个不同的 `arg` 值,输出遇到的第一个值。 -这个函数的Tuple版本将返回 `val` 最大值对应的元组。本函数适合和 `SimpleAggregateFunction` 搭配使用。 - **语法** ``` sql From 3afb070b15613f2f99ed25b4e6479f86c194ff9c Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Thu, 21 Oct 2021 15:20:52 +0300 Subject: [PATCH 292/919] Update argmin.md --- docs/ru/sql-reference/aggregate-functions/reference/argmin.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/argmin.md b/docs/ru/sql-reference/aggregate-functions/reference/argmin.md index 4d2363abe6d..dd923061943 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/argmin.md @@ -6,8 +6,6 @@ toc_priority: 105 Вычисляет значение `arg` при минимальном значении `val`. Если есть несколько разных значений `arg` для минимальных значений `val`, возвращает первое попавшееся из таких значений. -Если функции передан кортеж, то будет выведен кортеж с минимальным значением `val`. Удобно использовать для работы с [SimpleAggregateFunction](../../../sql-reference/data-types/simpleaggregatefunction.md). - **Синтаксис** ``` sql From a48da6294971be5f0a6e31aa11a95e13543dd742 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Thu, 21 Oct 2021 15:21:33 +0300 Subject: [PATCH 293/919] Update argmin.md --- docs/zh/sql-reference/aggregate-functions/reference/argmin.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/zh/sql-reference/aggregate-functions/reference/argmin.md b/docs/zh/sql-reference/aggregate-functions/reference/argmin.md index 6c2839062eb..8d987300ad4 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/argmin.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/argmin.md @@ -8,8 +8,6 @@ toc_priority: 105 计算 `val` 最小值对应的 `arg` 值。 如果 `val` 最小值存在几个不同的 `arg` 值,输出遇到的第一个(`arg`)值。 -这个函数的Tuple版本将返回 `val` 最小值对应的tuple。本函数适合和`SimpleAggregateFunction`搭配使用。 - **示例:** 输入表: From c1d54700db2fa8ecb9821c51e14ba81c5b340c84 Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Thu, 21 Oct 2021 15:32:58 +0300 Subject: [PATCH 294/919] Update json-functions.md --- docs/ru/sql-reference/functions/json-functions.md | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/docs/ru/sql-reference/functions/json-functions.md b/docs/ru/sql-reference/functions/json-functions.md index de89a50219b..243078c1756 100644 --- a/docs/ru/sql-reference/functions/json-functions.md +++ b/docs/ru/sql-reference/functions/json-functions.md @@ -307,7 +307,7 @@ SELECT JSONExtractKeysAndValuesRaw('{"a": [-100, 200.0], "b":{"c": {"d": "hello" └───────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` -## JSON_EXISTS(path, json) {#json-exists} +## JSON_EXISTS(json, path) {#json-exists} Если значение существует в документе JSON, то возвращается 1. @@ -322,7 +322,10 @@ SELECT JSON_EXISTS('$.hello[*]', '{"hello":["world"]}'); SELECT JSON_EXISTS('$.hello[0]', '{"hello":["world"]}'); ``` -## JSON_QUERY(path, json) {#json-query} +!!! note "Примечание" + до версии 21.11 порядок аргументов функции был обратный, т.е. JSON_EXISTS(path, json) + +## JSON_QUERY(json, path) {#json-query} Парсит JSON и извлекает строку. @@ -345,8 +348,10 @@ SELECT toTypeName(JSON_QUERY('$.hello', '{"hello":2}')); [2] String ``` +!!! note "Примечание" + до версии 21.11 порядок аргументов функции был обратный, т.е. JSON_QUERY(path, json) -## JSON_VALUE(path, json) {#json-value} +## JSON_VALUE(json, path) {#json-value} Парсит JSON и извлекает строку. @@ -370,6 +375,9 @@ SELECT toTypeName(JSON_VALUE('$.hello', '{"hello":2}')); String ``` +!!! note "Примечание" + до версии 21.11 порядок аргументов функции был обратный, т.е. JSON_VALUE(path, json) + ## toJSONString {#tojsonstring} Сериализует значение в JSON представление. Поддерживаются различные типы данных и вложенные структуры. From f9a50cd84abcf95f3bde90974b04e9b5387799b4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 21 Oct 2021 15:37:19 +0300 Subject: [PATCH 295/919] Fix permissions --- tests/ci/build_check.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index c010f272038..4cbca73ad08 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -153,6 +153,7 @@ if __name__ == "__main__": build_name = build_config_to_string(build_config) logging.info(f"Build short name {build_name}") + subprocess.check_call(f"echo 'BUILD_NAME={build_name}' >> $GITHUB_ENV", shell=True) os.environ['BUILD_NAME'] = build_name build_output_path = os.path.join(temp_path, build_name) @@ -171,6 +172,8 @@ if __name__ == "__main__": os.makedirs(build_clickhouse_log) log_path, success = build_clickhouse(packager_cmd, build_clickhouse_log) + subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {build_output_path}", shell=True) + subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {ccache_path}", shell=True) logging.info("Build finished with %s, log path %s", success, log_path) s3_helper = S3Helper('https://s3.amazonaws.com') @@ -182,6 +185,7 @@ if __name__ == "__main__": logging.info("Build log doesn't exist") build_urls = s3_helper.upload_build_folder_to_s3(build_output_path, s3_path_prefix, keep_dirs_in_s3_path=False, upload_symlinks=False) + logging.info("Got build URLs %s", build_urls) result = { "log_url": log_url, "build_urls": build_urls, From 9152e3c45d913161fc6acec0ff8d66c88d9b3abd Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 21 Oct 2021 15:42:21 +0300 Subject: [PATCH 296/919] Trying better with permissions --- .github/workflows/main.yml | 1 + tests/ci/build_check.py | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 8d97656e6bc..afe654f6961 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -64,6 +64,7 @@ jobs: env: TEMP_PATH: ${{runner.temp}}/build_check REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp/..}} CHECK_NAME: 'ClickHouse build check (actions)' run: mkdir -p $TEMP_PATH && cp -r $GITHUB_WORKSPACE $TEMP_PATH && cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" ${{ matrix.build_number }} '21.10.1.1' - name: Upload build URLs to artifacts diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 4cbca73ad08..88cd7d82d75 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -109,6 +109,7 @@ if __name__ == "__main__": logging.basicConfig(level=logging.INFO) repo_path = os.getenv("REPO_COPY", os.path.abspath("../../")) temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) + caches_path = os.getenv("CACHES_PATH", temp_path) build_check_name = sys.argv[1] build_number = int(sys.argv[2]) @@ -154,13 +155,12 @@ if __name__ == "__main__": build_name = build_config_to_string(build_config) logging.info(f"Build short name {build_name}") subprocess.check_call(f"echo 'BUILD_NAME={build_name}' >> $GITHUB_ENV", shell=True) - os.environ['BUILD_NAME'] = build_name build_output_path = os.path.join(temp_path, build_name) if not os.path.exists(build_output_path): os.makedirs(build_output_path) - ccache_path = os.path.join(temp_path, build_name + '_ccache') + ccache_path = os.path.join(caches_path, build_name + '_ccache') if not os.path.exists(ccache_path): os.makedirs(ccache_path) From ce9f9de999a8986692f056686971b11c04783bc2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 21 Oct 2021 15:46:25 +0300 Subject: [PATCH 297/919] Better --- .github/workflows/main.yml | 2 +- tests/ci/build_check.py | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index afe654f6961..f5cfae273c4 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -64,7 +64,7 @@ jobs: env: TEMP_PATH: ${{runner.temp}}/build_check REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp/..}} + CACHES_PATH: ${{runner.temp}}/.. CHECK_NAME: 'ClickHouse build check (actions)' run: mkdir -p $TEMP_PATH && cp -r $GITHUB_WORKSPACE $TEMP_PATH && cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" ${{ matrix.build_number }} '21.10.1.1' - name: Upload build URLs to artifacts diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 88cd7d82d75..3b82b45afe3 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -186,6 +186,10 @@ if __name__ == "__main__": build_urls = s3_helper.upload_build_folder_to_s3(build_output_path, s3_path_prefix, keep_dirs_in_s3_path=False, upload_symlinks=False) logging.info("Got build URLs %s", build_urls) + + for url in build_urls: + print("::notice ::Build URL: {}".format(url)) + result = { "log_url": log_url, "build_urls": build_urls, From c51fdd66f30a6400d3c48298574b77e605fa4cbb Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 21 Oct 2021 16:18:59 +0300 Subject: [PATCH 298/919] Fixes --- .github/workflows/main.yml | 4 +- tests/ci/build_check.py | 7 +- tests/ci/s3_helper.py | 3 +- tests/ci/version_helper.py | 139 +++++++++++++++++++++++++++++++++++++ 4 files changed, 148 insertions(+), 5 deletions(-) create mode 100644 tests/ci/version_helper.py diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index f5cfae273c4..6ea62e0a8b9 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -64,9 +64,9 @@ jobs: env: TEMP_PATH: ${{runner.temp}}/build_check REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/.. + CACHES_PATH: ${{runner.temp}}/ccaches/.. CHECK_NAME: 'ClickHouse build check (actions)' - run: mkdir -p $TEMP_PATH && cp -r $GITHUB_WORKSPACE $TEMP_PATH && cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" ${{ matrix.build_number }} '21.10.1.1' + run: mkdir -p $TEMP_PATH && cp -r $GITHUB_WORKSPACE $TEMP_PATH && cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" ${{ matrix.build_number }} - name: Upload build URLs to artifacts uses: actions/upload-artifact@v2 with: diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 3b82b45afe3..4e35d11ddc3 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -13,6 +13,7 @@ from get_robot_token import get_best_robot_token, get_parameter_from_ssm import os import sys import time +from version_helper import get_version_from_repo, update_version_local def get_build_config(build_check_name, build_number, repo_path): @@ -113,7 +114,6 @@ if __name__ == "__main__": build_check_name = sys.argv[1] build_number = int(sys.argv[2]) - build_version = sys.argv[3] build_config = get_build_config(build_check_name, build_number, repo_path) @@ -151,6 +151,9 @@ if __name__ == "__main__": else: raise Exception(f"Cannot pull dockerhub for image docker pull {image_name}:{image_version}") + version = get_version_from_repo(repo_path) + version.tweak_update() + update_version_local(repo_path, pr_info.sha, version) build_name = build_config_to_string(build_config) logging.info(f"Build short name {build_name}") @@ -164,7 +167,7 @@ if __name__ == "__main__": if not os.path.exists(ccache_path): os.makedirs(ccache_path) - packager_cmd = get_packager_cmd(build_config, os.path.join(repo_path, "docker/packager"), build_output_path, build_version, image_version, ccache_path) + packager_cmd = get_packager_cmd(build_config, os.path.join(repo_path, "docker/packager"), build_output_path, version.get_version_string(), image_version, ccache_path) logging.info("Going to run packager with %s", packager_cmd) build_clickhouse_log = os.path.join(temp_path, "build_log") diff --git a/tests/ci/s3_helper.py b/tests/ci/s3_helper.py index 3c930f26634..ed0e66ee7a1 100644 --- a/tests/ci/s3_helper.py +++ b/tests/ci/s3_helper.py @@ -56,7 +56,8 @@ class S3Helper(object): self.client.upload_file(file_path, bucket_name, s3_path, ExtraArgs=metadata) logging.info("Upload {} to {}. Meta: {}".format(file_path, s3_path, metadata)) - return "https://s3.amazonaws.com/{bucket}/{path}".format(bucket=bucket_name, path=s3_path) + # last two replacements are specifics of AWS urls: https://jamesd3142.wordpress.com/2018/02/28/amazon-s3-and-the-plus-symbol/ + return "https://s3.amazonaws.com/{bucket}/{path}".format(bucket=bucket_name, path=s3_path).replace('+', '%2B').replace(' ', '%20') def upload_test_report_to_s3(self, file_path, s3_path): return self._upload_file_to_s3('clickhouse-test-reports', file_path, s3_path) diff --git a/tests/ci/version_helper.py b/tests/ci/version_helper.py new file mode 100644 index 00000000000..9c89b4df65e --- /dev/null +++ b/tests/ci/version_helper.py @@ -0,0 +1,139 @@ +#!/usr/bin/env python3 +import os +import subprocess +import datetime + +FILE_WITH_VERSION_PATH = "cmake/autogenerated_versions.txt" +CHANGELOG_IN_PATH = "debian/changelog.in" +CHANGELOG_PATH = "debian/changelog" +CONTRIBUTORS_SCRIPT_DIR = "src/Storages/System/" + + +class ClickHouseVersion(object): + def __init__(self, major, minor, patch, tweak, revision): + self.major = major + self.minor = minor + self.patch = patch + self.tweak = tweak + self.revision = revision + + def minor_update(self): + return ClickHouseVersion( + self.major, + self.minor + 1, + 1, + 1, + self.revision + 1) + + def patch_update(self): + return ClickHouseVersion( + self.major, + self.minor, + self.patch + 1, + 1, + self.revision) + + def tweak_update(self): + return ClickHouseVersion( + self.major, + self.minor, + self.patch, + self.tweak + 1, + self.revision) + + def get_version_string(self): + return '.'.join([ + str(self.major), + str(self.minor), + str(self.patch), + str(self.tweak) + ]) + + def as_tuple(self): + return (self.major, self.minor, self.patch, self.tweak) + + +class VersionType(object): + STABLE = "stable" + TESTING = "testing" + + +def build_version_description(version, version_type): + return "v" + version.get_version_string() + "-" + version_type + + +def _get_version_from_line(line): + _, ver_with_bracket = line.strip().split(' ') + return ver_with_bracket[:-1] + + +def get_version_from_repo(repo_path): + path_to_file = os.path.join(repo_path, FILE_WITH_VERSION_PATH) + major = 0 + minor = 0 + patch = 0 + tweak = 0 + version_revision = 0 + with open(path_to_file, 'r') as ver_file: + for line in ver_file: + if "VERSION_MAJOR" in line and "math" not in line and "SET" in line: + major = _get_version_from_line(line) + elif "VERSION_MINOR" in line and "math" not in line and "SET" in line: + minor = _get_version_from_line(line) + elif "VERSION_PATCH" in line and "math" not in line and "SET" in line: + patch = _get_version_from_line(line) + elif "VERSION_REVISION" in line and "math" not in line: + version_revision = _get_version_from_line(line) + return ClickHouseVersion(major, minor, patch, tweak, version_revision) + + +def _update_cmake_version(repo_path, version, sha, version_type): + cmd = """sed -i --follow-symlinks -e "s/SET(VERSION_REVISION [^) ]*/SET(VERSION_REVISION {revision}/g;" \ + -e "s/SET(VERSION_DESCRIBE [^) ]*/SET(VERSION_DESCRIBE {version_desc}/g;" \ + -e "s/SET(VERSION_GITHASH [^) ]*/SET(VERSION_GITHASH {sha}/g;" \ + -e "s/SET(VERSION_MAJOR [^) ]*/SET(VERSION_MAJOR {major}/g;" \ + -e "s/SET(VERSION_MINOR [^) ]*/SET(VERSION_MINOR {minor}/g;" \ + -e "s/SET(VERSION_PATCH [^) ]*/SET(VERSION_PATCH {patch}/g;" \ + -e "s/SET(VERSION_STRING [^) ]*/SET(VERSION_STRING {version_string}/g;" \ + {path}""".format( + revision=version.revision, + version_desc=build_version_description(version, version_type), + sha=sha, + major=version.major, + minor=version.minor, + patch=version.patch, + version_string=version.get_version_string(), + path=os.path.join(repo_path, FILE_WITH_VERSION_PATH), + ) + subprocess.check_call(cmd, shell=True) + + +def _update_changelog(repo_path, version): + cmd = """sed \ + -e "s/[@]VERSION_STRING[@]/{version_str}/g" \ + -e "s/[@]DATE[@]/{date}/g" \ + -e "s/[@]AUTHOR[@]/clickhouse-release/g" \ + -e "s/[@]EMAIL[@]/clickhouse-release@yandex-team.ru/g" \ + < {in_path} > {changelog_path} + """.format( + version_str=version.get_version_string(), + date=datetime.datetime.now().strftime("%a, %d %b %Y %H:%M:%S") + " +0300", + in_path=os.path.join(repo_path, CHANGELOG_IN_PATH), + changelog_path=os.path.join(repo_path, CHANGELOG_PATH) + ) + subprocess.check_call(cmd, shell=True) + +def _update_contributors(repo_path): + cmd = "cd {} && ./StorageSystemContributors.sh".format(os.path.join(repo_path, CONTRIBUTORS_SCRIPT_DIR)) + subprocess.check_call(cmd, shell=True) + +def _update_dockerfile(repo_path, version): + version_str_for_docker = '.'.join([str(version.major), str(version.minor), str(version.patch), '*']) + cmd = "ls -1 {path}/docker/*/Dockerfile | xargs sed -i -r -e 's/ARG version=.+$/ARG version='{ver}'/'".format(path=repo_path, ver=version_str_for_docker) + subprocess.check_call(cmd, shell=True) + +def update_version_local(repo_path, sha, version, version_type="testing"): + _update_contributors(repo_path) + _update_cmake_version(repo_path, version, sha, version_type) + _update_changelog(repo_path, version) + _update_dockerfile(repo_path, version) From 900a35936296a17e7bd7be7b08523478f65a3bab Mon Sep 17 00:00:00 2001 From: Vladimir Goncharov Date: Thu, 21 Oct 2021 16:39:10 +0300 Subject: [PATCH 299/919] Update json-functions.md english version --- .../sql-reference/functions/json-functions.md | 71 +++++++++++++++++++ 1 file changed, 71 insertions(+) diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index fc49d3a810d..875df345ad3 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -306,6 +306,77 @@ Result: └───────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` +## JSON_EXISTS(json, path) {#json-exists} + +If the value exists in the JSON document, `1` will be returned. + +If the value does not exist, `0` will be returned. + +Examples: + +``` sql +SELECT JSON_EXISTS('$.hello', '{"hello":1}'); +SELECT JSON_EXISTS('$.hello.world', '{"hello":{"world":1}}'); +SELECT JSON_EXISTS('$.hello[*]', '{"hello":["world"]}'); +SELECT JSON_EXISTS('$.hello[0]', '{"hello":["world"]}'); +``` + +!!! note "Note" + before version 21.11 the order of arguments was wrong, i.e. JSON_EXISTS(path, json) + +## JSON_QUERY(json, path) {#json-query} + +Parses a JSON and extract a value. + +If the value does not exist, an empty string will be returned. + +Example: + +``` sql +SELECT JSON_QUERY('$.hello', '{"hello":"world"}'); +SELECT JSON_QUERY('$.array[*][0 to 2, 4]', '{"array":[[0, 1, 2, 3, 4, 5], [0, -1, -2, -3, -4, -5]]}'); +SELECT JSON_QUERY('$.hello', '{"hello":2}'); +SELECT toTypeName(JSON_QUERY('$.hello', '{"hello":2}')); +``` + +Result: + +``` text +["world"] +[0, 1, 4, 0, -1, -4] +[2] +String +``` +!!! note "Note" + before version 21.11 the order of arguments was wrong, i.e. JSON_QUERY(path, json) + +## JSON_VALUE(json, path) {#json-value} + +Parses a JSON and extract a value. + +If the value does not exist, an empty string will be returned. + +Example: + +``` sql +SELECT JSON_VALUE('$.hello', '{"hello":"world"}'); +SELECT JSON_VALUE('$.array[*][0 to 2, 4]', '{"array":[[0, 1, 2, 3, 4, 5], [0, -1, -2, -3, -4, -5]]}'); +SELECT JSON_VALUE('$.hello', '{"hello":2}'); +SELECT toTypeName(JSON_VALUE('$.hello', '{"hello":2}')); +``` + +Result: + +``` text +"world" +0 +2 +String +``` + +!!! note "Note" + before version 21.11 the order of arguments was wrong, i.e. JSON_VALUE(path, json) + ## toJSONString {#tojsonstring} Serializes a value to its JSON representation. Various data types and nested structures are supported. From 11c0d08d1cb1a16180e3abe1b9f29d0bc4a2f173 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 21 Oct 2021 16:46:27 +0300 Subject: [PATCH 300/919] Fix test again --- tests/integration/test_storage_kafka/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 95175202b04..78066b36143 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2907,7 +2907,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster): SELECT {raw_message} as raw_message, _error as error, _topic as topic, _partition as partition, _offset as offset FROM test.kafka_{format_name} WHERE length(_error) > 0; '''.format(topic_name=topic_name, format_name=format_name, raw_message=raw_message, - extra_settings=format_opts.get('extra_settings') or '') + extra_settings=format_opts.get('extra_settings') or '')) for format_name, format_opts in list(all_formats.items()): logging.debug('Checking {format_name}') From 786cf9d1713287d4ce5d5061c045368d06d5ba9e Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 21 Oct 2021 17:16:42 +0300 Subject: [PATCH 301/919] Add docker container to build docs --- docker/docs/builder/Dockerfile | 50 ++++++++++++++++++++++++++++++++++ docker/images.json | 4 +++ 2 files changed, 54 insertions(+) create mode 100644 docker/docs/builder/Dockerfile diff --git a/docker/docs/builder/Dockerfile b/docker/docs/builder/Dockerfile new file mode 100644 index 00000000000..1fc27c5fc18 --- /dev/null +++ b/docker/docs/builder/Dockerfile @@ -0,0 +1,50 @@ +# docker build -t yandex/clickhouse-docs-build . +FROM ubuntu:20.04 + +ENV LANG=C.UTF-8 + +RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list + +RUN apt-get update \ + && DEBIAN_FRONTEND=noninteractive apt-get install --yes --no-install-recommends \ + python3-setuptools \ + virtualenv \ + wget \ + bash \ + python \ + curl \ + python3-requests \ + sudo \ + git \ + openssl \ + python3-pip \ + software-properties-common \ + language-pack-zh* \ + chinese* \ + fonts-arphic-ukai \ + fonts-arphic-uming \ + fonts-ipafont-mincho \ + fonts-ipafont-gothic \ + fonts-unfonts-core \ + xvfb \ + nodejs \ + npm \ + openjdk-11-jdk \ + && pip --no-cache-dir install scipy \ + && apt-get autoremove --yes \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* + +RUN wget 'https://github.com/wkhtmltopdf/packaging/releases/download/0.12.6-1/wkhtmltox_0.12.6-1.focal_amd64.deb' + +RUN npm i -g purify-css + +RUN pip3 install Babel==2.8.0 backports-abc==0.5 backports.functools-lru-cache==1.6.1 beautifulsoup4==4.9.1 Pygments>=2.7.4 \ +certifi==2020.4.5.2 chardet==3.0.4 click==7.1.2 closure==20191111 cssmin==0.2.0 future==0.18.2 htmlmin==0.1.12 \ +idna==2.10 Jinja2>=2.11.3 jinja2-highlight==0.6.1 jsmin==3.0.0 livereload==2.6.2 Markdown==3.3.2 MarkupSafe==2.0.1 \ +mkdocs==1.2.3 mkdocs-htmlproofer-plugin==0.0.3 mkdocs-macros-plugin==0.4.20 nltk==3.5 nose==1.3.7 protobuf==3.14.0 \ +numpy==1.21.2 pymdown-extensions==9.0 python-slugify==4.0.1 PyYAML==5.4.1 repackage==0.7.3 requests==2.25.1 \ +singledispatch==3.4.0.3 six==1.15.0 soupsieve==2.0.1 termcolor==1.1.0 tornado==6.1 Unidecode==1.1.1 urllib3>=1.26.5 + + +COPY * / diff --git a/docker/images.json b/docker/images.json index 3e8adda868c..1cb999bd16f 100644 --- a/docker/images.json +++ b/docker/images.json @@ -166,5 +166,9 @@ "docker/test/keeper-jepsen": { "name": "clickhouse/keeper-jepsen-test", "dependent": [] + }, + "docker/docs/builder": { + "name": "clickhouse/docs-builder", + "dependent": [] } } From a3a780bbf5573be954f53247b24e1fde75533f9a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 21 Oct 2021 17:17:53 +0300 Subject: [PATCH 302/919] Dictionaries read support multiple threads --- src/Dictionaries/CacheDictionary.cpp | 27 ++- src/Dictionaries/CacheDictionary.h | 2 +- src/Dictionaries/DictionaryHelpers.h | 10 + src/Dictionaries/DictionarySource.cpp | 256 ++++++++++----------- src/Dictionaries/DictionarySource.h | 111 ++++----- src/Dictionaries/DictionarySourceBase.cpp | 21 -- src/Dictionaries/DictionarySourceBase.h | 22 -- src/Dictionaries/DirectDictionary.cpp | 2 +- src/Dictionaries/DirectDictionary.h | 2 +- src/Dictionaries/FlatDictionary.cpp | 18 +- src/Dictionaries/FlatDictionary.h | 2 +- src/Dictionaries/HashedArrayDictionary.cpp | 22 +- src/Dictionaries/HashedArrayDictionary.h | 2 +- src/Dictionaries/HashedDictionary.cpp | 24 +- src/Dictionaries/HashedDictionary.h | 2 +- src/Dictionaries/ICacheDictionaryStorage.h | 3 +- src/Dictionaries/IDictionary.h | 4 +- src/Dictionaries/IPAddressDictionary.cpp | 47 ++-- src/Dictionaries/IPAddressDictionary.h | 2 +- src/Dictionaries/PolygonDictionary.cpp | 2 +- src/Dictionaries/PolygonDictionary.h | 2 +- src/Dictionaries/RangeDictionarySource.h | 219 ------------------ src/Dictionaries/RangeHashedDictionary.cpp | 120 ++++++---- src/Dictionaries/RangeHashedDictionary.h | 9 +- src/Storages/StorageDictionary.cpp | 4 +- 25 files changed, 392 insertions(+), 543 deletions(-) delete mode 100644 src/Dictionaries/DictionarySourceBase.cpp delete mode 100644 src/Dictionaries/DictionarySourceBase.h delete mode 100644 src/Dictionaries/RangeDictionarySource.h diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index f7e9ce6624c..723457fba5b 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -484,24 +484,37 @@ MutableColumns CacheDictionary::aggregateColumns( } template -Pipe CacheDictionary::read(const Names & column_names, size_t max_block_size) const +Pipe CacheDictionary::read(const Names & column_names, size_t max_block_size, size_t num_streams) const { - Pipe pipe; - std::optional data; + ColumnsWithTypeAndName key_columns; + { /// Write lock on storage const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs}; - if constexpr (dictionary_key_type == DictionaryKeyType::Simple) - data.emplace(shared_from_this(), cache_storage_ptr->getCachedSimpleKeys(), column_names); + { + auto keys = cache_storage_ptr->getCachedSimpleKeys(); + key_columns = {ColumnWithTypeAndName(getColumnFromPODArray(keys), std::make_shared(), dict_struct.id->name)}; + } else { auto keys = cache_storage_ptr->getCachedComplexKeys(); - data.emplace(shared_from_this(), keys, column_names); + key_columns = deserializeColumnsWithTypeAndNameFromKeys(dict_struct, keys, 0, keys.size()); } } - return Pipe(std::make_shared(std::move(*data), max_block_size)); + std::shared_ptr dictionary = shared_from_this(); + auto coordinator = std::make_shared(dictionary, column_names, std::move(key_columns), max_block_size); + + Pipes pipes; + + for (size_t i = 0; i < num_streams; ++i) + { + auto source = std::make_shared(coordinator); + pipes.emplace_back(Pipe(std::move(source))); + } + + return Pipe::unitePipes(std::move(pipes)); } template diff --git a/src/Dictionaries/CacheDictionary.h b/src/Dictionaries/CacheDictionary.h index 8b3b4dbd30f..d2c2ed9a212 100644 --- a/src/Dictionaries/CacheDictionary.h +++ b/src/Dictionaries/CacheDictionary.h @@ -136,7 +136,7 @@ public: ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; - Pipe read(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size, size_t num_streams) const override; std::exception_ptr getLastException() const override; diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index 65c40898983..26fbb6f193f 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -668,6 +668,16 @@ static ColumnPtr getColumnFromPODArray(const PaddedPODArray & array) return column_vector; } +template +static ColumnPtr getColumnFromPODArray(const PaddedPODArray & array, size_t start, size_t length) +{ + auto column_vector = ColumnVector::create(); + column_vector->getData().reserve(length); + column_vector->getData().insert(array.begin() + start, array.begin() + start + length); + + return column_vector; +} + } diff --git a/src/Dictionaries/DictionarySource.cpp b/src/Dictionaries/DictionarySource.cpp index a164543e1ff..78fdd4e6098 100644 --- a/src/Dictionaries/DictionarySource.cpp +++ b/src/Dictionaries/DictionarySource.cpp @@ -4,158 +4,150 @@ namespace DB { -namespace ErrorCodes +bool DictionarySourceCoordinator::getKeyColumnsNextRangeToRead(ColumnsWithTypeAndName & key_columns, ColumnsWithTypeAndName & data_columns) { - extern const int LOGICAL_ERROR; + size_t read_block_index = parallel_read_block_index++; + + size_t start = read_block_index * max_block_size; + size_t end = (read_block_index + 1) * max_block_size; + + size_t keys_size = key_columns_with_type[0].column->size(); + + if (start >= keys_size) + return false; + + end = std::min(end, keys_size); + size_t length = end - start; + + key_columns = cutColumns(key_columns_with_type, start, length); + data_columns = cutColumns(data_columns_with_type, start, length); + + return true; } -DictionarySourceData::DictionarySourceData( - std::shared_ptr dictionary_, PaddedPODArray && ids_, const Names & column_names_) - : num_rows(ids_.size()) - , dictionary(dictionary_) - , column_names(column_names_.begin(), column_names_.end()) - , ids(std::move(ids_)) - , key_type(DictionaryInputStreamKeyType::Id) +void DictionarySourceCoordinator::initialize(const Names & column_names) { -} + ColumnsWithTypeAndName columns_with_type; -DictionarySourceData::DictionarySourceData( - std::shared_ptr dictionary_, - const PaddedPODArray & keys, - const Names & column_names_) - : num_rows(keys.size()) - , dictionary(dictionary_) - , column_names(column_names_.begin(), column_names_.end()) - , key_type(DictionaryInputStreamKeyType::ComplexKey) -{ - const DictionaryStructure & dictionary_structure = dictionary->getStructure(); - key_columns = deserializeColumnsWithTypeAndNameFromKeys(dictionary_structure, keys, 0, keys.size()); -} + const auto & dictionary_structure = dictionary->getStructure(); -DictionarySourceData::DictionarySourceData( - std::shared_ptr dictionary_, - const Columns & data_columns_, - const Names & column_names_, - GetColumnsFunction && get_key_columns_function_, - GetColumnsFunction && get_view_columns_function_) - : num_rows(data_columns_.front()->size()) - , dictionary(dictionary_) - , column_names(column_names_.begin(), column_names_.end()) - , data_columns(data_columns_) - , get_key_columns_function(std::move(get_key_columns_function_)) - , get_view_columns_function(std::move(get_view_columns_function_)) - , key_type(DictionaryInputStreamKeyType::Callback) -{ -} - -Block DictionarySourceData::getBlock(size_t start, size_t length) const -{ - /// TODO: Rewrite - switch (key_type) + for (const auto & column_name : column_names) { - case DictionaryInputStreamKeyType::ComplexKey: + ColumnWithTypeAndName column_with_type; + column_with_type.name = column_name; + + auto it = dictionary_structure.attribute_name_to_index.find(column_name); + if (it == dictionary_structure.attribute_name_to_index.end()) { - Columns columns; - ColumnsWithTypeAndName view_columns; - columns.reserve(key_columns.size()); - for (const auto & key_column : key_columns) + if (dictionary_structure.id.has_value() && column_name == dictionary_structure.id->name) { - ColumnPtr column = key_column.column->cut(start, length); - columns.emplace_back(column); - view_columns.emplace_back(column, key_column.type, key_column.name); + column_with_type.type = std::make_shared(); } - return fillBlock({}, columns, {}, std::move(view_columns)); - } - - case DictionaryInputStreamKeyType::Id: - { - PaddedPODArray ids_to_fill(ids.begin() + start, ids.begin() + start + length); - return fillBlock(ids_to_fill, {}, {}, {}); - } - - case DictionaryInputStreamKeyType::Callback: - { - Columns columns; - columns.reserve(data_columns.size()); - for (const auto & data_column : data_columns) - columns.push_back(data_column->cut(start, length)); - const DictionaryStructure & dictionaty_structure = dictionary->getStructure(); - const auto & attributes = *dictionaty_structure.key; - ColumnsWithTypeAndName keys_with_type_and_name = get_key_columns_function(columns, attributes); - ColumnsWithTypeAndName view_with_type_and_name = get_view_columns_function(columns, attributes); - DataTypes types; - columns.clear(); - for (const auto & key_column : keys_with_type_and_name) + else if (dictionary_structure.range_min.has_value() && column_name == dictionary_structure.range_min->name) { - columns.push_back(key_column.column); - types.push_back(key_column.type); + column_with_type.type = dictionary_structure.range_min->type; } - return fillBlock({}, columns, types, std::move(view_with_type_and_name)); - } - } - - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected DictionaryInputStreamKeyType."); -} - -Block DictionarySourceData::fillBlock( - const PaddedPODArray & ids_to_fill, - const Columns & keys, - const DataTypes & types, - ColumnsWithTypeAndName && view) const -{ - DataTypes data_types = types; - ColumnsWithTypeAndName block_columns; - - data_types.reserve(keys.size()); - const DictionaryStructure & dictionary_structure = dictionary->getStructure(); - if (data_types.empty() && dictionary_structure.key) - for (const auto & key : *dictionary_structure.key) - data_types.push_back(key.type); - - for (const auto & column : view) - if (column_names.find(column.name) != column_names.end()) - block_columns.push_back(column); - - const DictionaryStructure & structure = dictionary->getStructure(); - ColumnPtr ids_column = getColumnFromPODArray(ids_to_fill); - - if (structure.id && column_names.find(structure.id->name) != column_names.end()) - { - block_columns.emplace_back(ids_column, std::make_shared(), structure.id->name); - } - - auto dictionary_key_type = dictionary->getKeyType(); - - for (const auto & attribute : structure.attributes) - { - if (column_names.find(attribute.name) != column_names.end()) - { - ColumnPtr column; - - if (dictionary_key_type == DictionaryKeyType::Simple) + else if (dictionary_structure.range_max.has_value() && column_name == dictionary_structure.range_max->name) { - column = dictionary->getColumn( - attribute.name, - attribute.type, - {ids_column}, - {std::make_shared()}, - nullptr /* default_values_column */); + column_with_type.type = dictionary_structure.range_max->type; } else { - column = dictionary->getColumn( - attribute.name, - attribute.type, - keys, - data_types, - nullptr /* default_values_column*/); + const auto & dictionary_key_attributes = *dictionary_structure.key; + for (const auto & attribute : dictionary_key_attributes) + { + if (column_name == attribute.name) + { + column_with_type.type = attribute.type; + break; + } + } } - - block_columns.emplace_back(column, attribute.type, attribute.name); } + else + { + const auto & attribute = dictionary_structure.attributes[it->second]; + attributes_names_to_read.emplace_back(attribute.name); + attributes_types_to_read.emplace_back(attribute.type); + attributes_default_values_columns.emplace_back(nullptr); + + column_with_type.type = attribute.type; + } + + column_with_type.column = column_with_type.type->createColumn(); + columns_with_type.emplace_back(std::move(column_with_type)); } - return Block(block_columns); + header = Block(std::move(columns_with_type)); +} + +ColumnsWithTypeAndName +DictionarySourceCoordinator::cutColumns(const ColumnsWithTypeAndName & columns_with_type, size_t start, size_t length) +{ + ColumnsWithTypeAndName result; + result.reserve(columns_with_type.size()); + + for (const auto & column_with_type : columns_with_type) + { + ColumnWithTypeAndName result_column_with_type; + + result_column_with_type.column = column_with_type.column->cut(start, length); + result_column_with_type.type = column_with_type.type; + result_column_with_type.name = column_with_type.name; + + result.emplace_back(std::move(result_column_with_type)); + } + + return result; +} + + +Chunk DictionarySource::generate() +{ + ColumnsWithTypeAndName key_columns_to_read; + ColumnsWithTypeAndName data_columns; + + if (!coordinator->getKeyColumnsNextRangeToRead(key_columns_to_read, data_columns)) + return {}; + + const auto & header = coordinator->getHeader(); + + std::vector result_columns; + result_columns.reserve(header.columns()); + + std::vector key_columns; + std::vector key_types; + + key_columns.reserve(key_columns_to_read.size()); + key_types.reserve(key_columns_to_read.size()); + + for (const auto & key_column_to_read : key_columns_to_read) + { + key_columns.emplace_back(key_column_to_read.column); + key_types.emplace_back(key_column_to_read.type); + + if (header.has(key_column_to_read.name)) + result_columns.emplace_back(key_column_to_read.column); + } + + for (const auto & data_column : data_columns) + { + if (header.has(data_column.name)) + result_columns.emplace_back(data_column.column); + } + + const auto & attributes_names_to_read = coordinator->getAttributesNamesToRead(); + const auto & attributes_types_to_read = coordinator->getAttributesTypesToRead(); + const auto & attributes_default_values_columns = coordinator->getAttributesDefaultValuesColumns(); + + const auto & dictionary = coordinator->getDictionary(); + auto attributes_columns = dictionary->getColumns( + attributes_names_to_read, attributes_types_to_read, key_columns, key_types, attributes_default_values_columns); + + result_columns.insert(result_columns.end(), attributes_columns.begin(), attributes_columns.end()); + + size_t rows_size = result_columns[0]->size(); + return Chunk(result_columns, rows_size); } } diff --git a/src/Dictionaries/DictionarySource.h b/src/Dictionaries/DictionarySource.h index cd4b3120ac0..f54290a08f9 100644 --- a/src/Dictionaries/DictionarySource.h +++ b/src/Dictionaries/DictionarySource.h @@ -7,82 +7,87 @@ #include #include #include +#include #include #include -#include namespace DB { -class DictionarySourceData +class DictionarySourceCoordinator { public: - DictionarySourceData( - std::shared_ptr dictionary, - PaddedPODArray && ids, - const Names & column_names); - DictionarySourceData( - std::shared_ptr dictionary, - const PaddedPODArray & keys, - const Names & column_names); - - using GetColumnsFunction = std::function & attributes)>; - - // Used to separate key columns format for storage and view. - // Calls get_key_columns_function to get key column for dictionary get function call - // and get_view_columns_function to get key representation. - // Now used in trie dictionary, where columns are stored as ip and mask, and are showed as string - DictionarySourceData( - std::shared_ptr dictionary, - const Columns & data_columns, + explicit DictionarySourceCoordinator( + std::shared_ptr dictionary_, const Names & column_names, - GetColumnsFunction && get_key_columns_function, - GetColumnsFunction && get_view_columns_function); + ColumnsWithTypeAndName && key_columns_with_type_, + size_t max_block_size_) + : dictionary(std::move(dictionary_)) + , key_columns_with_type(std::move(key_columns_with_type_)) + , max_block_size(max_block_size_) + { + initialize(column_names); + } - Block getBlock(size_t start, size_t length) const; - size_t getNumRows() const { return num_rows; } + explicit DictionarySourceCoordinator( + std::shared_ptr dictionary_, + const Names & column_names, + ColumnsWithTypeAndName && key_columns_with_type_, + ColumnsWithTypeAndName && data_columns_with_type_, + size_t max_block_size_) + : dictionary(std::move(dictionary_)) + , key_columns_with_type(std::move(key_columns_with_type_)) + , data_columns_with_type(std::move(data_columns_with_type_)) + , max_block_size(max_block_size_) + { + initialize(column_names); + } + + bool getKeyColumnsNextRangeToRead(ColumnsWithTypeAndName & key_columns, ColumnsWithTypeAndName & data_columns); + + const Block & getHeader() const { return header; } + + const std::vector & getAttributesNamesToRead() const { return attributes_names_to_read; } + + const std::vector & getAttributesTypesToRead() const { return attributes_types_to_read; } + + const std::vector & getAttributesDefaultValuesColumns() const { return attributes_default_values_columns; } + + const std::shared_ptr & getDictionary() const { return dictionary; } private: - Block fillBlock( - const PaddedPODArray & ids_to_fill, - const Columns & keys, - const DataTypes & types, - ColumnsWithTypeAndName && view) const; + void initialize(const Names & column_names); + + static ColumnsWithTypeAndName cutColumns(const ColumnsWithTypeAndName & columns_with_type, size_t start, size_t length); - const size_t num_rows; std::shared_ptr dictionary; - std::unordered_set column_names; - PaddedPODArray ids; - ColumnsWithTypeAndName key_columns; - - Columns data_columns; - GetColumnsFunction get_key_columns_function; - GetColumnsFunction get_view_columns_function; - - enum class DictionaryInputStreamKeyType - { - Id, - ComplexKey, - Callback - }; - - DictionaryInputStreamKeyType key_type; + ColumnsWithTypeAndName key_columns_with_type; + ColumnsWithTypeAndName data_columns_with_type; + Block header; + std::vector attributes_names_to_read; + std::vector attributes_types_to_read; + std::vector attributes_default_values_columns; + const size_t max_block_size; + std::atomic parallel_read_block_index = 0; }; -class DictionarySource final : public DictionarySourceBase +class DictionarySource : public SourceWithProgress { public: - DictionarySource(DictionarySourceData data_, UInt64 max_block_size) - : DictionarySourceBase(data_.getBlock(0, 0), data_.getNumRows(), max_block_size) - , data(std::move(data_)) - {} + explicit DictionarySource(std::shared_ptr coordinator_) + : SourceWithProgress(coordinator_->getHeader()), coordinator(std::move(coordinator_)) + { + } + +private: String getName() const override { return "DictionarySource"; } - Block getBlock(size_t start, size_t length) const override { return data.getBlock(start, length); } - DictionarySourceData data; + Chunk generate() override; + + std::shared_ptr coordinator; }; } diff --git a/src/Dictionaries/DictionarySourceBase.cpp b/src/Dictionaries/DictionarySourceBase.cpp deleted file mode 100644 index cc420b33144..00000000000 --- a/src/Dictionaries/DictionarySourceBase.cpp +++ /dev/null @@ -1,21 +0,0 @@ -#include "DictionarySourceBase.h" - -namespace DB -{ -DictionarySourceBase::DictionarySourceBase(const Block & header, size_t rows_count_, size_t max_block_size_) - : SourceWithProgress(header), rows_count(rows_count_), max_block_size(max_block_size_) -{ -} - -Chunk DictionarySourceBase::generate() -{ - if (next_row == rows_count) - return {}; - - size_t size = std::min(max_block_size, rows_count - next_row); - auto block = getBlock(next_row, size); - next_row += size; - return Chunk(block.getColumns(), size); -} - -} diff --git a/src/Dictionaries/DictionarySourceBase.h b/src/Dictionaries/DictionarySourceBase.h deleted file mode 100644 index 1533cba82f6..00000000000 --- a/src/Dictionaries/DictionarySourceBase.h +++ /dev/null @@ -1,22 +0,0 @@ -#pragma once - -#include - -namespace DB -{ -class DictionarySourceBase : public SourceWithProgress -{ -protected: - DictionarySourceBase(const Block & header, size_t rows_count_, size_t max_block_size_); - - virtual Block getBlock(size_t start, size_t length) const = 0; - -private: - const size_t rows_count; - const size_t max_block_size; - size_t next_row = 0; - - Chunk generate() override; -}; - -} diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index f4f33439638..551f485e5bb 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -290,7 +290,7 @@ Pipe DirectDictionary::getSourceBlockInputStream( } template -Pipe DirectDictionary::read(const Names & /* column_names */, size_t /* max_block_size */) const +Pipe DirectDictionary::read(const Names & /* column_names */, size_t /* max_block_size */, size_t /* num_streams */) const { return source_ptr->loadAll(); } diff --git a/src/Dictionaries/DirectDictionary.h b/src/Dictionaries/DirectDictionary.h index ebe5f5fbbc7..edf4c8d1d9a 100644 --- a/src/Dictionaries/DirectDictionary.h +++ b/src/Dictionaries/DirectDictionary.h @@ -96,7 +96,7 @@ public: ColumnPtr in_key_column, const DataTypePtr & key_type) const override; - Pipe read(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size, size_t num_streams) const override; private: Pipe getSourceBlockInputStream(const Columns & key_columns, const PaddedPODArray & requested_keys) const; diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index a7cf69bb14d..b6c5f10564b 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -538,7 +538,7 @@ void FlatDictionary::setAttributeValue(Attribute & attribute, const UInt64 key, callOnDictionaryAttributeType(attribute.type, type_call); } -Pipe FlatDictionary::read(const Names & column_names, size_t max_block_size) const +Pipe FlatDictionary::read(const Names & column_names, size_t max_block_size, size_t num_streams) const { const auto keys_count = loaded_keys.size(); @@ -549,8 +549,20 @@ Pipe FlatDictionary::read(const Names & column_names, size_t max_block_size) con if (loaded_keys[key_index]) keys.push_back(key_index); - return Pipe(std::make_shared( - DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); + ColumnsWithTypeAndName key_columns = {ColumnWithTypeAndName(getColumnFromPODArray(keys), std::make_shared(), dict_struct.id->name)}; + + std::shared_ptr dictionary = shared_from_this(); + auto coordinator = std::make_shared(dictionary, column_names, std::move(key_columns), max_block_size); + + Pipes pipes; + + for (size_t i = 0; i < num_streams; ++i) + { + auto source = std::make_shared(coordinator); + pipes.emplace_back(Pipe(std::move(source))); + } + + return Pipe::unitePipes(std::move(pipes)); } void registerDictionaryFlat(DictionaryFactory & factory) diff --git a/src/Dictionaries/FlatDictionary.h b/src/Dictionaries/FlatDictionary.h index 196194ddb21..5c3a1d634d8 100644 --- a/src/Dictionaries/FlatDictionary.h +++ b/src/Dictionaries/FlatDictionary.h @@ -97,7 +97,7 @@ public: const DataTypePtr & key_type, size_t level) const override; - Pipe read(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size, size_t num_streams) const override; private: template diff --git a/src/Dictionaries/HashedArrayDictionary.cpp b/src/Dictionaries/HashedArrayDictionary.cpp index 3c02f377c3e..426857e055f 100644 --- a/src/Dictionaries/HashedArrayDictionary.cpp +++ b/src/Dictionaries/HashedArrayDictionary.cpp @@ -634,7 +634,7 @@ void HashedArrayDictionary::calculateBytesAllocated() } template -Pipe HashedArrayDictionary::read(const Names & column_names, size_t max_block_size) const +Pipe HashedArrayDictionary::read(const Names & column_names, size_t max_block_size, size_t num_streams) const { PaddedPODArray keys; keys.reserve(key_attribute.container.size()); @@ -642,7 +642,25 @@ Pipe HashedArrayDictionary::read(const Names & column_names for (auto & [key, _] : key_attribute.container) keys.emplace_back(key); - return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); + ColumnsWithTypeAndName key_columns; + + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) + key_columns = {ColumnWithTypeAndName(getColumnFromPODArray(keys), std::make_shared(), dict_struct.id->name)}; + else + key_columns = deserializeColumnsWithTypeAndNameFromKeys(dict_struct, keys, 0, keys.size()); + + std::shared_ptr dictionary = shared_from_this(); + auto coordinator = std::make_shared(dictionary, column_names, std::move(key_columns), max_block_size); + + Pipes pipes; + + for (size_t i = 0; i < num_streams; ++i) + { + auto source = std::make_shared(coordinator); + pipes.emplace_back(Pipe(std::move(source))); + } + + return Pipe::unitePipes(std::move(pipes)); } template class HashedArrayDictionary; diff --git a/src/Dictionaries/HashedArrayDictionary.h b/src/Dictionaries/HashedArrayDictionary.h index 053813bdc44..0af3fb65581 100644 --- a/src/Dictionaries/HashedArrayDictionary.h +++ b/src/Dictionaries/HashedArrayDictionary.h @@ -109,7 +109,7 @@ public: const DataTypePtr & key_type, size_t level) const override; - Pipe read(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size, size_t num_streams) const override; private: diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 44a0d8a62c1..728d8c1c352 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -6,6 +6,8 @@ #include #include +#include + #include #include #include @@ -640,7 +642,7 @@ void HashedDictionary::calculateBytesAllocated() } template -Pipe HashedDictionary::read(const Names & column_names, size_t max_block_size) const +Pipe HashedDictionary::read(const Names & column_names, size_t max_block_size, size_t num_streams) const { PaddedPODArray keys; @@ -669,7 +671,25 @@ Pipe HashedDictionary::read(const Names & column_na }); } - return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); + ColumnsWithTypeAndName key_columns; + + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) + key_columns = {ColumnWithTypeAndName(getColumnFromPODArray(keys), std::make_shared(), dict_struct.id->name)}; + else + key_columns = deserializeColumnsWithTypeAndNameFromKeys(dict_struct, keys, 0, keys.size()); + + std::shared_ptr dictionary = shared_from_this(); + auto coordinator = std::make_shared(dictionary, column_names, std::move(key_columns), max_block_size); + + Pipes pipes; + + for (size_t i = 0; i < num_streams; ++i) + { + auto source = std::make_shared(coordinator); + pipes.emplace_back(Pipe(std::move(source))); + } + + return Pipe::unitePipes(std::move(pipes)); } template diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index d1e1f681fa1..23919c009c5 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -115,7 +115,7 @@ public: const DataTypePtr & key_type, size_t level) const override; - Pipe read(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size, size_t num_streams) const override; private: template diff --git a/src/Dictionaries/ICacheDictionaryStorage.h b/src/Dictionaries/ICacheDictionaryStorage.h index 72b3ef76f11..b094d76a9a7 100644 --- a/src/Dictionaries/ICacheDictionaryStorage.h +++ b/src/Dictionaries/ICacheDictionaryStorage.h @@ -106,7 +106,8 @@ public: /// Insert default keys virtual void insertDefaultKeys(const PaddedPODArray & keys) = 0; - /// Return cached simple keys + /// Return cached complex keys. + /// It is client responsibility to ensure keys proper lifetime. virtual PaddedPODArray getCachedComplexKeys() const = 0; /// Return size of keys in storage diff --git a/src/Dictionaries/IDictionary.h b/src/Dictionaries/IDictionary.h index b40dc418c66..66e35c8fa12 100644 --- a/src/Dictionaries/IDictionary.h +++ b/src/Dictionaries/IDictionary.h @@ -110,7 +110,7 @@ struct IDictionary : public IExternalLoadable */ virtual DictionaryKeyType getKeyType() const = 0; - virtual DictionarySpecialKeyType getSpecialKeyType() const { return DictionarySpecialKeyType::None;} + virtual DictionarySpecialKeyType getSpecialKeyType() const { return DictionarySpecialKeyType::None; } /** Subclass must validate key columns and keys types * and return column representation of dictionary attribute. @@ -194,7 +194,7 @@ struct IDictionary : public IExternalLoadable getDictionaryID().getNameForLogs()); } - virtual Pipe read(const Names & column_names, size_t max_block_size) const = 0; + virtual Pipe read(const Names & column_names, size_t max_block_size, size_t num_streams) const = 0; bool supportUpdates() const override { return true; } diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index c37f90c50c6..3314ce42ecb 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -807,6 +807,7 @@ Columns IPAddressDictionary::getKeyColumns() const key_ip_column->insertData(data, IPV6_BINARY_LENGTH); key_mask_column->insertValue(mask_column[row]); } + return {std::move(key_ip_column), std::move(key_mask_column)}; } @@ -834,37 +835,53 @@ static auto keyViewGetter() }; } -Pipe IPAddressDictionary::read(const Names & column_names, size_t max_block_size) const +Pipe IPAddressDictionary::read(const Names & column_names, size_t max_block_size, size_t num_streams) const { const bool is_ipv4 = std::get_if(&ip_column) != nullptr; - auto get_keys = [is_ipv4](const Columns & columns, const std::vector & dict_attributes) + auto get_key_columns = [is_ipv4](const Columns & columns) { - const auto & attr = dict_attributes.front(); - std::shared_ptr key_typ; + std::shared_ptr key_type; if (is_ipv4) - key_typ = std::make_shared(); + key_type = std::make_shared(); else - key_typ = std::make_shared(IPV6_BINARY_LENGTH); + key_type = std::make_shared(IPV6_BINARY_LENGTH); return ColumnsWithTypeAndName({ - ColumnWithTypeAndName(columns.front(), key_typ, attr.name), - ColumnWithTypeAndName(columns.back(), std::make_shared(), attr.name + ".mask") + ColumnWithTypeAndName(columns.front(), key_type, ""), + ColumnWithTypeAndName(columns.back(), std::make_shared(), "") }); }; + auto key_columns = getKeyColumns(); + + auto key_columns_with_type = get_key_columns(key_columns); + + ColumnsWithTypeAndName view_columns; + if (is_ipv4) { auto get_view = keyViewGetter, true>(); - return Pipe(std::make_shared( - DictionarySourceData(shared_from_this(), getKeyColumns(), column_names, std::move(get_keys), std::move(get_view)), - max_block_size)); + view_columns = get_view(key_columns, dict_struct.attributes); + } + else + { + auto get_view = keyViewGetter(); + view_columns = get_view(key_columns, dict_struct.attributes); } - auto get_view = keyViewGetter(); - return Pipe(std::make_shared( - DictionarySourceData(shared_from_this(), getKeyColumns(), column_names, std::move(get_keys), std::move(get_view)), - max_block_size)); + std::shared_ptr dictionary = shared_from_this(); + auto coordinator = std::make_shared(dictionary, column_names, std::move(key_columns_with_type), std::move(view_columns), max_block_size); + + Pipes pipes; + + for (size_t i = 0; i < num_streams; ++i) + { + auto source = std::make_shared(coordinator); + pipes.emplace_back(Pipe(std::move(source))); + } + + return Pipe::unitePipes(std::move(pipes)); } IPAddressDictionary::RowIdxConstIter IPAddressDictionary::ipNotFound() const diff --git a/src/Dictionaries/IPAddressDictionary.h b/src/Dictionaries/IPAddressDictionary.h index 2103c702493..ed0d8692d21 100644 --- a/src/Dictionaries/IPAddressDictionary.h +++ b/src/Dictionaries/IPAddressDictionary.h @@ -78,7 +78,7 @@ public: ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; - Pipe read(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size, size_t num_streams) const override; private: diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index 0a9ba1f5ea8..8c31be329aa 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -118,7 +118,7 @@ ColumnPtr IPolygonDictionary::getColumn( return result; } -Pipe IPolygonDictionary::read(const Names & column_names, size_t) const +Pipe IPolygonDictionary::read(const Names & column_names, size_t, size_t) const { if (!configuration.store_polygon_key_column) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, diff --git a/src/Dictionaries/PolygonDictionary.h b/src/Dictionaries/PolygonDictionary.h index 00a151d546f..346160c342f 100644 --- a/src/Dictionaries/PolygonDictionary.h +++ b/src/Dictionaries/PolygonDictionary.h @@ -106,7 +106,7 @@ public: ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; - Pipe read(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size, size_t num_streams) const override; /** Single coordinate type. */ using Coord = Float32; diff --git a/src/Dictionaries/RangeDictionarySource.h b/src/Dictionaries/RangeDictionarySource.h deleted file mode 100644 index dcc5b1ea3b8..00000000000 --- a/src/Dictionaries/RangeDictionarySource.h +++ /dev/null @@ -1,219 +0,0 @@ -#pragma once -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -template -class RangeDictionarySourceData -{ -public: - - using KeyType = std::conditional_t; - - RangeDictionarySourceData( - std::shared_ptr dictionary, - const Names & column_names, - PaddedPODArray && keys, - PaddedPODArray && start_dates, - PaddedPODArray && end_dates); - - Block getBlock(size_t start, size_t length) const; - size_t getNumRows() const { return keys.size(); } - -private: - - Block fillBlock( - const PaddedPODArray & keys_to_fill, - const PaddedPODArray & block_start_dates, - const PaddedPODArray & block_end_dates, - size_t start, - size_t end) const; - - PaddedPODArray makeDateKeys( - const PaddedPODArray & block_start_dates, - const PaddedPODArray & block_end_dates) const; - - std::shared_ptr dictionary; - NameSet column_names; - PaddedPODArray keys; - PaddedPODArray start_dates; - PaddedPODArray end_dates; -}; - - -template -RangeDictionarySourceData::RangeDictionarySourceData( - std::shared_ptr dictionary_, - const Names & column_names_, - PaddedPODArray && keys, - PaddedPODArray && block_start_dates, - PaddedPODArray && block_end_dates) - : dictionary(dictionary_) - , column_names(column_names_.begin(), column_names_.end()) - , keys(std::move(keys)) - , start_dates(std::move(block_start_dates)) - , end_dates(std::move(block_end_dates)) -{ -} - -template -Block RangeDictionarySourceData::getBlock(size_t start, size_t length) const -{ - PaddedPODArray block_keys; - PaddedPODArray block_start_dates; - PaddedPODArray block_end_dates; - block_keys.reserve(length); - block_start_dates.reserve(length); - block_end_dates.reserve(length); - - for (size_t index = start; index < start + length; ++index) - { - block_keys.push_back(keys[index]); - block_start_dates.push_back(start_dates[index]); - block_end_dates.push_back(end_dates[index]); - } - - return fillBlock(block_keys, block_start_dates, block_end_dates, start, start + length); -} - -template -PaddedPODArray RangeDictionarySourceData::makeDateKeys( - const PaddedPODArray & block_start_dates, - const PaddedPODArray & block_end_dates) const -{ - PaddedPODArray keys(block_start_dates.size()); - - for (size_t i = 0; i < keys.size(); ++i) - { - if (Range::isCorrectDate(block_start_dates[i])) - keys[i] = block_start_dates[i]; - else - keys[i] = block_end_dates[i]; - } - - return keys; -} - - -template -Block RangeDictionarySourceData::fillBlock( - const PaddedPODArray & keys_to_fill, - const PaddedPODArray & block_start_dates, - const PaddedPODArray & block_end_dates, - size_t start, - size_t end) const -{ - ColumnsWithTypeAndName columns; - const DictionaryStructure & dictionary_structure = dictionary->getStructure(); - - DataTypes keys_types; - Columns keys_columns; - Strings keys_names = dictionary_structure.getKeysNames(); - - if constexpr (dictionary_key_type == DictionaryKeyType::Simple) - { - keys_columns = {getColumnFromPODArray(keys_to_fill)}; - keys_types = {std::make_shared()}; - } - else - { - for (const auto & attribute : *dictionary_structure.key) - keys_types.emplace_back(attribute.type); - - auto deserialized_columns = deserializeColumnsFromKeys(dictionary_structure, keys, start, end); - for (auto & deserialized_column : deserialized_columns) - keys_columns.emplace_back(std::move(deserialized_column)); - } - - size_t keys_size = keys_names.size(); - - assert(keys_columns.size() == keys_size); - assert(keys_types.size() == keys_size); - - for (size_t i = 0; i < keys_size; ++i) - { - auto & key_name = keys_names[i]; - - if (column_names.find(key_name) != column_names.end()) - columns.emplace_back(keys_columns[i], keys_types[i], key_name); - } - - auto date_key = makeDateKeys(block_start_dates, block_end_dates); - auto date_column = getColumnFromPODArray(date_key); - keys_columns.emplace_back(std::move(date_column)); - keys_types.emplace_back(std::make_shared()); - - const auto & range_min_column_name = dictionary_structure.range_min->name; - if (column_names.find(range_min_column_name) != column_names.end()) - { - auto range_min_column = getColumnFromPODArray(block_start_dates); - columns.emplace_back(range_min_column, dictionary_structure.range_max->type, range_min_column_name); - } - - const auto & range_max_column_name = dictionary_structure.range_max->name; - if (column_names.find(range_max_column_name) != column_names.end()) - { - auto range_max_column = getColumnFromPODArray(block_end_dates); - columns.emplace_back(range_max_column, dictionary_structure.range_max->type, range_max_column_name); - } - - size_t attributes_size = dictionary_structure.attributes.size(); - for (size_t attribute_index = 0; attribute_index < attributes_size; ++attribute_index) - { - const auto & attribute = dictionary_structure.attributes[attribute_index]; - if (column_names.find(attribute.name) == column_names.end()) - continue; - - auto column = dictionary->getColumn( - attribute.name, - attribute.type, - keys_columns, - keys_types, - nullptr /* default_values_column*/); - - columns.emplace_back(std::move(column), attribute.type, attribute.name); - } - - return Block(columns); -} - -template -class RangeDictionarySource : public DictionarySourceBase -{ -public: - - RangeDictionarySource(RangeDictionarySourceData data_, size_t max_block_size); - - String getName() const override { return "RangeDictionarySource"; } - -protected: - Block getBlock(size_t start, size_t length) const override; - - RangeDictionarySourceData data; -}; - -template -RangeDictionarySource::RangeDictionarySource(RangeDictionarySourceData data_, size_t max_block_size) - : DictionarySourceBase(data_.getBlock(0, 0), data_.getNumRows(), max_block_size) - , data(std::move(data_)) -{ -} - -template -Block RangeDictionarySource::getBlock(size_t start, size_t length) const -{ - return data.getBlock(start, length); -} - -} diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index 2e677c163d6..67264155954 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -1,11 +1,11 @@ -#include "RangeHashedDictionary.h" +#include + #include #include -#include #include #include #include -#include +#include namespace @@ -219,6 +219,7 @@ ColumnUInt8::Ptr RangeHashedDictionary::hasKeys(const Colum key_types_copy.pop_back(); dict_struct.validateKeyTypes(key_types_copy); } + auto range_column_storage_type = std::make_shared(); auto range_storage_column = key_columns.back(); ColumnWithTypeAndName column_to_cast = {range_storage_column->convertToFullColumnIfConst(), key_types.back(), ""}; @@ -567,7 +568,7 @@ void RangeHashedDictionary::getKeysAndDates( { const auto & attribute = attributes.front(); - auto type_call = [&](const auto &dictionary_attribute_type) + auto type_call = [&](const auto & dictionary_attribute_type) { using Type = std::decay_t; using AttributeType = typename Type::AttributeType; @@ -610,28 +611,6 @@ void RangeHashedDictionary::getKeysAndDates( } } -template -template -Pipe RangeHashedDictionary::readImpl(const Names & column_names, size_t max_block_size) const -{ - PaddedPODArray keys; - PaddedPODArray start_dates; - PaddedPODArray end_dates; - getKeysAndDates(keys, start_dates, end_dates); - - using RangeDictionarySourceType = RangeDictionarySource; - - auto source_data = RangeDictionarySourceData( - shared_from_this(), - column_names, - std::move(keys), - std::move(start_dates), - std::move(end_dates)); - auto source = std::make_shared(std::move(source_data), max_block_size); - - return Pipe(source); -} - template StringRef RangeHashedDictionary::copyKeyInArena(StringRef key) { @@ -643,38 +622,83 @@ StringRef RangeHashedDictionary::copyKeyInArena(StringRef k } template -struct RangeHashedDictionaryCallGetSourceImpl +template +PaddedPODArray RangeHashedDictionary::makeDateKeys( + const PaddedPODArray & block_start_dates, + const PaddedPODArray & block_end_dates) const { - Pipe pipe; - const RangeHashedDictionary * dict; - const Names * column_names; - size_t max_block_size; + PaddedPODArray keys(block_start_dates.size()); - template - void operator()(Id) + for (size_t i = 0; i < keys.size(); ++i) { - const auto & type = dict->dict_struct.range_min->type; - if (pipe.empty() && dynamic_cast *>(type.get())) - pipe = dict->template readImpl(*column_names, max_block_size); + if (Range::isCorrectDate(block_start_dates[i])) + keys[i] = block_start_dates[i]; + else + keys[i] = block_end_dates[i]; } -}; + + return keys; +} template -Pipe RangeHashedDictionary::read(const Names & column_names, size_t max_block_size) const +Pipe RangeHashedDictionary::read(const Names & column_names, size_t max_block_size, size_t num_streams) const { - RangeHashedDictionaryCallGetSourceImpl callable; - callable.dict = this; - callable.column_names = &column_names; - callable.max_block_size = max_block_size; + auto type = dict_struct.range_min->type; - TLUtils::forEach(TLIntegral{}, callable); + ColumnsWithTypeAndName key_columns; + ColumnWithTypeAndName range_min_column; + ColumnWithTypeAndName range_max_column; - if (callable.pipe.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Unexpected range type for RangeHashed dictionary: {}", - dict_struct.range_min->type->getName()); + auto type_call = [&](const auto & types) mutable -> bool + { + using Types = std::decay_t; + using LeftDataType = typename Types::LeftType; - return std::move(callable.pipe); + if constexpr (IsDataTypeNumber) + { + using RangeType = typename LeftDataType::FieldType; + + PaddedPODArray keys; + PaddedPODArray start_dates; + PaddedPODArray end_dates; + getKeysAndDates(keys, start_dates, end_dates); + + range_min_column = ColumnWithTypeAndName{getColumnFromPODArray(start_dates), dict_struct.range_min->type, dict_struct.range_min->name}; + range_max_column = ColumnWithTypeAndName{getColumnFromPODArray(end_dates), dict_struct.range_max->type, dict_struct.range_max->name}; + + if constexpr (dictionary_key_type == DictionaryKeyType::Simple) + key_columns = {ColumnWithTypeAndName(getColumnFromPODArray(keys), std::make_shared(), dict_struct.id->name)}; + else + key_columns = deserializeColumnsWithTypeAndNameFromKeys(dict_struct, keys, 0, keys.size()); + + auto date_column = getColumnFromPODArray(makeDateKeys(start_dates, end_dates)); + key_columns.emplace_back(ColumnWithTypeAndName{std::move(date_column), std::make_shared(), ""}); + + return true; + } + else + { + return false; + } + }; + + if (!callOnIndexAndDataType(type->getTypeId(), type_call)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "RangeHashedDictionary min max range type should be numeric"); + + ColumnsWithTypeAndName data_columns = {std::move(range_min_column), std::move(range_max_column)}; + + std::shared_ptr dictionary = shared_from_this(); + auto coordinator = std::make_shared(dictionary, column_names, std::move(key_columns), std::move(data_columns), max_block_size); + + Pipes pipes; + + for (size_t i = 0; i < num_streams; ++i) + { + auto source = std::make_shared(coordinator); + pipes.emplace_back(Pipe(std::move(source))); + } + + return Pipe::unitePipes(std::move(pipes)); } diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index 4cdab66a0e4..1ccd9708d79 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -90,7 +90,7 @@ public: ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; - Pipe read(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size, size_t num_streams) const override; private: template @@ -175,13 +175,12 @@ private: PaddedPODArray & end_dates) const; template - Pipe readImpl(const Names & column_names, size_t max_block_size) const; + PaddedPODArray makeDateKeys( + const PaddedPODArray & block_start_dates, + const PaddedPODArray & block_end_dates) const; StringRef copyKeyInArena(StringRef key); - template - friend struct RangeHashedDictionaryCallGetSourceImpl; - const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 349d894b2f1..be0b2d63402 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -164,11 +164,11 @@ Pipe StorageDictionary::read( ContextPtr local_context, QueryProcessingStage::Enum /*processed_stage*/, const size_t max_block_size, - const unsigned /*threads*/) + const unsigned threads) { auto registered_dictionary_name = location == Location::SameDatabaseAndNameAsDictionary ? getStorageID().getInternalDictionaryName() : dictionary_name; auto dictionary = getContext()->getExternalDictionariesLoader().getDictionary(registered_dictionary_name, local_context); - return dictionary->read(column_names, max_block_size); + return dictionary->read(column_names, max_block_size, threads); } void StorageDictionary::shutdown() From 8885748a6c0c033bc946b4f1d0b9e677b0539d0c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 21 Oct 2021 17:25:53 +0300 Subject: [PATCH 303/919] StorageDictionary fix potential configuration race --- src/Storages/StorageDictionary.cpp | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 349d894b2f1..51a73eb511e 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -212,11 +212,20 @@ void StorageDictionary::renameInMemory(const StorageID & new_table_id) auto old_table_id = getStorageID(); IStorage::renameInMemory(new_table_id); - if (configuration) + bool has_configuration = false; { - configuration->setString("dictionary.database", new_table_id.database_name); - configuration->setString("dictionary.name", new_table_id.table_name); + std::lock_guard lock(dictionary_config_mutex); + if (configuration) + { + has_configuration = true; + configuration->setString("dictionary.database", new_table_id.database_name); + configuration->setString("dictionary.name", new_table_id.table_name); + } + } + + if (has_configuration) + { const auto & external_dictionaries_loader = getContext()->getExternalDictionariesLoader(); auto result = external_dictionaries_loader.getLoadResult(old_table_id.getInternalDictionaryName()); From dd50b86a94b810d7c6435565b625c0a269904b46 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 21 Oct 2021 17:41:07 +0300 Subject: [PATCH 304/919] Add report builde --- .github/workflows/main.yml | 18 +++- tests/ci/build_check.py | 12 ++- tests/ci/build_report_check.py | 153 +++++++++++++++++++++++++++++++++ tests/ci/style_check.py | 9 -- 4 files changed, 178 insertions(+), 14 deletions(-) create mode 100644 tests/ci/build_report_check.py diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 6ea62e0a8b9..2b573747bbe 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -64,7 +64,7 @@ jobs: env: TEMP_PATH: ${{runner.temp}}/build_check REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/ccaches/.. + CACHES_PATH: ${{runner.temp}}/../ccaches CHECK_NAME: 'ClickHouse build check (actions)' run: mkdir -p $TEMP_PATH && cp -r $GITHUB_WORKSPACE $TEMP_PATH && cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" ${{ matrix.build_number }} - name: Upload build URLs to artifacts @@ -72,6 +72,22 @@ jobs: with: name: ${{ env.BUILD_NAME }} path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + BuilderReport: + needs: Builder + runs-on: [self-hosted, style-checker] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ runner.temp }}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Report builder + env: + TEMP_PATH: ${{runner.temp}}/report_check + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'ClickHouse build check (actions)' + run: mkdir -p $TEMP_PATH && cd $GITHUB_WORKSPACE/tests/ci && python3 build_report_check.py "$CHECK_NAME" FinishCheck: needs: [StyleCheck, DockerHubPush, CheckLabels, Builder] runs-on: [self-hosted, style-checker] diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 4e35d11ddc3..775e02a001d 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -157,7 +157,7 @@ if __name__ == "__main__": build_name = build_config_to_string(build_config) logging.info(f"Build short name {build_name}") - subprocess.check_call(f"echo 'BUILD_NAME={build_name}' >> $GITHUB_ENV", shell=True) + subprocess.check_call(f"echo 'BUILD_NAME=build_urls_{build_name}' >> $GITHUB_ENV", shell=True) build_output_path = os.path.join(temp_path, build_name) if not os.path.exists(build_output_path): @@ -174,7 +174,9 @@ if __name__ == "__main__": if not os.path.exists(build_clickhouse_log): os.makedirs(build_clickhouse_log) + start = time.time() log_path, success = build_clickhouse(packager_cmd, build_clickhouse_log) + elapsed = int(time.time() - start) subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {build_output_path}", shell=True) subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {ccache_path}", shell=True) logging.info("Build finished with %s, log path %s", success, log_path) @@ -190,15 +192,17 @@ if __name__ == "__main__": build_urls = s3_helper.upload_build_folder_to_s3(build_output_path, s3_path_prefix, keep_dirs_in_s3_path=False, upload_symlinks=False) logging.info("Got build URLs %s", build_urls) - for url in build_urls: - print("::notice ::Build URL: {}".format(url)) + print("::notice ::Build URLs: {}".format('\n'.join(build_urls))) result = { "log_url": log_url, "build_urls": build_urls, + "build_config": build_config, + "elapsed_seconds": elapsed, + "status": success, } print("::notice ::Log URL: {}".format(log_url)) - with open(os.path.join(temp_path, build_name + '.json'), 'w') as build_links: + with open(os.path.join(temp_path, "build_urls_" + build_name + '.json'), 'w') as build_links: json.dump(result, build_links) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py new file mode 100644 index 00000000000..2f557314fc9 --- /dev/null +++ b/tests/ci/build_report_check.py @@ -0,0 +1,153 @@ +#!/usr/bin/env python3 + +import json +from github import Github +from report import create_build_html_report +from s3_helper import S3Helper +import logging +import os +from get_robot_token import get_best_robot_token +import sys +from pr_info import PRInfo + +class BuildResult(object): + def __init__(self, compiler, build_type, sanitizer, bundled, splitted, status, elapsed_seconds, with_coverage): + self.compiler = compiler + self.build_type = build_type + self.sanitizer = sanitizer + self.bundled = bundled + self.splitted = splitted + self.status = status + self.elapsed_seconds = elapsed_seconds + self.with_coverage = with_coverage + +def group_by_artifacts(build_urls): + groups = {'deb': [], 'binary': [], 'tgz': [], 'rpm': [], 'preformance': []} + for url in build_urls: + if url.endswith('performance.tgz'): + groups['performance'].append(url) + elif url.endswith('.deb') or url.endswith('.buildinfo') or url.endswith('.changes') or url.endswith('.tar.gz'): + groups['deb'].append(url) + elif url.endswith('.rpm'): + groups['rpm'].append(url) + elif url.endswith('.tgz'): + groups['tgz'].append(url) + else: + groups['binary'].append(url) + return groups + +def get_commit(gh, commit_sha): + repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) + commit = repo.get_commit(commit_sha) + return commit + +def process_report(build_report): + build_config = build_report['build_config'] + build_result = BuildResult( + compiler=build_config['compiler'], + build_type=build_config['build-type'], + sanitizer=build_config['sanitizer'], + bundled=build_config['bundled'], + splitted=build_config['splitted'], + status=build_report['status'], + elapsed_seconds=build_report['elapsed_seconds'], + with_coverage=False + ) + build_results = [] + build_urls = [] + build_logs_urls = [] + urls_groups = group_by_artifacts(build_report['build_urls']) + for _, group_urls in urls_groups.items(): + if group_urls: + build_results.append(build_result) + build_urls.append(group_urls) + build_logs_urls.append(build_report['log_url']) + + return build_results, build_urls, build_logs_urls + + +if __name__ == "__main__": + logging.basicConfig(level=logging.INFO) + reports_path = os.getenv("REPORTS_PATH") + temp_path = os.path.join(os.getenv("TEMP_PATH")) + logging.info("Reports path %s", reports_path) + + build_check_name = sys.argv[1] + + build_reports = [] + for root, dirs, files in os.walk(reports_path): + for f in files: + if f.starts_with("build_urls_") and f.ends_with('.json'): + logging.info("Found build report json %s", f) + with open(os.path.join(root, f), 'r') as file_handler: + build_report = json.load(file_handler) + build_reports.append(build_report) + + + build_results = [] + build_artifacts = [] + build_logs = [] + + for build_report in build_reports: + build_result, build_artifacts_url, build_logs_url = process_report(build_report) + logging.info("Got %s result for report", len(build_result)) + build_results += build_result + build_artifacts += build_artifacts_url + build_logs += build_logs_url + + logging.info("Totally got %s results", len(build_results)) + + gh = Github(get_best_robot_token()) + s3_helper = S3Helper('https://s3.amazonaws.com') + with open(os.getenv('GITHUB_EVENT_PATH'), 'r') as event_file: + event = json.load(event_file) + + pr_info = PRInfo(event) + + branch_url = "https://github.com/ClickHouse/ClickHouse/commits/master" + branch_name = "master" + if pr_info.number != 0: + branch_name = "PR #{}".format(pr_info.number) + branch_url = "https://github.com/ClickHouse/ClickHouse/pull/" + str(pr_info.number) + commit_url = f"https://github.com/ClickHouse/ClickHouse/commit/{pr_info.commit}" + task_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + report = create_build_html_report( + build_check_name, + build_results, + build_logs, + build_artifacts, + task_url, + branch_url, + branch_name, + commit_url + ) + + report_path = os.path.join(temp_path, 'report.html') + with open(report_path, 'w') as f: + f.write(report) + + logging.info("Going to upload prepared report") + context_name_for_path = build_check_name.lower().replace(' ', '_') + s3_path_prefix = str(pr_info.number) + "/" + pr_info.sha + "/" + context_name_for_path + + url = s3_helper.upload_build_file_to_s3(report_path, s3_path_prefix + "/report.html") + logging.info("Report url %s", url) + + total_builds = len(build_results) + ok_builds = 0 + summary_status = "success" + for build_result in build_results: + if build_result.status == "failure" and summary_status != "error": + summary_status = "failure" + if build_result.status == "error" or not build_result.status: + summary_status = "error" + + if build_result.status == "success": + ok_builds += 1 + + description = "{}/{} builds are OK".format(ok_builds, total_builds) + + print("::notice ::Report url: {}".format(url)) + + commit = get_commit(gh, pr_info.sha) + commit.create_status(context=build_check_name, description=description, state=summary_status, target_url=url) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 4a8cde70bc2..147909516fa 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -85,15 +85,6 @@ def get_commit(gh, commit_sha): commit = repo.get_commit(commit_sha) return commit -def update_check_with_curl(check_id): - cmd_template = ("curl -v --request PATCH --url https://api.github.com/repos/ClickHouse/ClickHouse/check-runs/{} " - "--header 'authorization: Bearer {}' " - "--header 'Accept: application/vnd.github.v3+json' " - "--header 'content-type: application/json' " - "-d '{{\"name\" : \"hello-world-name\"}}'") - cmd = cmd_template.format(check_id, os.getenv("GITHUB_TOKEN")) - subprocess.check_call(cmd, shell=True) - if __name__ == "__main__": logging.basicConfig(level=logging.INFO) repo_path = os.path.join(os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../"))) From 039af96b61b4293618242a42d473b725a41c0323 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 21 Oct 2021 18:03:16 +0300 Subject: [PATCH 305/919] Set execute_scalar_subqueries in MutationsInterpreter only for MergeTree and prepareInterpreterSelectQuery --- src/Interpreters/MutationsInterpreter.cpp | 9 ++++++--- src/Interpreters/TreeRewriter.cpp | 4 ++-- src/Interpreters/TreeRewriter.h | 2 +- ...100_alter_scalar_circular_deadlock.reference | 2 ++ .../02100_alter_scalar_circular_deadlock.sql | 17 +++++++++++++++++ 5 files changed, 28 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index e7b09b6917f..057f7d7caab 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -442,7 +443,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) if (column.default_desc.kind == ColumnDefaultKind::Materialized) { auto query = column.default_desc.expression->clone(); - auto syntax_result = TreeRewriter(context).analyze(query, all_columns, {}, {}, false, true, dry_run); + auto syntax_result = TreeRewriter(context).analyze(query, all_columns); for (const String & dependency : syntax_result->requiredSourceColumns()) { if (updated_columns.count(dependency)) @@ -583,7 +584,7 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) throw Exception("Unknown index: " + command.index_name, ErrorCodes::BAD_ARGUMENTS); auto query = (*it).expression_list_ast->clone(); - auto syntax_result = TreeRewriter(context).analyze(query, all_columns, {}, {}, false, true, dry_run); + auto syntax_result = TreeRewriter(context).analyze(query, all_columns); const auto required_columns = syntax_result->requiredSourceColumns(); for (const auto & column : required_columns) dependencies.emplace(column, ColumnDependency::SKIP_INDEX); @@ -784,7 +785,9 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & for (const String & column : stage.output_columns) all_asts->children.push_back(std::make_shared(column)); - auto syntax_result = TreeRewriter(context).analyze(all_asts, all_columns, storage, metadata_snapshot, false, true, dry_run); + bool execute_scalar_subqueries = !dynamic_cast(storage.get()) || !dry_run; + auto syntax_result = TreeRewriter(context).analyze( + all_asts, all_columns, storage, metadata_snapshot, false, true, execute_scalar_subqueries); if (context->hasQueryContext()) for (const auto & it : syntax_result->getScalars()) context->getQueryContext()->addScalar(it.first, it.second); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 8f7846678ad..e0968b7fce4 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1101,7 +1101,7 @@ TreeRewriterResultPtr TreeRewriter::analyze( const StorageMetadataPtr & metadata_snapshot, bool allow_aggregations, bool allow_self_aliases, - bool dry_run) const + bool execute_scalar_subqueries) const { if (query->as()) throw Exception("Not select analyze for select asts.", ErrorCodes::LOGICAL_ERROR); @@ -1113,7 +1113,7 @@ TreeRewriterResultPtr TreeRewriter::analyze( normalize(query, result.aliases, result.source_columns_set, false, settings, allow_self_aliases); /// Executing scalar subqueries. Column defaults could be a scalar subquery. - executeScalarSubqueries(query, getContext(), 0, result.scalars, dry_run); + executeScalarSubqueries(query, getContext(), 0, result.scalars, !execute_scalar_subqueries); if (settings.legacy_column_name_of_tuple_literal) markTupleLiteralsAsLegacy(query); diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index 2430976825f..52c62cc4cec 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -111,7 +111,7 @@ public: const StorageMetadataPtr & metadata_snapshot = {}, bool allow_aggregations = false, bool allow_self_aliases = true, - bool dry_run = false) const; + bool execute_scalar_subqueries = true) const; /// Analyze and rewrite select query TreeRewriterResultPtr analyzeSelect( diff --git a/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.reference b/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.reference index 6ed281c757a..98fb6a68656 100644 --- a/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.reference +++ b/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.reference @@ -1,2 +1,4 @@ 1 1 +1 +1 diff --git a/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.sql b/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.sql index 5fde10d5d09..32b757f54a3 100644 --- a/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.sql +++ b/tests/queries/0_stateless/02100_alter_scalar_circular_deadlock.sql @@ -15,3 +15,20 @@ SELECT sum(x) == 42 FROM foo; ALTER TABLE foo UPDATE x = 1 WHERE x IN (SELECT x FROM foo WHERE x != 0); SELECT sum(x) == 9 FROM foo; + +DROP TABLE IF EXISTS bar; + +CREATE TABLE bar (ts DateTime, x UInt64) +ENGINE = Memory; + +INSERT INTO bar (ts, x) SELECT toDateTime('2020-01-01 00:05:00'), number from system.numbers_mt LIMIT 10; + +SET mutations_sync = 1; + +ALTER TABLE bar UPDATE x = 1 WHERE x = (SELECT x from bar WHERE x = 4); + +SELECT sum(x) == 42 FROM bar; + +ALTER TABLE bar UPDATE x = 1 WHERE x IN (SELECT x FROM bar WHERE x != 0); + +SELECT sum(x) == 9 FROM bar; From 120b01e0896deb0660f81467888aef369c3ac941 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 21 Oct 2021 13:01:04 +0300 Subject: [PATCH 306/919] Better --- ...chronousReadIndirectBufferFromRemoteFS.cpp | 36 ++++++------------- ...ynchronousReadIndirectBufferFromRemoteFS.h | 4 +-- ...ynchronousReadBufferFromFileDescriptor.cpp | 8 ++--- ...AsynchronousReadBufferFromFileDescriptor.h | 2 +- 4 files changed, 18 insertions(+), 32 deletions(-) diff --git a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp index eec9c2ed157..64c9ce6c433 100644 --- a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.cpp @@ -56,31 +56,20 @@ String AsynchronousReadIndirectBufferFromRemoteFS::getFileName() const } -size_t AsynchronousReadIndirectBufferFromRemoteFS::getNumBytesToRead() +bool AsynchronousReadIndirectBufferFromRemoteFS::hasPendingDataToRead() { - size_t num_bytes_to_read; - /// Position is set only for MergeTree tables. if (read_until_position) { /// Everything is already read. - if (file_offset_of_buffer_end == *read_until_position) - return 0; + if (file_offset_of_buffer_end == read_until_position) + return false; - if (file_offset_of_buffer_end > *read_until_position) + if (file_offset_of_buffer_end > read_until_position) throw Exception(ErrorCodes::LOGICAL_ERROR, "Read beyond last offset ({} > {})", - file_offset_of_buffer_end, *read_until_position); - - /// Read range [file_offset_of_buffer_end, read_until_position). - num_bytes_to_read = *read_until_position - file_offset_of_buffer_end; - num_bytes_to_read = std::min(num_bytes_to_read, internal_buffer.size()); + file_offset_of_buffer_end, read_until_position); } - else - { - num_bytes_to_read = internal_buffer.size(); - } - - return num_bytes_to_read; + return true; } @@ -107,12 +96,10 @@ void AsynchronousReadIndirectBufferFromRemoteFS::prefetch() if (prefetch_future.valid()) return; - auto num_bytes_to_read = getNumBytesToRead(); - if (!num_bytes_to_read) + if (!hasPendingDataToRead()) return; /// Prefetch even in case hasPendingData() == true. - prefetch_buffer.resize(num_bytes_to_read); prefetch_future = readInto(prefetch_buffer.data(), prefetch_buffer.size()); ProfileEvents::increment(ProfileEvents::RemoteFSPrefetches); } @@ -130,6 +117,9 @@ void AsynchronousReadIndirectBufferFromRemoteFS::setReadUntilPosition(size_t pos bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() { + if (!hasPendingDataToRead()) + return false; + size_t size = 0; if (prefetch_future.valid()) @@ -154,12 +144,8 @@ bool AsynchronousReadIndirectBufferFromRemoteFS::nextImpl() } else { - auto num_bytes_to_read = getNumBytesToRead(); - if (!num_bytes_to_read) /// Nothing to read. - return false; - ProfileEvents::increment(ProfileEvents::RemoteFSUnprefetchedReads); - size = readInto(memory.data(), num_bytes_to_read).get(); + size = readInto(memory.data(), memory.size()).get(); if (size) { diff --git a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h index 8b25aa94a61..5bc528bb123 100644 --- a/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h +++ b/src/Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h @@ -54,7 +54,7 @@ private: void finalize(); - size_t getNumBytesToRead(); + bool hasPendingDataToRead(); std::future readInto(char * data, size_t size); @@ -74,7 +74,7 @@ private: size_t bytes_to_ignore = 0; - std::optional read_until_position = 0; + size_t read_until_position = 0; }; } diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp index 8dbdc85a330..f443f04a278 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp @@ -57,15 +57,15 @@ size_t AsynchronousReadBufferFromFileDescriptor::getNumBytesToRead() if (read_until_position) { /// Everything is already read. - if (file_offset_of_buffer_end == *read_until_position) + if (file_offset_of_buffer_end == read_until_position) return 0; - if (file_offset_of_buffer_end > *read_until_position) + if (file_offset_of_buffer_end > read_until_position) throw Exception(ErrorCodes::LOGICAL_ERROR, "Read beyond last offset ({} > {})", - file_offset_of_buffer_end, *read_until_position); + file_offset_of_buffer_end, read_until_position); /// Read range [file_offset_of_buffer_end, read_until_position). - num_bytes_to_read = *read_until_position - file_offset_of_buffer_end; + num_bytes_to_read = read_until_position - file_offset_of_buffer_end; num_bytes_to_read = std::min(num_bytes_to_read, internal_buffer.size()); } else diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.h b/src/IO/AsynchronousReadBufferFromFileDescriptor.h index 12e34cf784e..b526ab7113c 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.h +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.h @@ -24,7 +24,7 @@ protected: const size_t required_alignment = 0; /// For O_DIRECT both file offsets and memory addresses have to be aligned. size_t file_offset_of_buffer_end = 0; /// What offset in file corresponds to working_buffer.end(). - std::optional read_until_position; + size_t read_until_position = 0; int fd; bool nextImpl() override; From a874ca29edfdaf5e39efb19ec0d3743441525661 Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Thu, 21 Oct 2021 09:31:38 -0600 Subject: [PATCH 307/919] Change link in nav from community -> learn (#30508) --- website/templates/global/nav.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/templates/global/nav.html b/website/templates/global/nav.html index 7e120c99126..d775c88e4a5 100644 --- a/website/templates/global/nav.html +++ b/website/templates/global/nav.html @@ -42,7 +42,7 @@ Documentation From 00be07b9af976c44a97eb7c47fb84d555f410c27 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 30 Oct 2021 01:09:07 +0300 Subject: [PATCH 840/919] Fix pagination in lambdas --- tests/ci/metrics_lambda/app.py | 15 ++++++++++++--- tests/ci/termination_lambda/app.py | 14 +++++++++++--- 2 files changed, 23 insertions(+), 6 deletions(-) diff --git a/tests/ci/metrics_lambda/app.py b/tests/ci/metrics_lambda/app.py index 8c7807e99cc..af0e0fe07f1 100644 --- a/tests/ci/metrics_lambda/app.py +++ b/tests/ci/metrics_lambda/app.py @@ -53,12 +53,21 @@ def list_runners(access_token): "Authorization": f"token {access_token}", "Accept": "application/vnd.github.v3+json", } - - response = requests.get("https://api.github.com/orgs/ClickHouse/actions/runners", headers=headers) + response = requests.get("https://api.github.com/orgs/ClickHouse/actions/runners?per_page=100", headers=headers) response.raise_for_status() data = response.json() - print("Total runners", data['total_count']) + total_runners = data['total_count'] runners = data['runners'] + + total_pages = int(total_runners / 100 + 1) + print("Total pages", total_pages) + for i in range(2, total_pages + 1): + response = requests.get(f"https://api.github.com/orgs/ClickHouse/actions/runners?page={i}&per_page=100", headers=headers) + response.raise_for_status() + data = response.json() + runners += data['runners'] + + print("Total runners", len(runners)) result = [] for runner in runners: tags = [tag['name'] for tag in runner['labels']] diff --git a/tests/ci/termination_lambda/app.py b/tests/ci/termination_lambda/app.py index 0b39cf73f25..cd7d51ae8eb 100644 --- a/tests/ci/termination_lambda/app.py +++ b/tests/ci/termination_lambda/app.py @@ -49,12 +49,20 @@ def list_runners(access_token): "Authorization": f"token {access_token}", "Accept": "application/vnd.github.v3+json", } - - response = requests.get("https://api.github.com/orgs/ClickHouse/actions/runners", headers=headers) + response = requests.get("https://api.github.com/orgs/ClickHouse/actions/runners?per_page=100", headers=headers) response.raise_for_status() data = response.json() - print("Total runners", data['total_count']) + total_runners = data['total_count'] runners = data['runners'] + + total_pages = int(total_runners / 100 + 1) + for i in range(2, total_pages + 1): + response = requests.get(f"https://api.github.com/orgs/ClickHouse/actions/runners?page={i}&per_page=100", headers=headers) + response.raise_for_status() + data = response.json() + runners += data['runners'] + + print("Total runners", len(runners)) result = [] for runner in runners: tags = [tag['name'] for tag in runner['labels']] From 0d7dfdccf964aca458ff9c18bde79209e8bbc78a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 30 Oct 2021 01:29:36 +0300 Subject: [PATCH 841/919] Fixed tests --- .../test_allowed_url_from_config/test.py | 14 +++++++------- tests/integration/test_dictionaries_ddl/test.py | 2 +- tests/integration/test_storage_s3/test.py | 4 ++-- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_allowed_url_from_config/test.py b/tests/integration/test_allowed_url_from_config/test.py index 16be9e755bb..71bcea482f8 100644 --- a/tests/integration/test_allowed_url_from_config/test.py +++ b/tests/integration/test_allowed_url_from_config/test.py @@ -73,23 +73,23 @@ def test_config_without_allowed_hosts(start_cluster): def test_table_function_remote(start_cluster): - assert "not allowed in config.xml" not in node6.query_and_get_error( + assert "not allowed in configuration file" not in node6.query_and_get_error( "SELECT * FROM remoteSecure('example01-01-{1|2}', system, events)", settings={"connections_with_failover_max_tries": 1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout": 1}) - assert "not allowed in config.xml" not in node6.query_and_get_error( + assert "not allowed in configuration file" not in node6.query_and_get_error( "SELECT * FROM remoteSecure('example01-01-1,example01-02-1', system, events)", settings={"connections_with_failover_max_tries": 1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout": 1}) - assert "not allowed in config.xml" not in node6.query_and_get_error( + assert "not allowed in configuration file" not in node6.query_and_get_error( "SELECT * FROM remote('example01-0{1,2}-1', system, events", settings={"connections_with_failover_max_tries": 1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout": 1}) - assert "not allowed in config.xml" not in node6.query_and_get_error( + assert "not allowed in configuration file" not in node6.query_and_get_error( "SELECT * FROM remote('example01-0{1,2}-{1|2}', system, events)", settings={"connections_with_failover_max_tries": 1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout": 1}) - assert "not allowed in config.xml" not in node6.query_and_get_error( + assert "not allowed in configuration file" not in node6.query_and_get_error( "SELECT * FROM remoteSecure('example01-{01..02}-{1|2}', system, events)", settings={"connections_with_failover_max_tries": 1, "connect_timeout_with_failover_ms": 1000, "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout": 1}) @@ -108,9 +108,9 @@ def test_table_function_remote(start_cluster): "connect_timeout_with_failover_secure_ms": 1000, "connect_timeout": 1, "send_timeout": 1}) assert node6.query("SELECT * FROM remote('localhost', system, events)") != "" assert node6.query("SELECT * FROM remoteSecure('localhost', system, metrics)") != "" - assert "URL \"localhost:800\" is not allowed in config.xml" in node6.query_and_get_error( + assert "URL \"localhost:800\" is not allowed in configuration file" in node6.query_and_get_error( "SELECT * FROM remoteSecure('localhost:800', system, events)") - assert "URL \"localhost:800\" is not allowed in config.xml" in node6.query_and_get_error( + assert "URL \"localhost:800\" is not allowed in configuration file" in node6.query_and_get_error( "SELECT * FROM remote('localhost:800', system, metrics)") diff --git a/tests/integration/test_dictionaries_ddl/test.py b/tests/integration/test_dictionaries_ddl/test.py index 6cce303781b..72652880c58 100644 --- a/tests/integration/test_dictionaries_ddl/test.py +++ b/tests/integration/test_dictionaries_ddl/test.py @@ -228,7 +228,7 @@ def test_http_dictionary_restrictions(started_cluster): """) node3.query("SELECT dictGetString('test.restricted_http_dictionary', 'value', toUInt64(1))") except QueryRuntimeException as ex: - assert 'is not allowed in config.xml' in str(ex) + assert 'is not allowed in configuration file' in str(ex) node3.query("DROP DICTIONARY test.restricted_http_dictionary") diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 2f49b462d19..e25535b860f 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -436,12 +436,12 @@ def test_remote_host_filter(started_cluster): query = "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format( "invalid_host", MINIO_INTERNAL_PORT, started_cluster.minio_bucket, format) - assert "not allowed in config.xml" in instance.query_and_get_error(query) + assert "not allowed in configuration file" in instance.query_and_get_error(query) other_values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)" query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format( "invalid_host", MINIO_INTERNAL_PORT, started_cluster.minio_bucket, format, other_values) - assert "not allowed in config.xml" in instance.query_and_get_error(query) + assert "not allowed in configuration file" in instance.query_and_get_error(query) @pytest.mark.parametrize("s3_storage_args", [ From b1117458e942ab7f4625773baeaf9bf060cb5b56 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 30 Oct 2021 11:21:41 +0300 Subject: [PATCH 842/919] Currently disable release check --- .github/workflows/release.yml | 96 +++++++++++++++++------------------ 1 file changed, 48 insertions(+), 48 deletions(-) diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index f1b100b7795..b37a0eb69ad 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -1,48 +1,48 @@ -name: ReleaseChecks -concurrency: - group: docs-release - cancel-in-progress: true -on: # yamllint disable-line rule:truthy - push: - branches: - - master -jobs: - DockerHubPush: - needs: CheckLabels - runs-on: [self-hosted, style-checker] - steps: - - name: Check out repository code - uses: actions/checkout@v2 - - name: Images check - run: | - cd $GITHUB_WORKSPACE/tests/ci - python3 docker_images_check.py - - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 - with: - name: changed_images - path: ${{ runner.temp }}/docker_images_check/changed_images.json - DocsRelease: - runs: [self-hosted, func-tester] - steps: - - name: Check out repository code - uses: actions/checkout@v2 - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{runner.temp}}/docs_release - - name: Docs Release - env: - TEMP_PATH: ${{runner.temp}}/docs_release - REPO_COPY: ${{runner.temp}}/docs_release/ClickHouse - run: | - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 docs_release.py - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH +#name: ReleaseChecks +#concurrency: +# group: docs-release +# cancel-in-progress: true +#on: # yamllint disable-line rule:truthy +# push: +# branches: +# - master +#jobs: +# DockerHubPush: +# needs: CheckLabels +# runs-on: [self-hosted, style-checker] +# steps: +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Images check +# run: | +# cd $GITHUB_WORKSPACE/tests/ci +# python3 docker_images_check.py +# - name: Upload images files to artifacts +# uses: actions/upload-artifact@v2 +# with: +# name: changed_images +# path: ${{ runner.temp }}/docker_images_check/changed_images.json +# DocsRelease: +# runs: [self-hosted, func-tester] +# steps: +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Download changed images +# uses: actions/download-artifact@v2 +# with: +# name: changed_images +# path: ${{runner.temp}}/docs_release +# - name: Docs Release +# env: +# TEMP_PATH: ${{runner.temp}}/docs_release +# REPO_COPY: ${{runner.temp}}/docs_release/ClickHouse +# run: | +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 docs_release.py +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH From ab8fe00eb2bbc329d4dee4511e98b42b43013051 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 30 Oct 2021 11:29:38 +0300 Subject: [PATCH 843/919] Avoid warning from github --- .github/workflows/release.yml | 48 ------------------------- tests/ci/disabled_workflows/release.yml | 48 +++++++++++++++++++++++++ 2 files changed, 48 insertions(+), 48 deletions(-) delete mode 100644 .github/workflows/release.yml create mode 100644 tests/ci/disabled_workflows/release.yml diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml deleted file mode 100644 index b37a0eb69ad..00000000000 --- a/.github/workflows/release.yml +++ /dev/null @@ -1,48 +0,0 @@ -#name: ReleaseChecks -#concurrency: -# group: docs-release -# cancel-in-progress: true -#on: # yamllint disable-line rule:truthy -# push: -# branches: -# - master -#jobs: -# DockerHubPush: -# needs: CheckLabels -# runs-on: [self-hosted, style-checker] -# steps: -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Images check -# run: | -# cd $GITHUB_WORKSPACE/tests/ci -# python3 docker_images_check.py -# - name: Upload images files to artifacts -# uses: actions/upload-artifact@v2 -# with: -# name: changed_images -# path: ${{ runner.temp }}/docker_images_check/changed_images.json -# DocsRelease: -# runs: [self-hosted, func-tester] -# steps: -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Download changed images -# uses: actions/download-artifact@v2 -# with: -# name: changed_images -# path: ${{runner.temp}}/docs_release -# - name: Docs Release -# env: -# TEMP_PATH: ${{runner.temp}}/docs_release -# REPO_COPY: ${{runner.temp}}/docs_release/ClickHouse -# run: | -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 docs_release.py -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH diff --git a/tests/ci/disabled_workflows/release.yml b/tests/ci/disabled_workflows/release.yml new file mode 100644 index 00000000000..f1b100b7795 --- /dev/null +++ b/tests/ci/disabled_workflows/release.yml @@ -0,0 +1,48 @@ +name: ReleaseChecks +concurrency: + group: docs-release + cancel-in-progress: true +on: # yamllint disable-line rule:truthy + push: + branches: + - master +jobs: + DockerHubPush: + needs: CheckLabels + runs-on: [self-hosted, style-checker] + steps: + - name: Check out repository code + uses: actions/checkout@v2 + - name: Images check + run: | + cd $GITHUB_WORKSPACE/tests/ci + python3 docker_images_check.py + - name: Upload images files to artifacts + uses: actions/upload-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/docker_images_check/changed_images.json + DocsRelease: + runs: [self-hosted, func-tester] + steps: + - name: Check out repository code + uses: actions/checkout@v2 + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{runner.temp}}/docs_release + - name: Docs Release + env: + TEMP_PATH: ${{runner.temp}}/docs_release + REPO_COPY: ${{runner.temp}}/docs_release/ClickHouse + run: | + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 docs_release.py + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH From d647a2db0c0517661654b134c07318859417720f Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 30 Oct 2021 17:26:43 +0800 Subject: [PATCH 844/919] Fix invalid header in projection block calculation --- src/Storages/ProjectionsDescription.cpp | 2 +- .../01710_aggregate_projection_with_hashing.reference | 0 .../01710_aggregate_projection_with_hashing.sql | 11 +++++++++++ 3 files changed, 12 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01710_aggregate_projection_with_hashing.reference create mode 100644 tests/queries/0_stateless/01710_aggregate_projection_with_hashing.sql diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index e5117a306ee..dc0598de72f 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -237,7 +237,7 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) auto builder = InterpreterSelectQuery( query_ast, context, - Pipe(std::make_shared(block, Chunk(block.getColumns(), block.rows()))), + Pipe(std::make_shared(block)), SelectQueryOptions{ type == ProjectionDescription::Type::Normal ? QueryProcessingStage::FetchColumns : QueryProcessingStage::WithMergeableState}) diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_hashing.reference b/tests/queries/0_stateless/01710_aggregate_projection_with_hashing.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01710_aggregate_projection_with_hashing.sql b/tests/queries/0_stateless/01710_aggregate_projection_with_hashing.sql new file mode 100644 index 00000000000..d5eaa2617a6 --- /dev/null +++ b/tests/queries/0_stateless/01710_aggregate_projection_with_hashing.sql @@ -0,0 +1,11 @@ +set allow_experimental_projection_optimization = 1, force_optimize_projection = 1; + +drop table if exists tp; + +create table tp (type Int32, device UUID, cnt UInt64) engine = MergeTree order by (type, device); +insert into tp select number%3, generateUUIDv4(), 1 from numbers(300); + +alter table tp add projection uniq_city_proj ( select type, uniq(cityHash64(device)), sum(cnt) group by type ); +alter table tp materialize projection uniq_city_proj settings mutations_sync = 1; + +drop table tp; From e068855c3e79a740a330e6b49ee06a69ba4a3163 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 30 Oct 2021 09:54:17 +0000 Subject: [PATCH 845/919] fix fix --- .../02022_storage_filelog_one_file.sh | 11 +++--- .../0_stateless/02023_storage_filelog.sh | 39 +++++++++---------- .../0_stateless/02024_storage_filelog_mv.sh | 22 +++++------ .../02025_storage_filelog_virtual_col.sh | 27 +++++++------ .../02026_storage_filelog_largefile.sh | 18 ++++----- tests/queries/shell_config.sh | 1 + 6 files changed, 58 insertions(+), 60 deletions(-) diff --git a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh index bed97c7c0be..8ae0ce0ec1c 100755 --- a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel set -eu @@ -14,23 +13,23 @@ user_files_path=$(clickhouse-client --query "select _path,_file from file('nonex for i in {1..20} do - echo $i, $i >> ${user_files_path}/02022_storage_filelog_one_file.txt + echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}.txt done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" -${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/02022_storage_filelog_one_file.txt', 'CSV');" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}.txt', 'CSV');" ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" for i in {100..120} do - echo $i, $i >> ${user_files_path}/02022_storage_filelog_one_file.txt + echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}.txt done ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" # touch does not change file content, no event -touch ${user_files_path}/02022_storage_filelog_one_file.txt +touch ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}.txt ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" ${CLICKHOUSE_CLIENT} --query "detach table file_log;" @@ -39,4 +38,4 @@ ${CLICKHOUSE_CLIENT} --query "attach table file_log;" # should no records return ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" -rm -rf ${user_files_path}/02022_storage_filelog_one_file.txt +rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}.txt diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index fd242aad359..7480e378d8b 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel set -eu @@ -12,52 +11,52 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" 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 ${user_files_path}/02023_storage_filelog/ +mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ -rm -rf ${user_files_path}/02023_storage_filelog/* +rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* for i in {1..20} do - echo $i, $i >> ${user_files_path}/02023_storage_filelog/a.txt + echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" -${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/02023_storage_filelog/', 'CSV');" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/', 'CSV');" ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" -cp ${user_files_path}/02023_storage_filelog/a.txt ${user_files_path}/02023_storage_filelog/b.txt +cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/b.txt ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" for i in {100..120} do - echo $i, $i >> ${user_files_path}/02023_storage_filelog/a.txt + echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt done # touch does not change file content, no event -touch ${user_files_path}/02023_storage_filelog/a.txt +touch ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt -cp ${user_files_path}/02023_storage_filelog/a.txt ${user_files_path}/02023_storage_filelog/c.txt -cp ${user_files_path}/02023_storage_filelog/a.txt ${user_files_path}/02023_storage_filelog/d.txt -cp ${user_files_path}/02023_storage_filelog/a.txt ${user_files_path}/02023_storage_filelog/e.txt -mv ${user_files_path}/02023_storage_filelog/b.txt ${user_files_path}/02023_storage_filelog/j.txt +cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/c.txt +cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/d.txt +cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/e.txt +mv ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/b.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/j.txt -rm ${user_files_path}/02023_storage_filelog/d.txt +rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/d.txt ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" ${CLICKHOUSE_CLIENT} --query "detach table file_log;" -cp ${user_files_path}/02023_storage_filelog/e.txt ${user_files_path}/02023_storage_filelog/f.txt -mv ${user_files_path}/02023_storage_filelog/e.txt ${user_files_path}/02023_storage_filelog/g.txt -mv ${user_files_path}/02023_storage_filelog/c.txt ${user_files_path}/02023_storage_filelog/h.txt +cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/e.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/f.txt +mv ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/e.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/g.txt +mv ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/c.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/h.txt for i in {150..200} do - echo $i, $i >> ${user_files_path}/02023_storage_filelog/h.txt + echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/h.txt done for i in {200..250} do - echo $i, $i >> ${user_files_path}/02023_storage_filelog/i.txt + echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/i.txt done ${CLICKHOUSE_CLIENT} --query "attach table file_log;" @@ -69,11 +68,11 @@ ${CLICKHOUSE_CLIENT} --query "attach table file_log;" # should no records return ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" -truncate ${user_files_path}/02023_storage_filelog/a.txt --size 0 +truncate ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt --size 0 # exception happend ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" 2>&1 | grep -q "Code: 33" && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} --query "drop table file_log;" -rm -rf ${user_files_path}/02023_storage_filelog +rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} diff --git a/tests/queries/0_stateless/02024_storage_filelog_mv.sh b/tests/queries/0_stateless/02024_storage_filelog_mv.sh index 1e2f990c135..67aa825ac67 100755 --- a/tests/queries/0_stateless/02024_storage_filelog_mv.sh +++ b/tests/queries/0_stateless/02024_storage_filelog_mv.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: long set -eu @@ -12,16 +12,16 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" 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 ${user_files_path}/02024_storage_filelog_mv/ -rm -rf ${user_files_path}/02024_storage_filelog_mv/* +mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ +rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* for i in {1..20} do - echo $i, $i >> ${user_files_path}/02024_storage_filelog_mv/a.txt + echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" -${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/02024_storage_filelog_mv/', 'CSV');" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/', 'CSV');" ${CLICKHOUSE_CLIENT} --query "drop table if exists mv;" ${CLICKHOUSE_CLIENT} --query "create Materialized View mv engine=MergeTree order by k as select * from file_log;" @@ -39,17 +39,17 @@ done ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" -cp ${user_files_path}/02024_storage_filelog_mv/a.txt ${user_files_path}/02024_storage_filelog_mv/b.txt +cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/b.txt # touch does not change file content, no event -touch ${user_files_path}/02024_storage_filelog_mv/a.txt +touch ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt -cp ${user_files_path}/02024_storage_filelog_mv/a.txt ${user_files_path}/02024_storage_filelog_mv/c.txt -cp ${user_files_path}/02024_storage_filelog_mv/a.txt ${user_files_path}/02024_storage_filelog_mv/d.txt +cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/c.txt +cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/d.txt for i in {100..120} do - echo $i, $i >> ${user_files_path}/02024_storage_filelog_mv/d.txt + echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/d.txt done while true; do @@ -62,4 +62,4 @@ ${CLICKHOUSE_CLIENT} --query "select * from mv order by k;" ${CLICKHOUSE_CLIENT} --query "drop table mv;" ${CLICKHOUSE_CLIENT} --query "drop table file_log;" -rm -rf ${user_files_path}/02024_storage_filelog_mv +rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} diff --git a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh index 4545a18e650..a475913b7d2 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel set -eu @@ -12,37 +11,37 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" 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 ${user_files_path}/02025_storage_filelog_virtual_col/ +mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ -rm -rf ${user_files_path}/02025_storage_filelog_virtual_col/* +rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* for i in {1..20} do - echo $i, $i >> ${user_files_path}/02025_storage_filelog_virtual_col/a.txt + echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" -${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/02025_storage_filelog_virtual_col/', 'CSV');" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/', 'CSV');" ${CLICKHOUSE_CLIENT} --query "select *, _filename, _offset from file_log order by _filename, _offset;" -cp ${user_files_path}/02025_storage_filelog_virtual_col/a.txt ${user_files_path}/02025_storage_filelog_virtual_col/b.txt +cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/b.txt ${CLICKHOUSE_CLIENT} --query "select *, _filename, _offset from file_log order by _filename, _offset;" for i in {100..120} do - echo $i, $i >> ${user_files_path}/02025_storage_filelog_virtual_col/a.txt + echo $i, $i >> ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt done # touch does not change file content, no event -touch ${user_files_path}/02025_storage_filelog_virtual_col/a.txt +touch ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt -cp ${user_files_path}/02025_storage_filelog_virtual_col/a.txt ${user_files_path}/02025_storage_filelog_virtual_col/c.txt -cp ${user_files_path}/02025_storage_filelog_virtual_col/a.txt ${user_files_path}/02025_storage_filelog_virtual_col/d.txt -cp ${user_files_path}/02025_storage_filelog_virtual_col/a.txt ${user_files_path}/02025_storage_filelog_virtual_col/e.txt +cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/c.txt +cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/d.txt +cp ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/e.txt -rm ${user_files_path}/02025_storage_filelog_virtual_col/d.txt +rm ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/d.txt ${CLICKHOUSE_CLIENT} --query "select *, _filename, _offset from file_log order by _filename, _offset;" @@ -52,11 +51,11 @@ ${CLICKHOUSE_CLIENT} --query "attach table file_log;" # should no records return ${CLICKHOUSE_CLIENT} --query "select *, _filename, _offset from file_log order by _filename, _offset;" -truncate ${user_files_path}/02025_storage_filelog_virtual_col/a.txt --size 0 +truncate ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/a.txt --size 0 # exception happend ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" 2>&1 | grep -q "Code: 33" && echo 'OK' || echo 'FAIL' ${CLICKHOUSE_CLIENT} --query "drop table file_log;" -rm -rf ${user_files_path}/02025_storage_filelog_virtual_col +rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} diff --git a/tests/queries/0_stateless/02026_storage_filelog_largefile.sh b/tests/queries/0_stateless/02026_storage_filelog_largefile.sh index c480fcc05d3..6babcc1e4f1 100755 --- a/tests/queries/0_stateless/02026_storage_filelog_largefile.sh +++ b/tests/queries/0_stateless/02026_storage_filelog_largefile.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: long set -eu @@ -12,36 +12,36 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # "insert into function file('exist.txt', 'CSV', 'val1 char') values ('aaaa'); select _path from file('exist.txt', 'CSV', 'val1 char')" 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 ${user_files_path}/02026_storage_filelog_largefile/ +mkdir -p ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ -rm -rf ${user_files_path}/02026_storage_filelog_largefile/* +rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?}/* -chmod 777 ${user_files_path}/02026_storage_filelog_largefile/ +chmod 777 ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/ for i in {1..200} do - ${CLICKHOUSE_CLIENT} --query "insert into function file('${user_files_path}/02026_storage_filelog_largefile/test$i.csv', 'CSV', 'k UInt32, v UInt32') select number, number from numbers(10000);" + ${CLICKHOUSE_CLIENT} --query "insert into function file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/test$i.csv', 'CSV', 'k UInt32, v UInt32') select number, number from numbers(10000);" done ${CLICKHOUSE_CLIENT} --query "drop table if exists file_log;" -${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt32, v UInt32) engine=FileLog('${user_files_path}/02026_storage_filelog_largefile/', 'CSV');" +${CLICKHOUSE_CLIENT} --query "create table file_log(k UInt32, v UInt32) engine=FileLog('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/', 'CSV');" ${CLICKHOUSE_CLIENT} --query "select count() from file_log " for i in {201..400} do - ${CLICKHOUSE_CLIENT} --query "insert into function file('${user_files_path}/02026_storage_filelog_largefile/test$i.csv', 'CSV', 'k UInt32, v UInt32') select number, number from numbers(10000);" + ${CLICKHOUSE_CLIENT} --query "insert into function file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/test$i.csv', 'CSV', 'k UInt32, v UInt32') select number, number from numbers(10000);" done ${CLICKHOUSE_CLIENT} --query "select count() from file_log " for i in {401..600} do - ${CLICKHOUSE_CLIENT} --query "insert into function file('${user_files_path}/02026_storage_filelog_largefile/test$i.csv', 'CSV', 'k UInt32, v UInt32') select number, number from numbers(10000);" + ${CLICKHOUSE_CLIENT} --query "insert into function file('${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME}/test$i.csv', 'CSV', 'k UInt32, v UInt32') select number, number from numbers(10000);" done ${CLICKHOUSE_CLIENT} --query "select count() from file_log " ${CLICKHOUSE_CLIENT} --query "drop table file_log;" -rm -rf ${user_files_path}/02026_storage_filelog_largefile +rm -rf ${user_files_path}/${CLICKHOUSE_TEST_UNIQUE_NAME:?} diff --git a/tests/queries/shell_config.sh b/tests/queries/shell_config.sh index ae279c93527..1f802ddab7b 100644 --- a/tests/queries/shell_config.sh +++ b/tests/queries/shell_config.sh @@ -11,6 +11,7 @@ export CLICKHOUSE_TEST_PATH="${BASH_SOURCE[1]}" CLICKHOUSE_TEST_NAME="$(basename "$CLICKHOUSE_TEST_PATH" .sh)" export CLICKHOUSE_TEST_NAME export CLICKHOUSE_TEST_ZOOKEEPER_PREFIX="${CLICKHOUSE_TEST_NAME}_${CLICKHOUSE_DATABASE}" +export CLICKHOUSE_TEST_UNIQUE_NAME="${CLICKHOUSE_TEST_NAME}_${CLICKHOUSE_DATABASE}" [ -v CLICKHOUSE_CONFIG_CLIENT ] && CLICKHOUSE_CLIENT_OPT0+=" --config-file=${CLICKHOUSE_CONFIG_CLIENT} " [ -v CLICKHOUSE_HOST ] && CLICKHOUSE_CLIENT_OPT0+=" --host=${CLICKHOUSE_HOST} " From 89dc19bbb8cf1aba5c99b19544c713fccbdef5eb Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 30 Oct 2021 15:11:31 +0300 Subject: [PATCH 846/919] Better --- .../ci/disabled_workflows => .github/workflows}/release.yml | 2 ++ tests/ci/docs_release.py | 4 ++-- tests/ci/ssh.py | 3 +-- 3 files changed, 5 insertions(+), 4 deletions(-) rename {tests/ci/disabled_workflows => .github/workflows}/release.yml (91%) diff --git a/tests/ci/disabled_workflows/release.yml b/.github/workflows/release.yml similarity index 91% rename from tests/ci/disabled_workflows/release.yml rename to .github/workflows/release.yml index f1b100b7795..a7d3151d41c 100644 --- a/tests/ci/disabled_workflows/release.yml +++ b/.github/workflows/release.yml @@ -36,6 +36,8 @@ jobs: env: TEMP_PATH: ${{runner.temp}}/docs_release REPO_COPY: ${{runner.temp}}/docs_release/ClickHouse + CLOUDFLARE_TOKEN: ${{secrets.CLOUDFLARE}} + ROBOT_CLICKHOUSE_SSH_KEY: ${{secrets.ROBOT_CLICKHOUSE_SSH_KEY}} run: | cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci diff --git a/tests/ci/docs_release.py b/tests/ci/docs_release.py index dfb3d50f1d8..9d41b334836 100644 --- a/tests/ci/docs_release.py +++ b/tests/ci/docs_release.py @@ -105,12 +105,12 @@ if __name__ == "__main__": if not os.path.exists(test_output): os.makedirs(test_output) - token = get_parameter_from_ssm('cloudflare_token', decrypt=True) + token = os.getenv('CLOUDFLARE_TOKEN') cmd = f"docker run --cap-add=SYS_PTRACE -e CLOUDFLARE_TOKEN={token} --volume={repo_path}:/repo_path --volume={test_output}:/output_path {docker_image}" run_log_path = os.path.join(test_output, 'runlog.log') - with open(run_log_path, 'w', encoding='utf-8') as log, SSHKey("robot-clickhouse-ssh"): + with open(run_log_path, 'w', encoding='utf-8') as log, SSHKey("ROBOT_CLICKHOUSE_SSH_KEY"): with subprocess.Popen(cmd, shell=True, stderr=log, stdout=log) as process: retcode = process.wait() if retcode == 0: diff --git a/tests/ci/ssh.py b/tests/ci/ssh.py index 3e0cca31664..1c0515364a8 100644 --- a/tests/ci/ssh.py +++ b/tests/ci/ssh.py @@ -6,7 +6,6 @@ import subprocess import tempfile import logging import signal -from get_robot_token import get_parameter_from_ssm class SSHAgent: @@ -104,7 +103,7 @@ class SSHAgent: class SSHKey: def __init__(self, key_name): - self.key = get_parameter_from_ssm(key_name) + self.key = os.getenv(key_name) self._key_pub = None self._ssh_agent = SSHAgent() From 98953bc27463c72af116e948b60bc326f2fcdeda Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 Oct 2021 15:29:38 +0300 Subject: [PATCH 847/919] Fix test --- .../01073_bad_alter_partition.reference | 22 +++++++++++++++++++ .../0_stateless/01073_bad_alter_partition.sql | 19 +++++++++------- 2 files changed, 33 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01073_bad_alter_partition.reference b/tests/queries/0_stateless/01073_bad_alter_partition.reference index 9008cf74944..580df2ddf08 100644 --- a/tests/queries/0_stateless/01073_bad_alter_partition.reference +++ b/tests/queries/0_stateless/01073_bad_alter_partition.reference @@ -1,3 +1,25 @@ +1 2020-01-01 1 2020-01-02 +1 2020-01-03 +1 2020-01-04 +1 2020-01-05 +1 2020-01-06 +3 2020-01-01 3 2020-01-02 +3 2020-01-03 +3 2020-01-04 +3 2020-01-05 +3 2020-01-06 +4 2020-01-01 4 2020-01-02 +4 2020-01-03 +4 2020-01-05 +4 2020-01-06 +5 2020-01-01 +5 2020-01-02 +5 2020-01-03 +5 2020-01-06 +6 2020-01-01 +6 2020-01-02 +6 2020-01-03 +6 2020-01-06 diff --git a/tests/queries/0_stateless/01073_bad_alter_partition.sql b/tests/queries/0_stateless/01073_bad_alter_partition.sql index 58fc39fd68b..2e3cd47d6a0 100644 --- a/tests/queries/0_stateless/01073_bad_alter_partition.sql +++ b/tests/queries/0_stateless/01073_bad_alter_partition.sql @@ -1,19 +1,22 @@ DROP TABLE IF EXISTS merge_tree; CREATE TABLE merge_tree (d Date) ENGINE = MergeTree ORDER BY d PARTITION BY d; -INSERT INTO merge_tree VALUES ('2020-01-02'); -SELECT 1, * FROM merge_tree; +INSERT INTO merge_tree VALUES ('2020-01-01'), ('2020-01-02'), ('2020-01-03'), ('2020-01-04'), ('2020-01-05'), ('2020-01-06'); +SELECT 1, * FROM merge_tree ORDER BY d; -- ALTER TABLE merge_tree DROP PARTITION 2020-01-02; -- This does not even parse -- SELECT 2, * FROM merge_tree; -ALTER TABLE merge_tree DROP PARTITION 20200102; -SELECT 3, * FROM merge_tree; +ALTER TABLE merge_tree DROP PARTITION 20200103; -- unfortunately, this works, but not as user expected. +SELECT 3, * FROM merge_tree ORDER BY d; -ALTER TABLE merge_tree DROP PARTITION '20200102'; -- { serverError 38 } -SELECT 4, * FROM merge_tree; +ALTER TABLE merge_tree DROP PARTITION '20200104'; +SELECT 4, * FROM merge_tree ORDER BY d; -ALTER TABLE merge_tree DROP PARTITION '2020-01-02'; -SELECT 5, * FROM merge_tree; +ALTER TABLE merge_tree DROP PARTITION '2020-01-05'; +SELECT 5, * FROM merge_tree ORDER BY d; + +ALTER TABLE merge_tree DROP PARTITION '202001-06'; -- { serverError 38 } +SELECT 6, * FROM merge_tree ORDER BY d; DROP TABLE merge_tree; From 40bffcec3a91ce6a6634155c35fb869355a98478 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 30 Oct 2021 15:32:29 +0300 Subject: [PATCH 848/919] Fix error message in Keeper --- src/Server/KeeperTCPHandler.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 71099c23655..7ea33cb3558 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -306,7 +306,14 @@ void KeeperTCPHandler::runImpl() } else { - String reason = keeper_dispatcher->checkInit() ? "server is not initialized yet" : "no alive leader exists"; + String reason; + if (!keeper_dispatcher->checkInit() && !keeper_dispathcer->hasLeader()) + reason = "server is not initialized yet and no alive leader exists"; + else if (!keeper_dispatcher->checkInit()) + reason = "server is not initialized yet"; + else + reason = "no alive leader exists"; + LOG_WARNING(log, "Ignoring user request, because {}", reason); sendHandshake(false); return; From 16a6f8f0c5503afa49e1ca1e6759a7bf195fbf94 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 30 Oct 2021 15:34:30 +0300 Subject: [PATCH 849/919] Fix style --- tests/ci/docs_release.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/docs_release.py b/tests/ci/docs_release.py index 9d41b334836..7ce7028fbf5 100644 --- a/tests/ci/docs_release.py +++ b/tests/ci/docs_release.py @@ -11,7 +11,7 @@ from github import Github from report import create_test_html_report from s3_helper import S3Helper from pr_info import PRInfo -from get_robot_token import get_best_robot_token, get_parameter_from_ssm +from get_robot_token import get_best_robot_token from ssh import SSHKey NAME = "Docs Release (actions)" From 30ab4b3a7fd9593cc1ea6ccb0b99d05e2f24758a Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 30 Oct 2021 16:02:05 +0300 Subject: [PATCH 850/919] Update KeeperTCPHandler.cpp --- src/Server/KeeperTCPHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 7ea33cb3558..17a13955043 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -307,7 +307,7 @@ void KeeperTCPHandler::runImpl() else { String reason; - if (!keeper_dispatcher->checkInit() && !keeper_dispathcer->hasLeader()) + if (!keeper_dispatcher->checkInit() && !keeper_dispatcher->hasLeader()) reason = "server is not initialized yet and no alive leader exists"; else if (!keeper_dispatcher->checkInit()) reason = "server is not initialized yet"; From e97233f33d718a97d6452c48be8834654b49c228 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 30 Oct 2021 15:28:50 +0300 Subject: [PATCH 851/919] Fix --verbose in local and logging --- programs/client/Client.cpp | 5 ---- programs/local/LocalServer.cpp | 47 ++++++++++++++++++++++------------ src/Client/ClientBase.cpp | 6 +++++ 3 files changed, 36 insertions(+), 22 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 4951106f595..0c4141c74f2 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1007,9 +1007,6 @@ void Client::addOptions(OptionsDescription & options_description) ("max_client_network_bandwidth", po::value(), "the maximum speed of data exchange over the network for the client in bytes per second.") ("compression", po::value(), "enable or disable compression") - ("log-level", po::value(), "client log level") - ("server_logs_file", po::value(), "put server logs into specified file") - ("query-fuzzer-runs", po::value()->default_value(0), "After executing every SELECT query, do random mutations in it and run again specified number of times. This is used for testing to discover unexpected corner cases.") ("interleave-queries-file", po::value>()->multitoken(), "file path with queries to execute before every file from 'queries-file'; multiple files can be specified (--queries-file file1 file2...); this is needed to enable more aggressive fuzzing of newly added tests (see 'query-fuzzer-runs' option)") @@ -1125,8 +1122,6 @@ void Client::processOptions(const OptionsDescription & options_description, max_client_network_bandwidth = options["max_client_network_bandwidth"].as(); if (options.count("compression")) config().setBool("compression", options["compression"].as()); - if (options.count("server_logs_file")) - server_logs_file = options["server_logs_file"].as(); if (options.count("no-warnings")) config().setBool("no-warnings", true); diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 2a2fe4a78c8..8f27b1df43b 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -182,23 +183,6 @@ void LocalServer::initialize(Poco::Util::Application & self) auto loaded_config = config_processor.loadConfig(); config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false); } - - if (config().has("logger.console") || config().has("logger.level") || config().has("logger.log")) - { - // force enable logging - config().setString("logger", "logger"); - // sensitive data rules are not used here - buildLoggers(config(), logger(), "clickhouse-local"); - } - else - { - // Turn off server logging to stderr - if (!config().has("verbose")) - { - Poco::Logger::root().setLevel("none"); - Poco::Logger::root().setChannel(Poco::AutoPtr(new Poco::NullChannel())); - } - } } @@ -497,6 +481,35 @@ void LocalServer::processConfig() } print_stack_trace = config().getBool("stacktrace", false); + auto logging = (config().has("logger.console") + || config().has("logger.level") + || config().has("log-level") + || config().has("logger.log")); + + auto file_logging = config().has("server_logs_file"); + if (is_interactive && logging && !file_logging) + throw Exception("For interactive mode logging is allowed only with --server_logs_file option", + ErrorCodes::BAD_ARGUMENTS); + + if (file_logging) + { + auto level = Poco::Logger::parseLevel(config().getString("log-level", "trace")); + Poco::Logger::root().setLevel(level); + Poco::Logger::root().setChannel(Poco::AutoPtr(new Poco::SimpleFileChannel(server_logs_file))); + } + else if (logging) + { + // force enable logging + config().setString("logger", "logger"); + // sensitive data rules are not used here + buildLoggers(config(), logger(), "clickhouse-local"); + } + else + { + Poco::Logger::root().setLevel("none"); + Poco::Logger::root().setChannel(Poco::AutoPtr(new Poco::NullChannel())); + } + shared_context = Context::createShared(); global_context = Context::createGlobal(shared_context.get()); diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index ed90c777e20..ecc4e8802de 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1594,9 +1594,13 @@ void ClientBase::init(int argc, char ** argv) ("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.") ("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)") + ("echo", "in batch mode, print query before execution") ("verbose", "print query and other debugging info") + ("log-level", po::value(), "log level") + ("server_logs_file", po::value(), "put server logs into specified file") + ("multiline,m", "multiline") ("multiquery,n", "multiquery") @@ -1683,6 +1687,8 @@ void ClientBase::init(int argc, char ** argv) config().setBool("verbose", true); if (options.count("log-level")) Poco::Logger::root().setLevel(options["log-level"].as()); + if (options.count("server_logs_file")) + server_logs_file = options["server_logs_file"].as(); if (options.count("hardware-utilization")) progress_indication.print_hardware_utilization = true; From 973a7aea92fe2d02e2272a078a99004330b85ecc Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 30 Oct 2021 17:51:58 +0300 Subject: [PATCH 852/919] Rename columns "session_id"->"auth_id", "session_name"->"session_id" in SessionLog. --- src/Interpreters/Session.cpp | 24 ++++++++++++------------ src/Interpreters/Session.h | 2 +- src/Interpreters/SessionLog.cpp | 26 +++++++++++++------------- src/Interpreters/SessionLog.h | 12 ++++++------ 4 files changed, 32 insertions(+), 32 deletions(-) diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 865c9551219..f98f172eb0c 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -244,7 +244,7 @@ void Session::shutdownNamedSessions() } Session::Session(const ContextPtr & global_context_, ClientInfo::Interface interface_) - : session_id(UUIDHelpers::generateV4()), + : auth_id(UUIDHelpers::generateV4()), global_context(global_context_), log(&Poco::Logger::get(String{magic_enum::enum_name(interface_)} + "-Session")) { @@ -255,7 +255,7 @@ Session::Session(const ContextPtr & global_context_, ClientInfo::Interface inter Session::~Session() { LOG_DEBUG(log, "{} Destroying {} of user {}", - toString(session_id), + toString(auth_id), (named_session ? "named session '" + named_session->key.second + "'" : "unnamed session"), (user_id ? toString(*user_id) : "") ); @@ -267,7 +267,7 @@ Session::~Session() if (notified_session_log_about_login) { if (auto session_log = getSessionLog(); session_log && user) - session_log->addLogOut(session_id, user->getName(), getClientInfo()); + session_log->addLogOut(auth_id, user->getName(), getClientInfo()); } } @@ -285,7 +285,7 @@ Authentication::Type Session::getAuthenticationTypeOrLogInFailure(const String & catch (const Exception & e) { if (auto session_log = getSessionLog()) - session_log->addLoginFailure(session_id, getClientInfo(), user_name, e); + session_log->addLoginFailure(auth_id, getClientInfo(), user_name, e); throw; } @@ -306,19 +306,19 @@ void Session::authenticate(const Credentials & credentials_, const Poco::Net::So address = Poco::Net::SocketAddress{"127.0.0.1", 0}; LOG_DEBUG(log, "{} Authenticating user '{}' from {}", - toString(session_id), credentials_.getUserName(), address.toString()); + toString(auth_id), credentials_.getUserName(), address.toString()); try { user_id = global_context->getAccessControlManager().login(credentials_, address.host()); LOG_DEBUG(log, "{} Authenticated with global context as user {}", - toString(session_id), user_id ? toString(*user_id) : ""); + toString(auth_id), user_id ? toString(*user_id) : ""); } catch (const Exception & e) { - LOG_DEBUG(log, "{} Authentication failed with error: {}", toString(session_id), e.what()); + LOG_DEBUG(log, "{} Authentication failed with error: {}", toString(auth_id), e.what()); if (auto session_log = getSessionLog()) - session_log->addLoginFailure(session_id, *prepared_client_info, credentials_.getUserName(), e); + session_log->addLoginFailure(auth_id, *prepared_client_info, credentials_.getUserName(), e); throw; } @@ -350,7 +350,7 @@ ContextMutablePtr Session::makeSessionContext() throw Exception("Session context must be created before any query context", ErrorCodes::LOGICAL_ERROR); LOG_DEBUG(log, "{} Creating session context with user_id: {}", - toString(session_id), user_id ? toString(*user_id) : ""); + toString(auth_id), user_id ? toString(*user_id) : ""); /// Make a new session context. ContextMutablePtr new_session_context; new_session_context = Context::createCopy(global_context); @@ -380,7 +380,7 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: throw Exception("Session context must be created before any query context", ErrorCodes::LOGICAL_ERROR); LOG_DEBUG(log, "{} Creating named session context with name: {}, user_id: {}", - toString(session_id), session_name_, user_id ? toString(*user_id) : ""); + toString(auth_id), session_name_, user_id ? toString(*user_id) : ""); /// Make a new session context OR /// if the `session_id` and `user_id` were used before then just get a previously created session context. @@ -439,7 +439,7 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t query_context->makeQueryContext(); LOG_DEBUG(log, "{} Creating query context from {} context, user_id: {}, parent context user: {}", - toString(session_id), + toString(auth_id), from_session_context ? "session" : "global", user_id ? toString(*user_id) : "", query_context->getUser() ? query_context->getUser()->getName() : ""); @@ -487,7 +487,7 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t if (auto session_log = getSessionLog(); user && user_id && session_log) { session_log->addLoginSuccess( - session_id, + auth_id, named_session ? std::optional(named_session->key.second) : std::nullopt, *query_context); diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index b62327103e9..f3cae33d752 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -77,7 +77,7 @@ private: ContextMutablePtr makeQueryContextImpl(const ClientInfo * client_info_to_copy, ClientInfo * client_info_to_move) const; mutable bool notified_session_log_about_login = false; - const UUID session_id; + const UUID auth_id; const ContextPtr global_context; /// ClientInfo that will be copied to a session context when it's created. diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index 4967cb867c8..2a6f0f0316b 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -67,8 +67,8 @@ void fillColumnArray(const Strings & data, IColumn & column) namespace DB { -SessionLogElement::SessionLogElement(const UUID & session_id_, Type type_) - : session_id(session_id_), +SessionLogElement::SessionLogElement(const UUID & auth_id_, Type type_) + : auth_id(auth_id_), type(type_) { std::tie(event_time, event_time_microseconds) = eventTime(); @@ -121,8 +121,8 @@ NamesAndTypesList SessionLogElement::getNamesAndTypes() return { {"type", std::move(event_type)}, - {"session_id", std::make_shared()}, - {"session_name", std::make_shared()}, + {"auth_id", std::make_shared()}, + {"session_id", std::make_shared()}, {"event_date", std::make_shared()}, {"event_time", std::make_shared()}, {"event_time_microseconds", std::make_shared(6)}, @@ -157,8 +157,8 @@ void SessionLogElement::appendToBlock(MutableColumns & columns) const size_t i = 0; columns[i++]->insert(type); + columns[i++]->insert(auth_id); columns[i++]->insert(session_id); - columns[i++]->insert(session_name); columns[i++]->insert(static_cast(DateLUT::instance().toDayNum(event_time).toUnderType())); columns[i++]->insert(event_time); columns[i++]->insert(event_time_microseconds); @@ -202,13 +202,13 @@ void SessionLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insertData(auth_failure_reason.data(), auth_failure_reason.length()); } -void SessionLog::addLoginSuccess(const UUID & session_id, std::optional session_name, const Context & login_context) +void SessionLog::addLoginSuccess(const UUID & auth_id, std::optional session_id, const Context & login_context) { const auto access = login_context.getAccess(); const auto & settings = login_context.getSettingsRef(); const auto & client_info = login_context.getClientInfo(); - DB::SessionLogElement log_entry(session_id, SESSION_LOGIN_SUCCESS); + DB::SessionLogElement log_entry(auth_id, SESSION_LOGIN_SUCCESS); log_entry.client_info = client_info; { @@ -218,8 +218,8 @@ void SessionLog::addLoginSuccess(const UUID & session_id, std::optional log_entry.external_auth_server = user->authentication.getLDAPServerName(); } - if (session_name) - log_entry.session_name = *session_name; + if (session_id) + log_entry.session_id = *session_id; if (const auto roles_info = access->getRolesInfo()) log_entry.roles = roles_info->getCurrentRolesNames(); @@ -234,12 +234,12 @@ void SessionLog::addLoginSuccess(const UUID & session_id, std::optional } void SessionLog::addLoginFailure( - const UUID & session_id, + const UUID & auth_id, const ClientInfo & info, const String & user, const Exception & reason) { - SessionLogElement log_entry(session_id, SESSION_LOGIN_FAILURE); + SessionLogElement log_entry(auth_id, SESSION_LOGIN_FAILURE); log_entry.user = user; log_entry.auth_failure_reason = reason.message(); @@ -249,9 +249,9 @@ void SessionLog::addLoginFailure( add(log_entry); } -void SessionLog::addLogOut(const UUID & session_id, const String & user, const ClientInfo & client_info) +void SessionLog::addLogOut(const UUID & auth_id, const String & user, const ClientInfo & client_info) { - auto log_entry = SessionLogElement(session_id, SESSION_LOGOUT); + auto log_entry = SessionLogElement(auth_id, SESSION_LOGOUT); log_entry.user = user; log_entry.client_info = client_info; diff --git a/src/Interpreters/SessionLog.h b/src/Interpreters/SessionLog.h index fddabf45e4e..d2d1ac58d1e 100644 --- a/src/Interpreters/SessionLog.h +++ b/src/Interpreters/SessionLog.h @@ -27,17 +27,17 @@ struct SessionLogElement using Type = SessionLogElementType; SessionLogElement() = default; - SessionLogElement(const UUID & session_id_, Type type_); + SessionLogElement(const UUID & auth_id_, Type type_); SessionLogElement(const SessionLogElement &) = default; SessionLogElement & operator=(const SessionLogElement &) = default; SessionLogElement(SessionLogElement &&) = default; SessionLogElement & operator=(SessionLogElement &&) = default; - UUID session_id; + UUID auth_id; Type type = SESSION_LOGIN_FAILURE; - String session_name; + String session_id; time_t event_time{}; Decimal64 event_time_microseconds{}; @@ -66,9 +66,9 @@ class SessionLog : public SystemLog using SystemLog::SystemLog; public: - void addLoginSuccess(const UUID & session_id, std::optional session_name, const Context & login_context); - void addLoginFailure(const UUID & session_id, const ClientInfo & info, const String & user, const Exception & reason); - void addLogOut(const UUID & session_id, const String & user, const ClientInfo & client_info); + void addLoginSuccess(const UUID & auth_id, std::optional session_id, const Context & login_context); + void addLoginFailure(const UUID & auth_id, const ClientInfo & info, const String & user, const Exception & reason); + void addLogOut(const UUID & auth_id, const String & user, const ClientInfo & client_info); }; } From 54c89e0f0e9b7b18ab40e755805c115a462a6669 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 30 Oct 2021 17:59:22 +0300 Subject: [PATCH 853/919] Rename column "changed_settings"->"settings" in SessionLog. --- src/Interpreters/SessionLog.cpp | 20 ++++++++++---------- src/Interpreters/SessionLog.h | 2 +- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index 2a6f0f0316b..a4847d4c492 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -109,7 +109,7 @@ NamesAndTypesList SessionLogElement::getNamesAndTypes() const auto lc_string_datatype = std::make_shared(std::make_shared()); - const auto changed_settings_type_column = std::make_shared( + const auto settings_type_column = std::make_shared( std::make_shared( DataTypes({ // setting name @@ -132,7 +132,7 @@ NamesAndTypesList SessionLogElement::getNamesAndTypes() {"profiles", std::make_shared(lc_string_datatype)}, {"roles", std::make_shared(lc_string_datatype)}, - {"changed_settings", std::move(changed_settings_type_column)}, + {"settings", std::move(settings_type_column)}, {"client_address", DataTypeFactory::instance().get("IPv6")}, {"client_port", std::make_shared()}, @@ -170,21 +170,21 @@ void SessionLogElement::appendToBlock(MutableColumns & columns) const fillColumnArray(roles, *columns[i++]); { - auto & changed_settings_array_col = assert_cast(*columns[i++]); - auto & changed_settings_tuple_col = assert_cast(changed_settings_array_col.getData()); - auto & names_col = *changed_settings_tuple_col.getColumnPtr(0)->assumeMutable(); - auto & values_col = assert_cast(*changed_settings_tuple_col.getColumnPtr(1)->assumeMutable()); + auto & settings_array_col = assert_cast(*columns[i++]); + auto & settings_tuple_col = assert_cast(settings_array_col.getData()); + auto & names_col = *settings_tuple_col.getColumnPtr(0)->assumeMutable(); + auto & values_col = assert_cast(*settings_tuple_col.getColumnPtr(1)->assumeMutable()); size_t items_added = 0; - for (const auto & kv : changed_settings) + for (const auto & kv : settings) { names_col.insert(kv.first); values_col.insert(kv.second); ++items_added; } - auto & offsets = changed_settings_array_col.getOffsets(); - offsets.push_back(changed_settings_tuple_col.size()); + auto & offsets = settings_array_col.getOffsets(); + offsets.push_back(settings_tuple_col.size()); } columns[i++]->insertData(IPv6ToBinary(client_info.current_address.host()).data(), 16); @@ -228,7 +228,7 @@ void SessionLog::addLoginSuccess(const UUID & auth_id, std::optional ses log_entry.profiles = profile_info->getProfileNames(); for (const auto & s : settings.allChanged()) - log_entry.changed_settings.emplace_back(s.getName(), s.getValueString()); + log_entry.settings.emplace_back(s.getName(), s.getValueString()); add(log_entry); } diff --git a/src/Interpreters/SessionLog.h b/src/Interpreters/SessionLog.h index d2d1ac58d1e..6d302c74d5f 100644 --- a/src/Interpreters/SessionLog.h +++ b/src/Interpreters/SessionLog.h @@ -46,7 +46,7 @@ struct SessionLogElement String external_auth_server; Strings roles; Strings profiles; - std::vector> changed_settings; + std::vector> settings; ClientInfo client_info; String auth_failure_reason; From 05c2cd098c2078689774b72d86ccafa809ae4a9e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 Oct 2021 21:02:33 +0300 Subject: [PATCH 854/919] Fix parallel formatting and progress flicker in clickhouse-client --- src/Client/ClientBase.cpp | 37 +++++++++++++++++++++++++++---------- src/Client/ClientBase.h | 1 + 2 files changed, 28 insertions(+), 10 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index ed90c777e20..f101ffaf73d 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -57,7 +57,12 @@ namespace fs = std::filesystem; namespace DB { -static const NameSet exit_strings{"exit", "quit", "logout", "учше", "йгше", "дщпщге", "exit;", "quit;", "logout;", "учшеж", "йгшеж", "дщпщгеж", "q", "й", "\\q", "\\Q", "\\й", "\\Й", ":q", "Жй"}; +static const NameSet exit_strings +{ + "exit", "quit", "logout", "учше", "йгше", "дщпщге", + "exit;", "quit;", "logout;", "учшеж", "йгшеж", "дщпщгеж", + "q", "й", "\\q", "\\Q", "\\й", "\\Й", ":q", "Жй" +}; namespace ErrorCodes { @@ -103,9 +108,11 @@ void interruptSignalHandler(int signum) _exit(signum); } + ClientBase::~ClientBase() = default; ClientBase::ClientBase() = default; + void ClientBase::setupSignalHandler() { exit_on_signal.test_and_set(); @@ -168,8 +175,7 @@ ASTPtr ClientBase::parseQuery(const char *& pos, const char * end, bool allow_mu } -// Consumes trailing semicolons and tries to consume the same-line trailing -// comment. +/// Consumes trailing semicolons and tries to consume the same-line trailing comment. void ClientBase::adjustQueryEnd(const char *& this_query_end, const char * all_queries_end, int max_parser_depth) { // We have to skip the trailing semicolon that might be left @@ -246,7 +252,8 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) if (block.rows() == 0 || (query_fuzzer_runs != 0 && processed_rows >= 100)) return; - if (need_render_progress && (stdout_is_a_tty || is_interactive)) + /// If results are written INTO OUTFILE, we can avoid clearing progress to avoid flicker. + if (need_render_progress && (stdout_is_a_tty || is_interactive) && !select_into_file) progress_indication.clearProgressOutput(); output_format->write(materializeBlock(block)); @@ -257,7 +264,11 @@ void ClientBase::onData(Block & block, ASTPtr parsed_query) /// Restore progress bar after data block. if (need_render_progress && (stdout_is_a_tty || is_interactive)) + { + if (select_into_file) + std::cerr << "\r"; progress_indication.writeProgress(); + } } @@ -328,12 +339,16 @@ void ClientBase::initBlockOutputStream(const Block & block, ASTPtr parsed_query) String current_format = format; + select_into_file = false; + /// The query can specify output format or output file. /// FIXME: try to prettify this cast using `as<>()` if (const auto * query_with_output = dynamic_cast(parsed_query.get())) { if (query_with_output->out_file) { + select_into_file = true; + const auto & out_file_node = query_with_output->out_file->as(); const auto & out_file = out_file_node.value.safeGet(); @@ -366,11 +381,14 @@ void ClientBase::initBlockOutputStream(const Block & block, ASTPtr parsed_query) if (has_vertical_output_suffix) current_format = "Vertical"; - /// It is not clear how to write progress with parallel formatting. It may increase code complexity significantly. - if (!need_render_progress) - output_format = global_context->getOutputFormatParallelIfPossible(current_format, out_file_buf ? *out_file_buf : *out_buf, block); + /// It is not clear how to write progress intermixed with data with parallel formatting. + /// It may increase code complexity significantly. + if (!need_render_progress || select_into_file) + output_format = global_context->getOutputFormatParallelIfPossible( + current_format, out_file_buf ? *out_file_buf : *out_buf, block); else - output_format = global_context->getOutputFormat(current_format, out_file_buf ? *out_file_buf : *out_buf, block); + output_format = global_context->getOutputFormat( + current_format, out_file_buf ? *out_file_buf : *out_buf, block); output_format->doWritePrefix(); } @@ -1446,8 +1464,7 @@ void ClientBase::clearTerminal() /// It is needed if garbage is left in terminal. /// Show cursor. It can be left hidden by invocation of previous programs. /// A test for this feature: perl -e 'print "x"x100000'; echo -ne '\033[0;0H\033[?25l'; clickhouse-client - std::cout << "\033[0J" - "\033[?25h"; + std::cout << "\033[0J" "\033[?25h"; } diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index f7b418ad13a..93906946616 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -155,6 +155,7 @@ protected: ConnectionParameters connection_parameters; String format; /// Query results output format. + bool select_into_file = false; /// If writing result INTO OUTFILE. It affects progress rendering. bool is_default_format = true; /// false, if format is set in the config or command line. size_t format_max_block_size = 0; /// Max block size for console output. String insert_format; /// Format of INSERT data that is read from stdin in batch mode. From c504e0c08df7a926bb479a1d297f326f5c48a32f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 30 Oct 2021 21:17:38 +0300 Subject: [PATCH 855/919] Update ClientBase.cpp --- src/Client/ClientBase.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index f101ffaf73d..dba8fc53045 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -342,7 +342,6 @@ void ClientBase::initBlockOutputStream(const Block & block, ASTPtr parsed_query) select_into_file = false; /// The query can specify output format or output file. - /// FIXME: try to prettify this cast using `as<>()` if (const auto * query_with_output = dynamic_cast(parsed_query.get())) { if (query_with_output->out_file) From 5fe4bcd22ef259314527df87f27047935b4a6835 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 30 Oct 2021 21:33:31 +0300 Subject: [PATCH 856/919] fix WITH FILL with set TO and FROM and no rows in result set --- .../Transforms/FillingTransform.cpp | 7 +++++-- .../02111_with_fill_no_rows.reference | 4 ++++ .../0_stateless/02111_with_fill_no_rows.sql | 19 +++++++++++++++++++ 3 files changed, 28 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02111_with_fill_no_rows.reference create mode 100644 tests/queries/0_stateless/02111_with_fill_no_rows.sql diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 3ff89c302ff..5fe051e9498 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -117,12 +117,12 @@ IProcessor::Status FillingTransform::prepare() { if (!on_totals && input.isFinished() && !output.isFinished() && !has_input && !generate_suffix) { - should_insert_first = next_row < filling_row; + should_insert_first = next_row < filling_row || first; for (size_t i = 0, size = filling_row.size(); i < size; ++i) next_row[i] = filling_row.getFillDescription(i).fill_to; - if (filling_row < next_row) + if (first || filling_row < next_row) { generate_suffix = true; return Status::Ready; @@ -160,6 +160,9 @@ void FillingTransform::transform(Chunk & chunk) init_columns_by_positions(empty_columns, old_fill_columns, res_fill_columns, fill_column_positions); init_columns_by_positions(empty_columns, old_other_columns, res_other_columns, other_column_positions); + if (first) + filling_row.initFromDefaults(); + if (should_insert_first && filling_row < next_row) insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); diff --git a/tests/queries/0_stateless/02111_with_fill_no_rows.reference b/tests/queries/0_stateless/02111_with_fill_no_rows.reference new file mode 100644 index 00000000000..c0cc69a2d63 --- /dev/null +++ b/tests/queries/0_stateless/02111_with_fill_no_rows.reference @@ -0,0 +1,4 @@ +2019 0 +2020 0 +2021 0 +2022 0 diff --git a/tests/queries/0_stateless/02111_with_fill_no_rows.sql b/tests/queries/0_stateless/02111_with_fill_no_rows.sql new file mode 100644 index 00000000000..e671dd5f0f2 --- /dev/null +++ b/tests/queries/0_stateless/02111_with_fill_no_rows.sql @@ -0,0 +1,19 @@ +SELECT toYear(d) AS y, count() +FROM ( SELECT today() AS d WHERE 0) +GROUP BY y +ORDER BY y ASC WITH FILL FROM 2019 TO 2023; + +SELECT toYear(d) AS y, count() +FROM ( SELECT today() AS d WHERE 0) +GROUP BY y +ORDER BY y ASC WITH FILL FROM 2019; + +SELECT toYear(d) AS y, count() +FROM ( SELECT today() AS d WHERE 0) +GROUP BY y +ORDER BY y ASC WITH FILL TO 2023; + +SELECT toYear(d) AS y, count() +FROM ( SELECT today() AS d WHERE 0) +GROUP BY y +ORDER BY y ASC WITH FILL; From cd14be90aa52544edd4d33e8f670a517cdb4afb8 Mon Sep 17 00:00:00 2001 From: Teja Srivastasa Date: Sat, 30 Oct 2021 22:18:33 +0200 Subject: [PATCH 857/919] fix unresolved pid --- programs/install/Install.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 57e56c8b9ea..3cfcb849b61 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -809,6 +809,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) if (has_password_for_default_user) maybe_password = " --password"; + fs::path pid_file = prefix / options["pid-path"].as() / "clickhouse-server.pid"; if (fs::exists(pid_file)) { fmt::print( "\nClickHouse has been successfully installed.\n" From f040912ba659e92818a590defd2d1e60d1a97dd7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 Oct 2021 23:20:34 +0300 Subject: [PATCH 858/919] Add benchmark results for TimescaleDB --- benchmark/timescaledb/benchmark.sh | 11 + benchmark/timescaledb/log | 215 ++++++++++++++++++ benchmark/timescaledb/queries.sql | 43 ++++ .../dbms/results/013_timescaledb.json | 56 +++++ 4 files changed, 325 insertions(+) create mode 100755 benchmark/timescaledb/benchmark.sh create mode 100644 benchmark/timescaledb/log create mode 100644 benchmark/timescaledb/queries.sql create mode 100644 website/benchmark/dbms/results/013_timescaledb.json diff --git a/benchmark/timescaledb/benchmark.sh b/benchmark/timescaledb/benchmark.sh new file mode 100755 index 00000000000..691f4c42bfa --- /dev/null +++ b/benchmark/timescaledb/benchmark.sh @@ -0,0 +1,11 @@ +#!/bin/bash + +grep -v -P '^#' queries.sql | sed -e 's/{table}/hits_100m_obfuscated/' | while read query; do + + echo 3 | sudo tee /proc/sys/vm/drop_caches + + 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; diff --git a/benchmark/timescaledb/log b/benchmark/timescaledb/log new file mode 100644 index 00000000000..9c98da60612 --- /dev/null +++ b/benchmark/timescaledb/log @@ -0,0 +1,215 @@ +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/queries.sql b/benchmark/timescaledb/queries.sql new file mode 100644 index 00000000000..d7a2fe2d8bf --- /dev/null +++ b/benchmark/timescaledb/queries.sql @@ -0,0 +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); diff --git a/website/benchmark/dbms/results/013_timescaledb.json b/website/benchmark/dbms/results/013_timescaledb.json new file mode 100644 index 00000000000..fb829b0b040 --- /dev/null +++ b/website/benchmark/dbms/results/013_timescaledb.json @@ -0,0 +1,56 @@ +[ + { + "system": "TimescaleDB", + "version": "2021-10-30", + "data_size": 100000000, + "time": "", + "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] + ] + } +] + From 85fa26f2e85689f8f30257cfad8102e98f870871 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 Oct 2021 23:23:00 +0300 Subject: [PATCH 859/919] Add usability testing --- benchmark/timescaledb/usability.md | 1638 ++++++++++++++++++++++++++++ 1 file changed, 1638 insertions(+) create mode 100644 benchmark/timescaledb/usability.md diff --git a/benchmark/timescaledb/usability.md b/benchmark/timescaledb/usability.md new file mode 100644 index 00000000000..1e491ac3f35 --- /dev/null +++ b/benchmark/timescaledb/usability.md @@ -0,0 +1,1638 @@ +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). + + +## 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:~/example_datasets/weather2$ 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:~/example_datasets/weather2$ psql +psql: error: connection to server on socket "/var/run/postgresql/.s.PGSQL.5432" failed: FATAL: role "milovidov" does not exist +milovidov@mtlog-perftest03j:~/example_datasets/weather2$ 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:~/example_datasets/weather2$ 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:~/example_datasets/weather2$ psql -U postgres -h localost +psql: error: could not translate host name "localost" to address: Name or service not known +milovidov@mtlog-perftest03j:~/example_datasets/weather2$ sudo psql -U postgres -h localost +psql: error: could not translate host name "localost" to address: Name or service not known +milovidov@mtlog-perftest03j:~/example_datasets/weather2$ 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:~/example_datasets/weather2$ 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:~/example_datasets/weather2$ 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:~/example_datasets/weather2$ 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:~/example_datasets/weather2$ 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:~/example_datasets/weather2$ 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:~/example_datasets/weather2$ 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:~/example_datasets/weather2$ 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:~/example_datasets/weather2$ 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:~/example_datasets/weather2$ 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:~/example_datasets/weather2$ 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:~/example_datasets/weather2$ 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:~/example_datasets/weather2$ 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:~/example_datasets/weather2$ 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? From 2424c546bdbff9a86367ccbf085b4479b7a8749c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 Oct 2021 23:35:20 +0300 Subject: [PATCH 860/919] Update --- benchmark/timescaledb/usability.md | 54 ++++++++++++++++++------------ 1 file changed, 32 insertions(+), 22 deletions(-) diff --git a/benchmark/timescaledb/usability.md b/benchmark/timescaledb/usability.md index 1e491ac3f35..0c19521f59f 100644 --- a/benchmark/timescaledb/usability.md +++ b/benchmark/timescaledb/usability.md @@ -71,23 +71,23 @@ psql: error: connection to server at "localhost" (::1), port 5432 failed: fe_sen How to set up password? ``` -milovidov@mtlog-perftest03j:~/example_datasets/weather2$ psql -U postgres -h localhost +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:~/example_datasets/weather2$ psql +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:~/example_datasets/weather2$ sudo psql +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:~/example_datasets/weather2$ psql -U postgres +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:~/example_datasets/weather2$ psql -U postgres -h localost +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:~/example_datasets/weather2$ sudo psql -U postgres -h localost +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:~/example_datasets/weather2$ sudo psql -U postgres -h localhost +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:~/example_datasets/weather2$ sudo -u postgres psql -h localhost +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 ``` @@ -242,7 +242,8 @@ sudo mcedit /etc/postgresql/13/main/postgresql.conf #listen_addresses = 'localhost' ``` -Looks like I need to uncomment it. +Looks like I need to uncomment it.ERROR: cannot change configuration on already compressed chunks +DETAIL: There are compressed chunks that prevent changing the existing compression configuration. ``` sudo service postgresql restart @@ -310,13 +311,13 @@ sudo service postgresql restart But now it does not accept password: ``` -milovidov@mtlog-perftest03j:~/example_datasets/weather2$ sudo -u postgres psql -h 127.0.0.1 +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:~/example_datasets/weather2$ sudo -u postgres psql -h 127.0.0.1 --password '' +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:~/example_datasets/weather2$ sudo -u postgres psql -h 127.0.0.1 +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 ``` @@ -875,7 +876,7 @@ 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:~/example_datasets/weather2$ sudo -u postgres psql +milovidov@mtlog-perftest03j:~$ sudo -u postgres psql psql (13.4 (Ubuntu 13.4-4.pgdg18.04+1)) Type "help" for help. @@ -1083,7 +1084,7 @@ sudo apt install llvm It does not help: ``` -milovidov@mtlog-perftest03j:~/example_datasets/weather2$ sudo -u postgres psql +milovidov@mtlog-perftest03j:~$ sudo -u postgres psql psql (13.4 (Ubuntu 13.4-4.pgdg18.04+1)) Type "help" for help. @@ -1097,16 +1098,16 @@ tutorial=# Dependency on system libraries is harmful. ``` -milovidov@mtlog-perftest03j:~/example_datasets/weather2$ ls -l /usr/lib/x86_64-linux-gnu/libLLVM-6.0.so +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:~/example_datasets/weather2$ ls -l /usr/lib/x86_64-linux-gnu/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:~/example_datasets/weather2$ dpkg -S libLLVM-6.0.so.1 +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 ``` @@ -1114,7 +1115,7 @@ libllvm6.0:amd64: /usr/lib/x86_64-linux-gnu/libLLVM-6.0.so.1 Wow, it's absolutely broken: ``` -milovidov@mtlog-perftest03j:~/example_datasets/weather2$ sudo apt remove llvm-6.0-dev +milovidov@mtlog-perftest03j:~$ sudo apt remove llvm-6.0-dev Reading package lists... Done Building dependency tree Reading state information... Done @@ -1133,7 +1134,7 @@ 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:~/example_datasets/weather2$ sudo apt install llvm-6.0-dev +milovidov@mtlog-perftest03j:~$ sudo apt install llvm-6.0-dev Reading package lists... Done Building dependency tree Reading state information... Done @@ -1153,9 +1154,9 @@ 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:~/example_datasets/weather2$ ls -l /usr/lib/x86_64-linux-gnu/libLLVM-6.0.so +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:~/example_datasets/weather2$ ls -l /usr/lib/x86_64-linux-gnu/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 ``` @@ -1608,7 +1609,7 @@ SELECT add_compression_policy('hits_100m_obfuscated', INTERVAL '0 seconds'); ``` ``` -milovidov@mtlog-perftest03j:~/ClickHouse/benchmark/timescaledb$ sudo -u postgres psql tutorial +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. @@ -1636,3 +1637,12 @@ ALTER TABLE hits_100m_obfuscated ``` 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. From e3f478c641e5bbd2ec1c2546456184199b29e784 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 Oct 2021 02:25:58 +0300 Subject: [PATCH 861/919] Add results from TimescaleDB with compression --- benchmark/timescaledb/log_compressed | 129 ++++++++++++++++++ .../dbms/results/014_timescaledb.json | 56 ++++++++ 2 files changed, 185 insertions(+) create mode 100644 benchmark/timescaledb/log_compressed create mode 100644 website/benchmark/dbms/results/014_timescaledb.json diff --git a/benchmark/timescaledb/log_compressed b/benchmark/timescaledb/log_compressed new file mode 100644 index 00000000000..235f659791f --- /dev/null +++ b/benchmark/timescaledb/log_compressed @@ -0,0 +1,129 @@ +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/website/benchmark/dbms/results/014_timescaledb.json b/website/benchmark/dbms/results/014_timescaledb.json new file mode 100644 index 00000000000..a1e63201c15 --- /dev/null +++ b/website/benchmark/dbms/results/014_timescaledb.json @@ -0,0 +1,56 @@ +[ + { + "system": "TimescaleDB (compressed)", + "version": "2021-10-31", + "data_size": 100000000, + "time": "", + "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] + ] + } +] + From 545289bd195908aac24f462ce286145594936a4f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 Oct 2021 02:26:39 +0300 Subject: [PATCH 862/919] Update --- benchmark/timescaledb/usability.md | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/benchmark/timescaledb/usability.md b/benchmark/timescaledb/usability.md index 0c19521f59f..207a1dfa2b6 100644 --- a/benchmark/timescaledb/usability.md +++ b/benchmark/timescaledb/usability.md @@ -1,6 +1,8 @@ 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 @@ -1646,3 +1648,17 @@ DETAIL: There are compressed chunks that prevent changing the existing compress ``` 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. From 42596b16bc95d5da882ee91afe1fea6ba516fc4f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 27 Aug 2021 01:15:24 +0300 Subject: [PATCH 863/919] Make StorageLog.cpp to implement both Log and TinyLog engines. Refactoring of implementation of the Log family. --- src/Common/FileChecker.cpp | 33 +- src/Common/FileChecker.h | 14 +- src/Storages/StorageLog.cpp | 581 +++++++++++++--------- src/Storages/StorageLog.h | 83 ++-- src/Storages/StorageTinyLog.cpp | 594 ----------------------- src/Storages/StorageTinyLog.h | 82 ---- src/Storages/registerStorages.cpp | 2 - src/Storages/tests/gtest_storage_log.cpp | 3 +- 8 files changed, 418 insertions(+), 974 deletions(-) delete mode 100644 src/Storages/StorageTinyLog.cpp delete mode 100644 src/Storages/StorageTinyLog.h diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index 14705a9b0b0..88dece64e5f 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -15,6 +15,7 @@ namespace DB namespace ErrorCodes { extern const int UNEXPECTED_END_OF_FILE; + extern const int LOGICAL_ERROR; } @@ -39,7 +40,9 @@ void FileChecker::setPath(const String & file_info_path_) void FileChecker::update(const String & full_file_path) { - map[fileName(full_file_path)] = disk->getFileSize(full_file_path); + bool exists = disk->exists(full_file_path); + auto real_size = exists ? disk->getFileSize(full_file_path) : 0; /// No race condition assuming no one else is working with these files. + map[fileName(full_file_path)] = real_size; } void FileChecker::setEmpty(const String & full_file_path) @@ -47,11 +50,19 @@ void FileChecker::setEmpty(const String & full_file_path) map[fileName(full_file_path)] = 0; } -FileChecker::Map FileChecker::getFileSizes() const +const FileChecker::Map & FileChecker::getFileSizes() const { return map; } +size_t FileChecker::getFileSize(const String & full_file_path) const +{ + auto it = map.find(fileName(full_file_path)); + if (it == map.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "File {} is not added to the file checker", full_file_path); + return it->second; +} + CheckResults FileChecker::check() const { if (map.empty()) @@ -63,18 +74,18 @@ CheckResults FileChecker::check() const { const String & name = name_size.first; String path = parentPath(files_info_path) + name; - if (!disk->exists(path)) + bool exists = disk->exists(path); + auto real_size = exists ? disk->getFileSize(path) : 0; /// No race condition assuming no one else is working with these files. + + if (real_size != name_size.second) { - results.emplace_back(name, false, "File " + path + " doesn't exist"); + String failure_message = exists + ? ("Size of " + path + " is wrong. Size is " + toString(real_size) + " but should be " + toString(name_size.second)) + : ("File " + path + " doesn't exist"); + results.emplace_back(name, false, failure_message); break; } - auto real_size = disk->getFileSize(path); - if (real_size != name_size.second) - { - results.emplace_back(name, false, "Size of " + path + " is wrong. Size is " + toString(real_size) + " but should be " + toString(name_size.second)); - break; - } results.emplace_back(name, true, ""); } @@ -97,7 +108,7 @@ void FileChecker::repair() if (real_size > expected_size) { - LOG_WARNING(&Poco::Logger::get("FileChecker"), "Will truncate file {} that has size {} to size {}", path, real_size, expected_size); + LOG_WARNING(log, "Will truncate file {} that has size {} to size {}", path, real_size, expected_size); disk->truncateFile(path, expected_size); } } diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index 688c2b78486..60003e0cabc 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -8,7 +8,7 @@ namespace DB { -/// stores the sizes of all columns, and can check whether the columns are corrupted +/// Stores the sizes of all columns, and can check whether the columns are corrupted. class FileChecker { public: @@ -31,17 +31,19 @@ public: /// File name -> size. using Map = std::map; - Map getFileSizes() const; + const Map & getFileSizes() const; + + /// Returns stored file size. + size_t getFileSize(const String & full_file_path) const; private: void load(); - DiskPtr disk; + const DiskPtr disk; + const Poco::Logger * log = &Poco::Logger::get("FileChecker"); + String files_info_path; - Map map; - - Poco::Logger * log = &Poco::Logger::get("FileChecker"); }; } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 54a8e4cd1ce..c4e9826e3a6 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -21,8 +21,8 @@ #include #include #include "StorageLogSettings.h" -#include #include +#include #include #include @@ -47,6 +47,8 @@ namespace ErrorCodes extern const int INCORRECT_FILE_NAME; } +/// NOTE: The lock `StorageLog::rwlock` is NOT kept locked while reading, +/// because we read ranges of data that do not change. class LogSource final : public SourceWithProgress { public: @@ -61,14 +63,14 @@ public: } LogSource( - size_t block_size_, const NamesAndTypesList & columns_, StorageLog & storage_, - size_t mark_number_, size_t rows_limit_, ReadSettings read_settings_) + size_t block_size_, const NamesAndTypesList & columns_, const StorageLog & storage_, + size_t rows_limit_, const std::vector & offsets_, ReadSettings read_settings_) : SourceWithProgress(getHeader(columns_)), block_size(block_size_), columns(columns_), storage(storage_), - mark_number(mark_number_), rows_limit(rows_limit_), + offsets(offsets_), read_settings(std::move(read_settings_)) { } @@ -81,16 +83,18 @@ protected: private: size_t block_size; NamesAndTypesList columns; - StorageLog & storage; - size_t mark_number; /// from what mark to read data + const StorageLog & storage; size_t rows_limit; /// The maximum number of rows that can be read size_t rows_read = 0; + bool is_finished = false; + std::vector offsets; ReadSettings read_settings; - std::unordered_map serializations; - struct Stream { + /// We use `disk->getFileSize(data_path)` to get the size of the file here. + /// We cannot just use `storage.file_checker` for that purpose, because `storage.rwlock` is not locked at this point. + Stream(const DiskPtr & disk, const String & data_path, size_t offset, ReadSettings read_settings_) : plain(disk->readFile(data_path, read_settings_.adjustBufferSize(disk->getFileSize(data_path)))) , compressed(*plain) @@ -111,19 +115,25 @@ private: DeserializeStates deserialize_states; void readData(const NameAndTypePair & name_and_type, ColumnPtr & column, size_t max_rows_to_read, ISerialization::SubstreamsCache & cache); + bool isFinished(); }; Chunk LogSource::generate() { - Block res; - - if (rows_read == rows_limit) + if (isFinished()) + { + /// Close the files (before destroying the object). + /// When many sources are created, but simultaneously reading only a few of them, + /// buffers don't waste memory. + streams.clear(); return {}; + } /// How many rows to read for the next block. size_t max_rows_to_read = std::min(block_size, rows_limit - rows_read); std::unordered_map caches; + Block res; for (const auto & name_type : columns) { @@ -146,12 +156,14 @@ Chunk LogSource::generate() if (res) rows_read += res.rows(); - if (!res || rows_read == rows_limit) + if (!res) + is_finished = true; + + if (isFinished()) { - /** Close the files (before destroying the object). - * When many sources are created, but simultaneously reading only a few of them, - * buffers don't waste memory. - */ + /// Close the files (before destroying the object). + /// When many sources are created, but simultaneously reading only a few of them, + /// buffers don't waste memory. streams.clear(); } @@ -174,21 +186,16 @@ void LogSource::readData(const NameAndTypePair & name_and_type, ColumnPtr & colu if (cache.count(ISerialization::getSubcolumnNameForStream(path))) return nullptr; - String stream_name = ISerialization::getFileNameForStream(name_and_type, path); - const auto & file_it = storage.files.find(stream_name); - if (storage.files.end() == file_it) - throw Exception("Logical error: no information about file " + stream_name + " in StorageLog", ErrorCodes::LOGICAL_ERROR); + String data_file_name = ISerialization::getFileNameForStream(name_and_type, path); - UInt64 offset = 0; - if (!stream_for_prefix && mark_number) - { - std::lock_guard marks_lock(file_it->second.marks_mutex); - offset = file_it->second.marks[mark_number].offset; - } + const auto & data_file_it = storage.data_files_by_names.find(data_file_name); + if (data_file_it == storage.data_files_by_names.end()) + throw Exception("Logical error: no information about file " + data_file_name + " in StorageLog", ErrorCodes::LOGICAL_ERROR); + const auto & data_file = *data_file_it->second; - auto & data_file_path = file_it->second.data_file_path; - auto it = streams.try_emplace(stream_name, storage.disk, data_file_path, offset, read_settings).first; + size_t offset = stream_for_prefix ? 0 : offsets[data_file.index]; + auto it = streams.try_emplace(data_file_name, storage.disk, data_file.path, offset, read_settings).first; return &it->second.compressed; }; }; @@ -203,28 +210,49 @@ void LogSource::readData(const NameAndTypePair & name_and_type, ColumnPtr & colu serialization->deserializeBinaryBulkWithMultipleStreams(column, max_rows_to_read, settings, deserialize_states[name], &cache); } +bool LogSource::isFinished() +{ + if (is_finished) + return true; + if (rows_limit == std::numeric_limits::max()) + { + /// No limit for the row count, check for EOF. + if (!streams.empty() && streams.begin()->second.compressed.eof()) + is_finished = true; + } + else + { + /// There is a limit for the row count, check that limit. + if (rows_read == rows_limit) + is_finished = true; + } + + return is_finished; +} + + +/// NOTE: The lock `StorageLog::rwlock` is kept locked in exclusive mode while writing. class LogSink final : public SinkToStorage { public: + using WriteLock = std::unique_lock; + explicit LogSink( - StorageLog & storage_, const StorageMetadataPtr & metadata_snapshot_, std::unique_lock && lock_) + StorageLog & storage_, const StorageMetadataPtr & metadata_snapshot_, WriteLock && lock_) : SinkToStorage(metadata_snapshot_->getSampleBlock()) , storage(storage_) , metadata_snapshot(metadata_snapshot_) , lock(std::move(lock_)) - , marks_stream(storage.disk->writeFile(storage.marks_file_path, 4096, WriteMode::Append)) { if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - /// If there were no files, add info to rollback in case of error. - if (storage.file_checker.empty()) - { - for (const auto & file : storage.files) - storage.file_checker.setEmpty(file.second.data_file_path); - storage.file_checker.save(); - } + /// Ensure that marks are loaded because we're going to update them. + storage.loadMarks(lock); + + /// If there were no files, save zero file sizes to be able to rollback in case of error. + storage.saveFileSizes(lock); } String getName() const override { return "LogSink"; } @@ -236,8 +264,15 @@ public: if (!done) { /// Rollback partial writes. + + /// No more writing. streams.clear(); + + /// Truncate files to the older sizes. storage.file_checker.repair(); + + /// Remove excessive marks. + storage.removeUnsavedMarks(lock); } } catch (...) @@ -252,22 +287,26 @@ public: private: StorageLog & storage; StorageMetadataPtr metadata_snapshot; - std::unique_lock lock; + WriteLock lock; bool done = false; struct Stream { - Stream(const DiskPtr & disk, const String & data_path, CompressionCodecPtr codec, size_t max_compress_block_size) : + Stream(const DiskPtr & disk, const String & data_path, size_t initial_data_size, CompressionCodecPtr codec, size_t max_compress_block_size) : plain(disk->writeFile(data_path, max_compress_block_size, WriteMode::Append)), compressed(*plain, std::move(codec), max_compress_block_size), - plain_offset(disk->getFileSize(data_path)) + plain_offset(initial_data_size) { } std::unique_ptr plain; CompressedWriteBuffer compressed; - size_t plain_offset; /// How many bytes were in the file at the time the LogBlockOutputStream was created. + /// How many bytes were in the file at the time the Stream was created. + size_t plain_offset; + + /// Used to not write shared offsets of columns for nested structures multiple times. + bool written = false; void finalize() { @@ -276,29 +315,16 @@ private: } }; - using Mark = StorageLog::Mark; - using MarksForColumns = std::vector>; - using FileStreams = std::map; FileStreams streams; - using WrittenStreams = std::set; - - std::unique_ptr marks_stream; /// Declared below `lock` to make the file open when rwlock is captured. - using SerializeState = ISerialization::SerializeBinaryBulkStatePtr; using SerializeStates = std::map; SerializeStates serialize_states; - ISerialization::OutputStreamGetter createStreamGetter(const NameAndTypePair & name_and_type, WrittenStreams & written_streams); + ISerialization::OutputStreamGetter createStreamGetter(const NameAndTypePair & name_and_type); - void writeData( - const NameAndTypePair & name_and_type, - const IColumn & column, - MarksForColumns & out_marks, - WrittenStreams & written_streams); - - void writeMarks(MarksForColumns && marks); + void writeData(const NameAndTypePair & name_and_type, const IColumn & column); }; @@ -307,19 +333,14 @@ void LogSink::consume(Chunk chunk) auto block = getHeader().cloneWithColumns(chunk.detachColumns()); metadata_snapshot->check(block, true); - /// The set of written offset columns so that you do not write shared offsets of columns for nested structures multiple times - WrittenStreams written_streams; - - MarksForColumns marks; - marks.reserve(storage.file_count); + for (auto & stream : streams | boost::adaptors::map_values) + stream.written = false; for (size_t i = 0; i < block.columns(); ++i) { const ColumnWithTypeAndName & column = block.safeGetByPosition(i); - writeData(NameAndTypePair(column.name, column.type), *column.column, marks, written_streams); + writeData(NameAndTypePair(column.name, column.type), *column.column); } - - writeMarks(std::move(marks)); } @@ -328,36 +349,29 @@ void LogSink::onFinish() if (done) return; - WrittenStreams written_streams; + for (auto & stream : streams | boost::adaptors::map_values) + stream.written = false; + ISerialization::SerializeBinaryBulkSettings settings; for (const auto & column : getHeader()) { auto it = serialize_states.find(column.name); if (it != serialize_states.end()) { - settings.getter = createStreamGetter(NameAndTypePair(column.name, column.type), written_streams); + settings.getter = createStreamGetter(NameAndTypePair(column.name, column.type)); auto serialization = column.type->getDefaultSerialization(); serialization->serializeBinaryBulkStateSuffix(settings, it->second); } } /// Finish write. - marks_stream->next(); - marks_stream->finalize(); - - for (auto & name_stream : streams) - name_stream.second.finalize(); - - Strings column_files; - for (const auto & name_stream : streams) - column_files.push_back(storage.files[name_stream.first].data_file_path); - column_files.push_back(storage.marks_file_path); - - for (const auto & file : column_files) - storage.file_checker.update(file); - storage.file_checker.save(); - + for (auto & stream : streams | boost::adaptors::map_values) + stream.finalize(); streams.clear(); + + storage.saveMarks(lock); + storage.saveFileSizes(lock); + done = true; /// unlock should be done from the same thread as lock, and dtor may be @@ -367,26 +381,26 @@ void LogSink::onFinish() } -ISerialization::OutputStreamGetter LogSink::createStreamGetter(const NameAndTypePair & name_and_type, - WrittenStreams & written_streams) +ISerialization::OutputStreamGetter LogSink::createStreamGetter(const NameAndTypePair & name_and_type) { return [&] (const ISerialization::SubstreamPath & path) -> WriteBuffer * { - String stream_name = ISerialization::getFileNameForStream(name_and_type, path); - if (written_streams.count(stream_name)) + String data_file_name = ISerialization::getFileNameForStream(name_and_type, path); + auto it = streams.find(data_file_name); + if (it == streams.end()) + throw Exception("Logical error: stream was not created when writing data in LogSink", + ErrorCodes::LOGICAL_ERROR); + + Stream & stream = it->second; + if (stream.written) return nullptr; - auto it = streams.find(stream_name); - if (streams.end() == it) - throw Exception("Logical error: stream was not created when writing data in LogBlockOutputStream", - ErrorCodes::LOGICAL_ERROR); - return &it->second.compressed; + return &stream.compressed; }; } -void LogSink::writeData(const NameAndTypePair & name_and_type, const IColumn & column, - MarksForColumns & out_marks, WrittenStreams & written_streams) +void LogSink::writeData(const NameAndTypePair & name_and_type, const IColumn & column) { ISerialization::SerializeBinaryBulkSettings settings; const auto & [name, type] = name_and_type; @@ -394,79 +408,97 @@ void LogSink::writeData(const NameAndTypePair & name_and_type, const IColumn & c serialization->enumerateStreams([&] (const ISerialization::SubstreamPath & path) { - String stream_name = ISerialization::getFileNameForStream(name_and_type, path); - if (written_streams.count(stream_name)) - return; + String data_file_name = ISerialization::getFileNameForStream(name_and_type, path); + auto it = streams.find(data_file_name); + if (it == streams.end()) + { + const auto & data_file_it = storage.data_files_by_names.find(data_file_name); + if (data_file_it == storage.data_files_by_names.end()) + throw Exception("Logical error: no information about file " + data_file_name + " in StorageLog", ErrorCodes::LOGICAL_ERROR); - const auto & columns = metadata_snapshot->getColumns(); - streams.try_emplace( - stream_name, - storage.disk, - storage.files[stream_name].data_file_path, - columns.getCodecOrDefault(name_and_type.name), - storage.max_compress_block_size); + const auto & data_file = *data_file_it->second; + const auto & columns = metadata_snapshot->getColumns(); + + it = streams.try_emplace(data_file.name, storage.disk, data_file.path, + storage.file_checker.getFileSize(data_file.path), + columns.getCodecOrDefault(name_and_type.name), + storage.max_compress_block_size).first; + } + + auto & stream = it->second; + if (stream.written) + return; }); - settings.getter = createStreamGetter(name_and_type, written_streams); + settings.getter = createStreamGetter(name_and_type); if (serialize_states.count(name) == 0) serialization->serializeBinaryBulkStatePrefix(settings, serialize_states[name]); - serialization->enumerateStreams([&] (const ISerialization::SubstreamPath & path) + if (storage.use_marks_file) { - String stream_name = ISerialization::getFileNameForStream(name_and_type, path); - if (written_streams.count(stream_name)) - return; + serialization->enumerateStreams([&] (const ISerialization::SubstreamPath & path) + { + String data_file_name = ISerialization::getFileNameForStream(name_and_type, path); + const auto & stream = streams.at(data_file_name); + if (stream.written) + return; - const auto & file = storage.files[stream_name]; - const auto stream_it = streams.find(stream_name); - - Mark mark; - mark.rows = (file.marks.empty() ? 0 : file.marks.back().rows) + column.size(); - mark.offset = stream_it->second.plain_offset + stream_it->second.plain->count(); - - out_marks.emplace_back(file.column_index, mark); - }); + auto & data_file = *storage.data_files_by_names.at(data_file_name); + auto & marks = data_file.marks; + size_t prev_num_rows = marks.empty() ? 0 : marks.back().rows; + auto & mark = marks.emplace_back(); + mark.rows = prev_num_rows + column.size(); + mark.offset = stream.plain_offset + stream.plain->count(); + }); + } serialization->serializeBinaryBulkWithMultipleStreams(column, 0, 0, settings, serialize_states[name]); serialization->enumerateStreams([&] (const ISerialization::SubstreamPath & path) { - String stream_name = ISerialization::getFileNameForStream(name_and_type, path); - if (!written_streams.emplace(stream_name).second) + String data_file_name = ISerialization::getFileNameForStream(name_and_type, path); + auto & stream = streams.at(data_file_name); + if (stream.written) return; - auto it = streams.find(stream_name); - if (streams.end() == it) - throw Exception("Logical error: stream was not created when writing data in LogBlockOutputStream", ErrorCodes::LOGICAL_ERROR); - it->second.compressed.next(); + stream.written = true; + stream.compressed.next(); }); } -void LogSink::writeMarks(MarksForColumns && marks) +void StorageLog::Mark::write(WriteBuffer & out) const { - if (marks.size() != storage.file_count) - throw Exception("Wrong number of marks generated from block. Makes no sense.", ErrorCodes::LOGICAL_ERROR); - - std::sort(marks.begin(), marks.end(), [](const auto & a, const auto & b) { return a.first < b.first; }); - - for (const auto & mark : marks) - { - writeIntBinary(mark.second.rows, *marks_stream); - writeIntBinary(mark.second.offset, *marks_stream); - - size_t column_index = mark.first; - - auto & file = storage.files[storage.column_names_by_idx[column_index]]; - std::lock_guard marks_lock(file.marks_mutex); - file.marks.push_back(mark.second); - } + writeIntBinary(rows, out); + writeIntBinary(offset, out); } + +void StorageLog::Mark::read(ReadBuffer & in) +{ + readIntBinary(rows, in); + readIntBinary(offset, in); +} + + +namespace +{ + /// NOTE: We extract the number of rows from the marks. + /// For normal columns, the number of rows in the block is specified in the marks. + /// For array columns and nested structures, there are more than one group of marks that correspond to different files + /// - for elements (file name.bin) - the total number of array elements in the block is specified, + /// - for array sizes (file name.size0.bin) - the number of rows (the whole arrays themselves) in the block is specified. + /// So for Array data type, first stream is array sizes; and number of array sizes is the number of arrays. + /// Thus we assume we can always get the real number of rows from the first column. + constexpr size_t INDEX_WITH_REAL_ROW_COUNT = 0; +} + + StorageLog::~StorageLog() = default; StorageLog::StorageLog( + const String & engine_name_, DiskPtr disk_, const String & relative_path_, const StorageID & table_id_, @@ -476,10 +508,13 @@ StorageLog::StorageLog( bool attach, size_t max_compress_block_size_) : IStorage(table_id_) + , engine_name(engine_name_) , disk(std::move(disk_)) , table_path(relative_path_) - , max_compress_block_size(max_compress_block_size_) + , use_marks_file(engine_name == "Log") + , marks_file_path(table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME) , file_checker(disk, table_path + "sizes.json") + , max_compress_block_size(max_compress_block_size_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); @@ -490,6 +525,19 @@ StorageLog::StorageLog( if (relative_path_.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); + /// Enumerate data files. + for (const auto & column : storage_metadata.getColumns().getAllPhysical()) + addDataFiles(column); + + /// Ensure the file checker is initialized. + if (file_checker.empty()) + { + for (const auto & data_file : data_files) + file_checker.setEmpty(data_file.path); + if (use_marks_file) + file_checker.setEmpty(marks_file_path); + } + if (!attach) { /// create directories if they do not exist @@ -506,80 +554,138 @@ StorageLog::StorageLog( tryLogCurrentException(__PRETTY_FUNCTION__); } } - - for (const auto & column : storage_metadata.getColumns().getAllPhysical()) - addFiles(column); - - marks_file_path = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME; } -void StorageLog::addFiles(const NameAndTypePair & column) +void StorageLog::addDataFiles(const NameAndTypePair & column) { - if (files.end() != files.find(column.name)) + if (data_files_by_names.contains(column.name)) throw Exception("Duplicate column with name " + column.name + " in constructor of StorageLog.", ErrorCodes::DUPLICATE_COLUMN); ISerialization::StreamCallback stream_callback = [&] (const ISerialization::SubstreamPath & substream_path) { - String stream_name = ISerialization::getFileNameForStream(column, substream_path); - - if (!files.count(stream_name)) + String data_file_name = ISerialization::getFileNameForStream(column, substream_path); + if (!data_files_by_names.contains(data_file_name)) { - ColumnData & column_data = files[stream_name]; - column_data.column_index = file_count; - column_data.data_file_path = table_path + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION; - - column_names_by_idx.push_back(stream_name); - ++file_count; + DataFile & data_file = data_files.emplace_back(); + data_file.name = data_file_name; + data_file.path = table_path + data_file_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION; + data_file.index = num_data_files++; + data_files_by_names.emplace(data_file_name, nullptr); } }; - auto serialization = column.type->getDefaultSerialization(); - serialization->enumerateStreams(stream_callback); + column.type->getDefaultSerialization()->enumerateStreams(stream_callback); + + for (auto & data_file : data_files) + data_files_by_names[data_file.name] = &data_file; } void StorageLog::loadMarks(std::chrono::seconds lock_timeout) { - std::unique_lock lock(rwlock, lock_timeout); + if (!use_marks_file || marks_loaded) + return; + + /// We load marks with an exclusive lock (i.e. the write lock) because we don't want + /// a data race between two threads trying to load marks simultaneously. + WriteLock lock{rwlock, lock_timeout}; if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - if (loaded_marks) + loadMarks(lock); +} + +void StorageLog::loadMarks(const WriteLock & /* already locked exclusively */) +{ + if (!use_marks_file || marks_loaded) return; - using FilesByIndex = std::vector; - - FilesByIndex files_by_index(file_count); - for (Files::iterator it = files.begin(); it != files.end(); ++it) - files_by_index[it->second.column_index] = it; - + size_t num_marks = 0; if (disk->exists(marks_file_path)) { size_t file_size = disk->getFileSize(marks_file_path); - if (file_size % (file_count * sizeof(Mark)) != 0) + if (file_size % (num_data_files * sizeof(Mark)) != 0) throw Exception("Size of marks file is inconsistent", ErrorCodes::SIZES_OF_MARKS_FILES_ARE_INCONSISTENT); - size_t marks_count = file_size / (file_count * sizeof(Mark)); + num_marks = file_size / (num_data_files * sizeof(Mark)); - for (auto & file : files_by_index) - file->second.marks.reserve(marks_count); + for (auto & data_file : data_files) + data_file.marks.resize(num_marks); std::unique_ptr marks_rb = disk->readFile(marks_file_path, ReadSettings().adjustBufferSize(32768)); - while (!marks_rb->eof()) + for (size_t i = 0; i != num_marks; ++i) { - for (auto & file : files_by_index) + for (auto & data_file : data_files) { Mark mark; - readIntBinary(mark.rows, *marks_rb); - readIntBinary(mark.offset, *marks_rb); - file->second.marks.push_back(mark); + mark.read(*marks_rb); + data_file.marks[i] = mark; } } } - loaded_marks = true; + marks_loaded = true; + num_marks_saved = num_marks; +} + +void StorageLog::saveMarks(const WriteLock & /* already locked for writing */) +{ + if (!use_marks_file) + return; + + size_t num_marks = num_data_files ? data_files[0].marks.size() : 0; + if (num_marks_saved == num_marks) + return; + + for (const auto & data_file : data_files) + { + if (data_file.marks.size() != num_marks) + throw Exception("Wrong number of marks generated from block. Makes no sense.", ErrorCodes::LOGICAL_ERROR); + } + + size_t start = num_marks_saved; + auto marks_stream = disk->writeFile(marks_file_path, 4096, WriteMode::Append); + + for (size_t i = start; i != num_marks; ++i) + { + for (const auto & data_file : data_files) + { + const auto & mark = data_file.marks[i]; + mark.write(*marks_stream); + } + } + + marks_stream->next(); + marks_stream->finalize(); + + num_marks_saved = num_marks; +} + + +void StorageLog::removeUnsavedMarks(const WriteLock & /* already locked for writing */) +{ + if (!use_marks_file) + return; + + for (auto & data_file : data_files) + { + if (data_file.marks.size() > num_marks_saved) + data_file.marks.resize(num_marks_saved); + } +} + + +void StorageLog::saveFileSizes(const WriteLock & /* already locked for writing */) +{ + for (const auto & data_file : data_files) + file_checker.update(data_file.path); + + if (use_marks_file) + file_checker.update(marks_file_path); + + file_checker.save(); } @@ -592,52 +698,29 @@ void StorageLog::rename(const String & new_path_to_table_data, const StorageID & table_path = new_path_to_table_data; file_checker.setPath(table_path + "sizes.json"); - for (auto & file : files) - file.second.data_file_path = table_path + fileName(file.second.data_file_path); + for (auto & data_file : data_files) + data_file.path = table_path + fileName(data_file.path); marks_file_path = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME; } renameInMemory(new_table_id); } -void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) +void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) { - files.clear(); - file_count = 0; - loaded_marks = false; - disk->clearDirectory(table_path); - for (const auto & column : metadata_snapshot->getColumns().getAllPhysical()) - addFiles(column); - - file_checker = FileChecker{disk, table_path + "sizes.json"}; - marks_file_path = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME; -} - - -const StorageLog::Marks & StorageLog::getMarksWithRealRowCount(const StorageMetadataPtr & metadata_snapshot) const -{ - /// There should be at least one physical column - auto column = *metadata_snapshot->getColumns().getAllPhysical().begin(); - String filename; - - /** We take marks from first column. - * If this is a data type with multiple stream, get the first stream, that we assume have real row count. - * (Example: for Array data type, first stream is array sizes; and number of array sizes is the number of arrays). - */ - auto serialization = column.type->getDefaultSerialization(); - serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) + for (auto & data_file : data_files) { - if (filename.empty()) - filename = ISerialization::getFileNameForStream(column, substream_path); - }); + data_file.marks.clear(); + file_checker.setEmpty(data_file.path); + } - Files::const_iterator it = files.find(filename); - if (files.end() == it) - throw Exception("Cannot find file " + filename, ErrorCodes::LOGICAL_ERROR); + if (use_marks_file) + file_checker.setEmpty(marks_file_path); - return it->second.marks; + marks_loaded = true; + num_marks_saved = 0; } @@ -665,40 +748,53 @@ Pipe StorageLog::read( auto lock_timeout = getLockTimeout(context); loadMarks(lock_timeout); - auto all_columns = metadata_snapshot->getColumns().getByNames(ColumnsDescription::All, column_names, true); - all_columns = Nested::convertToSubcolumns(all_columns); - - std::shared_lock lock(rwlock, lock_timeout); + ReadLock lock{rwlock, lock_timeout}; if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - if (file_checker.empty()) + if (!num_data_files || !file_checker.getFileSize(data_files[INDEX_WITH_REAL_ROW_COUNT].path)) return Pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); - Pipes pipes; + const Marks & marks_with_real_row_count = data_files[INDEX_WITH_REAL_ROW_COUNT].marks; + size_t num_marks = marks_with_real_row_count.size(); - const Marks & marks = getMarksWithRealRowCount(metadata_snapshot); - size_t marks_size = marks.size(); + size_t max_streams = use_marks_file ? num_marks : 1; + if (num_streams > max_streams) + num_streams = max_streams; - if (num_streams > marks_size) - num_streams = marks_size; + auto all_columns = metadata_snapshot->getColumns().getByNames(ColumnsDescription::All, column_names, true); + all_columns = Nested::convertToSubcolumns(all_columns); + + std::vector offsets; + offsets.resize(num_data_files, 0); ReadSettings read_settings = context->getReadSettings(); + Pipes pipes; for (size_t stream = 0; stream < num_streams; ++stream) { - size_t mark_begin = stream * marks_size / num_streams; - size_t mark_end = (stream + 1) * marks_size / num_streams; - - size_t rows_begin = mark_begin ? marks[mark_begin - 1].rows : 0; - size_t rows_end = mark_end ? marks[mark_end - 1].rows : 0; + size_t start_row, end_row; + if (use_marks_file) + { + size_t mark_begin = stream * num_marks / num_streams; + size_t mark_end = (stream + 1) * num_marks / num_streams; + start_row = mark_begin ? marks_with_real_row_count[mark_begin - 1].rows : 0; + end_row = mark_end ? marks_with_real_row_count[mark_end - 1].rows : 0; + for (const auto & data_file : data_files) + offsets[data_file.index] = data_file.marks[mark_begin].offset; + } + else + { + start_row = 0; + end_row = std::numeric_limits::max(); // row limit not set + } pipes.emplace_back(std::make_shared( max_block_size, all_columns, *this, - mark_begin, - rows_end - rows_begin, + end_row - start_row, + offsets, read_settings)); } @@ -708,10 +804,7 @@ Pipe StorageLog::read( SinkToStoragePtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) { - auto lock_timeout = getLockTimeout(context); - loadMarks(lock_timeout); - - std::unique_lock lock(rwlock, lock_timeout); + WriteLock lock{rwlock, getLockTimeout(context)}; if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); @@ -720,7 +813,7 @@ SinkToStoragePtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetada CheckResults StorageLog::checkData(const ASTPtr & /* query */, ContextPtr context) { - std::shared_lock lock(rwlock, getLockTimeout(context)); + ReadLock lock{rwlock, getLockTimeout(context)}; if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); @@ -730,22 +823,23 @@ CheckResults StorageLog::checkData(const ASTPtr & /* query */, ContextPtr contex IStorage::ColumnSizeByName StorageLog::getColumnSizes() const { - std::shared_lock lock(rwlock, std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC)); + ReadLock lock{rwlock, std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC)}; if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); ColumnSizeByName column_sizes; - FileChecker::Map file_sizes = file_checker.getFileSizes(); for (const auto & column : getInMemoryMetadata().getColumns().getAllPhysical()) { ISerialization::StreamCallback stream_callback = [&, this] (const ISerialization::SubstreamPath & substream_path) { - String stream_name = ISerialization::getFileNameForStream(column, substream_path); - ColumnSize & size = column_sizes[column.name]; - auto it = files.find(stream_name); - if (it != files.end()) - size.data_compressed += file_sizes[fileName(it->second.data_file_path)]; + String data_file_name = ISerialization::getFileNameForStream(column, substream_path); + auto it = data_files_by_names.find(data_file_name); + if (it != data_files_by_names.end()) + { + const auto & data_file = *it->second; + column_sizes[column.name].data_compressed += file_checker.getFileSize(data_file.path); + } }; auto serialization = column.type->getDefaultSerialization(); @@ -755,13 +849,14 @@ IStorage::ColumnSizeByName StorageLog::getColumnSizes() const return column_sizes; } + void registerStorageLog(StorageFactory & factory) { StorageFactory::StorageFeatures features{ .supports_settings = true }; - factory.registerStorage("Log", [](const StorageFactory::Arguments & args) + auto create_fn = [](const StorageFactory::Arguments & args) { if (!args.engine_args.empty()) throw Exception( @@ -772,6 +867,7 @@ void registerStorageLog(StorageFactory & factory) DiskPtr disk = args.getContext()->getDisk(disk_name); return StorageLog::create( + args.engine_name, disk, args.relative_data_path, args.table_id, @@ -780,7 +876,10 @@ void registerStorageLog(StorageFactory & factory) args.comment, args.attach, args.getContext()->getSettings().max_compress_block_size); - }, features); + }; + + factory.registerStorage("Log", create_fn, features); + factory.registerStorage("TinyLog", create_fn, features); } } diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index ada971ecf43..ca87d7dcf3e 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -13,8 +13,11 @@ namespace DB { -/** Implements simple table engine without support of indices. +/** Implements Log - a simple table engine without support of indices. * The data is stored in a compressed form. + * + * Also implements TinyLog - a table engine that is suitable for small chunks of the log. + * It differs from Log in the absence of mark files. */ class StorageLog final : public shared_ptr_helper, public IStorage { @@ -24,7 +27,7 @@ class StorageLog final : public shared_ptr_helper, public IStorage public: ~StorageLog() override; - String getName() const override { return "Log"; } + String getName() const override { return engine_name; } Pipe read( const Names & column_names, @@ -41,7 +44,7 @@ public: CheckResults checkData(const ASTPtr & /* query */, ContextPtr /* context */) override; - void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) override; + void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; bool storesDataOnDisk() const override { return true; } Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; } @@ -54,6 +57,7 @@ protected: * consisting of the specified columns; Create files if they do not exist. */ StorageLog( + const String & engine_name_, DiskPtr disk_, const std::string & relative_path_, const StorageID & table_id_, @@ -64,6 +68,27 @@ protected: size_t max_compress_block_size_); private: + using ReadLock = std::shared_lock; + using WriteLock = std::unique_lock; + + /// The order of adding files should not change: it corresponds to the order of the columns in the marks file. + /// Should be called from the constructor only. + void addDataFiles(const NameAndTypePair & column); + + /// Reads the marks file if it hasn't read yet. + /// It is done lazily, so that with a large number of tables, the server starts quickly. + void loadMarks(std::chrono::seconds lock_timeout); + void loadMarks(const WriteLock &); + + /// Saves the marks file. + void saveMarks(const WriteLock &); + + /// Removes all unsaved marks. + void removeUnsavedMarks(const WriteLock &); + + /// Saves the sizes of the data and marks files. + void saveFileSizes(const WriteLock &); + /** Offsets to some row number in a file for column in table. * They are needed so that you can read the data in several threads. */ @@ -71,57 +96,41 @@ private: { size_t rows; /// How many rows are before this offset including the block at this offset. size_t offset; /// The offset in compressed file. + + void write(WriteBuffer & out) const; + void read(ReadBuffer & in); }; using Marks = std::vector; /// Column data - struct ColumnData + struct DataFile { - /// Specifies the column number in the marks file. - /// Does not necessarily match the column number among the columns of the table: columns with lengths of arrays are also numbered here. - size_t column_index; - - String data_file_path; - - std::mutex marks_mutex; + size_t index; + String name; + String path; Marks marks; }; - using Files = std::map; /// file name -> column data - DiskPtr disk; + const String engine_name; + const DiskPtr disk; String table_path; - mutable std::shared_timed_mutex rwlock; + std::vector data_files; + size_t num_data_files = 0; + std::map data_files_by_names; - Files files; - - Names column_names_by_idx; /// column_index -> name + /// The Log engine uses the marks file, and the TinyLog engine doesn't. + const bool use_marks_file; String marks_file_path; - - /// The order of adding files should not change: it corresponds to the order of the columns in the marks file. - void addFiles(const NameAndTypePair & column); - - bool loaded_marks = false; - - size_t max_compress_block_size; - size_t file_count = 0; + std::atomic marks_loaded = false; + size_t num_marks_saved = 0; FileChecker file_checker; - /// Read marks files if they are not already read. - /// It is done lazily, so that with a large number of tables, the server starts quickly. - /// You can not call with a write locked `rwlock`. - void loadMarks(std::chrono::seconds lock_timeout); + const size_t max_compress_block_size; - /** For normal columns, the number of rows in the block is specified in the marks. - * For array columns and nested structures, there are more than one group of marks that correspond to different files - * - for elements (file name.bin) - the total number of array elements in the block is specified, - * - for array sizes (file name.size0.bin) - the number of rows (the whole arrays themselves) in the block is specified. - * - * Return the first group of marks that contain the number of rows, but not the internals of the arrays. - */ - const Marks & getMarksWithRealRowCount(const StorageMetadataPtr & metadata_snapshot) const; + mutable std::shared_timed_mutex rwlock; }; } diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp deleted file mode 100644 index 8657467226f..00000000000 --- a/src/Storages/StorageTinyLog.cpp +++ /dev/null @@ -1,594 +0,0 @@ -#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 "StorageLogSettings.h" - -#include -#include -#include - -#define DBMS_STORAGE_LOG_DATA_FILE_EXTENSION ".bin" - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int TIMEOUT_EXCEEDED; - extern const int DUPLICATE_COLUMN; - extern const int INCORRECT_FILE_NAME; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} - - -class TinyLogSource final : public SourceWithProgress -{ -public: - static Block getHeader(const NamesAndTypesList & columns) - { - Block res; - - for (const auto & name_type : columns) - res.insert({ name_type.type->createColumn(), name_type.type, name_type.name }); - - return res; - } - - TinyLogSource( - size_t block_size_, - const NamesAndTypesList & columns_, - StorageTinyLog & storage_, - ReadSettings read_settings_, - FileChecker::Map file_sizes_) - : SourceWithProgress(getHeader(columns_)) - , block_size(block_size_), columns(columns_), storage(storage_) - , read_settings(std::move(read_settings_)), file_sizes(std::move(file_sizes_)) - { - } - - String getName() const override { return "TinyLog"; } - -protected: - Chunk generate() override; - -private: - size_t block_size; - NamesAndTypesList columns; - StorageTinyLog & storage; - bool is_finished = false; - ReadSettings read_settings; - FileChecker::Map file_sizes; - - struct Stream - { - Stream(const DiskPtr & disk, const String & data_path, ReadSettings read_settings_, size_t file_size) - : plain(file_size - ? disk->readFile(data_path, read_settings_.adjustBufferSize(file_size)) - : std::make_unique(nullptr, 0)), - limited(std::make_unique(*plain, file_size, false)), - compressed(*limited) - { - } - - std::unique_ptr plain; - std::unique_ptr limited; - CompressedReadBuffer compressed; - }; - - using FileStreams = std::map>; - FileStreams streams; - - using DeserializeState = ISerialization::DeserializeBinaryBulkStatePtr; - using DeserializeStates = std::map; - DeserializeStates deserialize_states; - - void readData(const NameAndTypePair & name_and_type, ColumnPtr & column, UInt64 limit, ISerialization::SubstreamsCache & cache); -}; - - -Chunk TinyLogSource::generate() -{ - Block res; - - if (is_finished || file_sizes.empty() || (!streams.empty() && streams.begin()->second->compressed.eof())) - { - /** Close the files (before destroying the object). - * When many sources are created, but simultaneously reading only a few of them, - * buffers don't waste memory. - */ - is_finished = true; - streams.clear(); - return {}; - } - - std::unordered_map caches; - for (const auto & name_type : columns) - { - ColumnPtr column; - try - { - column = name_type.type->createColumn(); - readData(name_type, column, block_size, caches[name_type.getNameInStorage()]); - } - catch (Exception & e) - { - e.addMessage("while reading column " + name_type.name + " at " + fullPath(storage.disk, storage.table_path)); - throw; - } - - if (!column->empty()) - res.insert(ColumnWithTypeAndName(std::move(column), name_type.type, name_type.name)); - } - - if (!res || streams.begin()->second->compressed.eof()) - { - is_finished = true; - streams.clear(); - } - - return Chunk(res.getColumns(), res.rows()); -} - - -void TinyLogSource::readData(const NameAndTypePair & name_and_type, - ColumnPtr & column, UInt64 limit, ISerialization::SubstreamsCache & cache) -{ - ISerialization::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint. - const auto & [name, type] = name_and_type; - auto serialization = IDataType::getSerialization(name_and_type); - - settings.getter = [&] (const ISerialization::SubstreamPath & path) -> ReadBuffer * - { - if (cache.count(ISerialization::getSubcolumnNameForStream(path))) - return nullptr; - - String stream_name = ISerialization::getFileNameForStream(name_and_type, path); - auto & stream = streams[stream_name]; - if (!stream) - { - String file_path = storage.files[stream_name].data_file_path; - stream = std::make_unique( - storage.disk, file_path, read_settings, file_sizes[fileName(file_path)]); - } - - return &stream->compressed; - }; - - if (deserialize_states.count(name) == 0) - serialization->deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]); - - serialization->deserializeBinaryBulkWithMultipleStreams(column, limit, settings, deserialize_states[name], &cache); -} - - -class TinyLogSink final : public SinkToStorage -{ -public: - explicit TinyLogSink( - StorageTinyLog & storage_, - const StorageMetadataPtr & metadata_snapshot_, - std::unique_lock && lock_) - : SinkToStorage(metadata_snapshot_->getSampleBlock()) - , storage(storage_), metadata_snapshot(metadata_snapshot_), lock(std::move(lock_)) - { - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - - /// If there were no files, add info to rollback in case of error. - if (storage.file_checker.empty()) - { - for (const auto & file : storage.files) - storage.file_checker.setEmpty(file.second.data_file_path); - storage.file_checker.save(); - } - } - - ~TinyLogSink() override - { - try - { - if (!done) - { - /// Rollback partial writes. - LOG_WARNING(storage.log, "Rollback partial writes"); - streams.clear(); - storage.file_checker.repair(); - } - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - - String getName() const override { return "TinyLogSink"; } - - void consume(Chunk chunk) override; - void onFinish() override; - -private: - StorageTinyLog & storage; - StorageMetadataPtr metadata_snapshot; - std::unique_lock lock; - bool done = false; - - struct Stream - { - Stream(const DiskPtr & disk, const String & data_path, CompressionCodecPtr codec, size_t max_compress_block_size) : - plain(disk->writeFile(data_path, max_compress_block_size, WriteMode::Append)), - compressed(*plain, std::move(codec), max_compress_block_size) - { - } - - std::unique_ptr plain; - CompressedWriteBuffer compressed; - - void finalize() - { - compressed.next(); - plain->finalize(); - } - }; - - using FileStreams = std::map>; - FileStreams streams; - - using SerializeState = ISerialization::SerializeBinaryBulkStatePtr; - using SerializeStates = std::map; - SerializeStates serialize_states; - - using WrittenStreams = std::set; - - ISerialization::OutputStreamGetter createStreamGetter(const NameAndTypePair & column, WrittenStreams & written_streams); - void writeData(const NameAndTypePair & name_and_type, const IColumn & column, WrittenStreams & written_streams); -}; - - -ISerialization::OutputStreamGetter TinyLogSink::createStreamGetter( - const NameAndTypePair & column, - WrittenStreams & written_streams) -{ - return [&] (const ISerialization::SubstreamPath & path) -> WriteBuffer * - { - String stream_name = ISerialization::getFileNameForStream(column, path); - - if (!written_streams.insert(stream_name).second) - return nullptr; - - const auto & columns = metadata_snapshot->getColumns(); - if (!streams.count(stream_name)) - streams[stream_name] = std::make_unique( - storage.disk, - storage.files[stream_name].data_file_path, - columns.getCodecOrDefault(column.name), - storage.max_compress_block_size); - - return &streams[stream_name]->compressed; - }; -} - - -void TinyLogSink::writeData(const NameAndTypePair & name_and_type, const IColumn & column, WrittenStreams & written_streams) -{ - ISerialization::SerializeBinaryBulkSettings settings; - const auto & [name, type] = name_and_type; - auto serialization = type->getDefaultSerialization(); - - if (serialize_states.count(name) == 0) - { - /// Some stream getters may be called form `serializeBinaryBulkStatePrefix`. - /// Use different WrittenStreams set, or we get nullptr for them in `serializeBinaryBulkWithMultipleStreams` - WrittenStreams prefix_written_streams; - settings.getter = createStreamGetter(name_and_type, prefix_written_streams); - serialization->serializeBinaryBulkStatePrefix(settings, serialize_states[name]); - } - - settings.getter = createStreamGetter(name_and_type, written_streams); - serialization->serializeBinaryBulkWithMultipleStreams(column, 0, 0, settings, serialize_states[name]); -} - - -void TinyLogSink::onFinish() -{ - if (done) - return; - - /// If nothing was written - leave the table in initial state. - if (streams.empty()) - { - done = true; - return; - } - - WrittenStreams written_streams; - ISerialization::SerializeBinaryBulkSettings settings; - for (const auto & column : getHeader()) - { - auto it = serialize_states.find(column.name); - if (it != serialize_states.end()) - { - settings.getter = createStreamGetter(NameAndTypePair(column.name, column.type), written_streams); - auto serialization = column.type->getDefaultSerialization(); - serialization->serializeBinaryBulkStateSuffix(settings, it->second); - } - } - - /// Finish write. - for (auto & stream : streams) - stream.second->finalize(); - - Strings column_files; - for (auto & pair : streams) - column_files.push_back(storage.files[pair.first].data_file_path); - - streams.clear(); - done = true; - - for (const auto & file : column_files) - storage.file_checker.update(file); - storage.file_checker.save(); - - /// unlock should be done from the same thread as lock, and dtor may be - /// called from different thread, so it should be done here (at least in - /// case of no exceptions occurred) - lock.unlock(); -} - - -void TinyLogSink::consume(Chunk chunk) -{ - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); - metadata_snapshot->check(block, true); - - /// The set of written offset columns so that you do not write shared columns for nested structures multiple times - WrittenStreams written_streams; - - for (size_t i = 0; i < block.columns(); ++i) - { - const ColumnWithTypeAndName & column = block.safeGetByPosition(i); - writeData(NameAndTypePair(column.name, column.type), *column.column, written_streams); - } -} - - -StorageTinyLog::StorageTinyLog( - DiskPtr disk_, - const String & relative_path_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - bool attach, - size_t max_compress_block_size_) - : IStorage(table_id_) - , disk(std::move(disk_)) - , table_path(relative_path_) - , max_compress_block_size(max_compress_block_size_) - , file_checker(disk, table_path + "sizes.json") - , log(&Poco::Logger::get("StorageTinyLog")) -{ - StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(columns_); - storage_metadata.setConstraints(constraints_); - storage_metadata.setComment(comment); - setInMemoryMetadata(storage_metadata); - - if (relative_path_.empty()) - throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); - - if (!attach) - { - /// create directories if they do not exist - disk->createDirectories(table_path); - } - else - { - try - { - file_checker.repair(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - - for (const auto & col : storage_metadata.getColumns().getAllPhysical()) - addFiles(col); -} - - -void StorageTinyLog::addFiles(const NameAndTypePair & column) -{ - const auto & [name, type] = column; - if (files.end() != files.find(name)) - throw Exception("Duplicate column with name " + name + " in constructor of StorageTinyLog.", - ErrorCodes::DUPLICATE_COLUMN); - - ISerialization::StreamCallback stream_callback = [&] (const ISerialization::SubstreamPath & substream_path) - { - String stream_name = ISerialization::getFileNameForStream(column, substream_path); - if (!files.count(stream_name)) - { - ColumnData column_data; - files.insert(std::make_pair(stream_name, column_data)); - files[stream_name].data_file_path = table_path + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION; - } - }; - - auto serialization = type->getDefaultSerialization(); - serialization->enumerateStreams(stream_callback); -} - - -void StorageTinyLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id) -{ - assert(table_path != new_path_to_table_data); - { - disk->moveDirectory(table_path, new_path_to_table_data); - - table_path = new_path_to_table_data; - file_checker.setPath(table_path + "sizes.json"); - - for (auto & file : files) - file.second.data_file_path = table_path + fileName(file.second.data_file_path); - } - renameInMemory(new_table_id); -} - - -static std::chrono::seconds getLockTimeout(ContextPtr context) -{ - const Settings & settings = context->getSettingsRef(); - Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds(); - if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout) - lock_timeout = settings.max_execution_time.totalSeconds(); - return std::chrono::seconds{lock_timeout}; -} - - -Pipe StorageTinyLog::read( - const Names & column_names, - const StorageMetadataPtr & metadata_snapshot, - SelectQueryInfo & /*query_info*/, - ContextPtr context, - QueryProcessingStage::Enum /*processed_stage*/, - const size_t max_block_size, - const unsigned /*num_streams*/) -{ - metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); - - auto all_columns = metadata_snapshot->getColumns().getByNames(ColumnsDescription::All, column_names, true); - - // When reading, we lock the entire storage, because we only have one file - // per column and can't modify it concurrently. - std::shared_lock lock{rwlock, getLockTimeout(context)}; - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - - /// No need to hold lock while reading because we read fixed range of data that does not change while appending more data. - return Pipe(std::make_shared( - max_block_size, - Nested::convertToSubcolumns(all_columns), - *this, - context->getReadSettings(), - file_checker.getFileSizes())); -} - - -SinkToStoragePtr StorageTinyLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) -{ - return std::make_shared(*this, metadata_snapshot, std::unique_lock{rwlock, getLockTimeout(context)}); -} - - -CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, ContextPtr context) -{ - std::shared_lock lock(rwlock, getLockTimeout(context)); - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - - return file_checker.check(); -} - -IStorage::ColumnSizeByName StorageTinyLog::getColumnSizes() const -{ - std::shared_lock lock(rwlock, std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC)); - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - - ColumnSizeByName column_sizes; - FileChecker::Map file_sizes = file_checker.getFileSizes(); - - for (const auto & column : getInMemoryMetadata().getColumns().getAllPhysical()) - { - ISerialization::StreamCallback stream_callback = [&, this] (const ISerialization::SubstreamPath & substream_path) - { - String stream_name = ISerialization::getFileNameForStream(column, substream_path); - ColumnSize & size = column_sizes[column.name]; - auto it = files.find(stream_name); - if (it != files.end()) - size.data_compressed += file_sizes[fileName(it->second.data_file_path)]; - }; - - auto serialization = column.type->getDefaultSerialization(); - serialization->enumerateStreams(stream_callback); - } - - return column_sizes; -} - -void StorageTinyLog::truncate( - const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) -{ - disk->clearDirectory(table_path); - - files.clear(); - file_checker = FileChecker{disk, table_path + "sizes.json"}; - - for (const auto & column : metadata_snapshot->getColumns().getAllPhysical()) - addFiles(column); -} - - -void registerStorageTinyLog(StorageFactory & factory) -{ - StorageFactory::StorageFeatures features{ - .supports_settings = true - }; - - factory.registerStorage("TinyLog", [](const StorageFactory::Arguments & args) - { - if (!args.engine_args.empty()) - throw Exception( - "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - String disk_name = getDiskName(*args.storage_def); - DiskPtr disk = args.getContext()->getDisk(disk_name); - - return StorageTinyLog::create( - disk, - args.relative_data_path, - args.table_id, - args.columns, - args.constraints, - args.comment, - args.attach, - args.getContext()->getSettings().max_compress_block_size); - }, features); -} - -} diff --git a/src/Storages/StorageTinyLog.h b/src/Storages/StorageTinyLog.h deleted file mode 100644 index b25b79afe1f..00000000000 --- a/src/Storages/StorageTinyLog.h +++ /dev/null @@ -1,82 +0,0 @@ -#pragma once - -#include - -#include - -#include -#include -#include -#include - - -namespace DB -{ -/** Implements a table engine that is suitable for small chunks of the log. - * It differs from StorageLog in the absence of mark files. - */ -class StorageTinyLog final : public shared_ptr_helper, public IStorage -{ - friend class TinyLogSource; - friend class TinyLogSink; - friend struct shared_ptr_helper; - -public: - String getName() const override { return "TinyLog"; } - - Pipe read( - const Names & column_names, - const StorageMetadataPtr & /*metadata_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 rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; - - CheckResults checkData(const ASTPtr & /* query */, ContextPtr /* context */) override; - - bool storesDataOnDisk() const override { return true; } - Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; } - bool supportsSubcolumns() const override { return true; } - - void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder &) override; - - ColumnSizeByName getColumnSizes() const override; -protected: - StorageTinyLog( - DiskPtr disk_, - const String & relative_path_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - bool attach, - size_t max_compress_block_size_); - -private: - struct ColumnData - { - String data_file_path; - }; - using Files = std::map; /// file name -> column data - - DiskPtr disk; - String table_path; - - size_t max_compress_block_size; - - Files files; - - FileChecker file_checker; - mutable std::shared_timed_mutex rwlock; - - Poco::Logger * log; - - void addFiles(const NameAndTypePair & column); -}; - -} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index af2e4732803..e8cbd7a0c19 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -10,7 +10,6 @@ namespace DB { void registerStorageLog(StorageFactory & factory); -void registerStorageTinyLog(StorageFactory & factory); void registerStorageStripeLog(StorageFactory & factory); void registerStorageMergeTree(StorageFactory & factory); void registerStorageNull(StorageFactory & factory); @@ -78,7 +77,6 @@ void registerStorages() auto & factory = StorageFactory::instance(); registerStorageLog(factory); - registerStorageTinyLog(factory); registerStorageStripeLog(factory); registerStorageMergeTree(factory); registerStorageNull(factory); diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index f86295cd06b..a48b764b62c 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -35,7 +35,8 @@ DB::StoragePtr createStorage(DB::DiskPtr & disk) names_and_types.emplace_back("a", std::make_shared()); StoragePtr table = StorageLog::create( - disk, "table/", StorageID("test", "test"), ColumnsDescription{names_and_types}, ConstraintsDescription{}, String{}, false, 1048576); + "Log", disk, "table/", StorageID("test", "test"), ColumnsDescription{names_and_types}, + ConstraintsDescription{}, String{}, false, 1048576); table->startup(); From 0e8c9b089f2613dc7f3945b230f5d0dabca2b0ad Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 27 Aug 2021 01:15:24 +0300 Subject: [PATCH 864/919] Keep indices for StorageStripeLog in memory. --- src/Common/FileChecker.cpp | 5 - src/Common/FileChecker.h | 7 +- src/Formats/IndexForNativeFormat.cpp | 91 ++++++++++++ src/Formats/IndexForNativeFormat.h | 60 ++++++++ src/Formats/NativeReader.cpp | 35 ----- src/Formats/NativeReader.h | 43 +----- src/Formats/NativeWriter.cpp | 33 ++-- src/Formats/NativeWriter.h | 6 +- src/Storages/StorageStripeLog.cpp | 215 +++++++++++++++++++-------- src/Storages/StorageStripeLog.h | 39 ++++- 10 files changed, 359 insertions(+), 175 deletions(-) create mode 100644 src/Formats/IndexForNativeFormat.cpp create mode 100644 src/Formats/IndexForNativeFormat.h diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index 88dece64e5f..64c13ceb69c 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -50,11 +50,6 @@ void FileChecker::setEmpty(const String & full_file_path) map[fileName(full_file_path)] = 0; } -const FileChecker::Map & FileChecker::getFileSizes() const -{ - return map; -} - size_t FileChecker::getFileSize(const String & full_file_path) const { auto it = map.find(fileName(full_file_path)); diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index 60003e0cabc..325e9325267 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -28,11 +28,6 @@ public: /// The purpose of this function is to rollback a group of unfinished writes. void repair(); - /// File name -> size. - using Map = std::map; - - const Map & getFileSizes() const; - /// Returns stored file size. size_t getFileSize(const String & full_file_path) const; @@ -43,7 +38,7 @@ private: const Poco::Logger * log = &Poco::Logger::get("FileChecker"); String files_info_path; - Map map; + std::map map; }; } diff --git a/src/Formats/IndexForNativeFormat.cpp b/src/Formats/IndexForNativeFormat.cpp new file mode 100644 index 00000000000..a383fb7bd48 --- /dev/null +++ b/src/Formats/IndexForNativeFormat.cpp @@ -0,0 +1,91 @@ +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_INDEX; +} + +void IndexOfBlockForNativeFormat::read(ReadBuffer & istr) +{ + readVarUInt(num_columns, istr); + readVarUInt(num_rows, istr); + columns.clear(); + for (size_t i = 0; i < num_columns; ++i) + { + auto & column = columns.emplace_back(); + readBinary(column.name, istr); + readBinary(column.type, istr); + readBinary(column.location.offset_in_compressed_file, istr); + readBinary(column.location.offset_in_decompressed_block, istr); + } +} + +void IndexOfBlockForNativeFormat::write(WriteBuffer & ostr) const +{ + writeVarUInt(num_columns, ostr); + writeVarUInt(num_rows, ostr); + for (size_t i = 0; i < num_columns; ++i) + { + const auto & column = columns[i]; + writeBinary(column.name, ostr); + writeBinary(column.type, ostr); + writeBinary(column.location.offset_in_compressed_file, ostr); + writeBinary(column.location.offset_in_decompressed_block, ostr); + } +} + +IndexOfBlockForNativeFormat IndexOfBlockForNativeFormat::extractIndexForColumns(const NameSet & required_columns) const +{ + if (num_columns < required_columns.size()) + throw Exception("Index contain less than required columns", ErrorCodes::INCORRECT_INDEX); + + IndexOfBlockForNativeFormat res; + for (size_t i = 0; i < num_columns; ++i) + { + const auto & column = columns[i]; + if (required_columns.contains(column.name)) + res.columns.push_back(column); + } + + if (res.columns.size() < required_columns.size()) + throw Exception("Index contain less than required columns", ErrorCodes::INCORRECT_INDEX); + if (res.columns.size() > required_columns.size()) + throw Exception("Index contain duplicate columns", ErrorCodes::INCORRECT_INDEX); + + res.num_columns = res.columns.size(); + res.num_rows = num_rows; + return res; +} + + +void IndexForNativeFormat::read(ReadBuffer & istr) +{ + blocks.clear(); + while (!istr.eof()) + { + auto & block = blocks.emplace_back(); + block.read(istr); + } +} + +void IndexForNativeFormat::write(WriteBuffer & ostr) const +{ + for (const auto & block : blocks) + block.write(ostr); +} + +IndexForNativeFormat IndexForNativeFormat::extractIndexForColumns(const NameSet & required_columns) const +{ + IndexForNativeFormat res; + res.blocks.reserve(blocks.size()); + for (const auto & block : blocks) + res.blocks.emplace_back(block.extractIndexForColumns(required_columns)); + return res; +} + +} diff --git a/src/Formats/IndexForNativeFormat.h b/src/Formats/IndexForNativeFormat.h new file mode 100644 index 00000000000..646f539ebd0 --- /dev/null +++ b/src/Formats/IndexForNativeFormat.h @@ -0,0 +1,60 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/** The Native format can contain a separately located index, + * which allows you to understand where what column is located, + * and skip unnecessary columns. + */ + +/** The position of one piece of a single column. */ +struct IndexOfOneColumnForNativeFormat +{ + String name; + String type; + MarkInCompressedFile location; +}; + +/** The index for the data block. */ +struct IndexOfBlockForNativeFormat +{ + using Columns = std::vector; + + size_t num_columns; + size_t num_rows; + Columns columns; + + /// Reads the index for the data block. + void read(ReadBuffer & istr); + + /// Writes the index for the data block. + void write(WriteBuffer & ostr) const; + + /// Returns the index only for the required columns. + IndexOfBlockForNativeFormat extractIndexForColumns(const NameSet & required_columns) const; +}; + +/** The whole index. */ +struct IndexForNativeFormat +{ + using Blocks = std::vector; + Blocks blocks; + + bool empty() const { return blocks.empty(); } + void clear() { blocks.clear(); } + + /// Reads the index. + void read(ReadBuffer & istr); + + /// Writes the index. + void write(WriteBuffer & ostr) const; + + /// Returns the index only for the required columns. + IndexForNativeFormat extractIndexForColumns(const NameSet & required_columns) const; +}; + +} diff --git a/src/Formats/NativeReader.cpp b/src/Formats/NativeReader.cpp index 9ef248dc904..e817d23e03d 100644 --- a/src/Formats/NativeReader.cpp +++ b/src/Formats/NativeReader.cpp @@ -221,39 +221,4 @@ void NativeReader::updateAvgValueSizeHints(const Block & block) } } -void IndexForNativeFormat::read(ReadBuffer & istr, const NameSet & required_columns) -{ - while (!istr.eof()) - { - blocks.emplace_back(); - IndexOfBlockForNativeFormat & block = blocks.back(); - - readVarUInt(block.num_columns, istr); - readVarUInt(block.num_rows, istr); - - if (block.num_columns < required_columns.size()) - throw Exception("Index contain less than required columns", ErrorCodes::INCORRECT_INDEX); - - for (size_t i = 0; i < block.num_columns; ++i) - { - IndexOfOneColumnForNativeFormat column_index; - - readBinary(column_index.name, istr); - readBinary(column_index.type, istr); - readBinary(column_index.location.offset_in_compressed_file, istr); - readBinary(column_index.location.offset_in_decompressed_block, istr); - - if (required_columns.count(column_index.name)) - block.columns.push_back(std::move(column_index)); - } - - if (block.columns.size() < required_columns.size()) - throw Exception("Index contain less than required columns", ErrorCodes::INCORRECT_INDEX); - if (block.columns.size() > required_columns.size()) - throw Exception("Index contain duplicate columns", ErrorCodes::INCORRECT_INDEX); - - block.num_columns = block.columns.size(); - } -} - } diff --git a/src/Formats/NativeReader.h b/src/Formats/NativeReader.h index 49c2db7703f..dfc7a749187 100644 --- a/src/Formats/NativeReader.h +++ b/src/Formats/NativeReader.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -9,48 +10,6 @@ namespace DB class CompressedReadBufferFromFile; - -/** The Native format can contain a separately located index, - * which allows you to understand where what column is located, - * and skip unnecessary columns. - */ - -/** The position of one piece of a single column. */ -struct IndexOfOneColumnForNativeFormat -{ - String name; - String type; - MarkInCompressedFile location; -}; - -/** The index for the data block. */ -struct IndexOfBlockForNativeFormat -{ - using Columns = std::vector; - - size_t num_columns; - size_t num_rows; - Columns columns; -}; - -/** The whole index. */ -struct IndexForNativeFormat -{ - using Blocks = std::vector; - Blocks blocks; - - IndexForNativeFormat() {} - - IndexForNativeFormat(ReadBuffer & istr, const NameSet & required_columns) - { - read(istr, required_columns); - } - - /// Read the index, only for the required columns. - void read(ReadBuffer & istr, const NameSet & required_columns); -}; - - /** Deserializes the stream of blocks from the native binary format (with names and column types). * Designed for communication between servers. * diff --git a/src/Formats/NativeWriter.cpp b/src/Formats/NativeWriter.cpp index 9da0c312362..41a9e5dc414 100644 --- a/src/Formats/NativeWriter.cpp +++ b/src/Formats/NativeWriter.cpp @@ -5,6 +5,7 @@ #include #include +#include #include #include @@ -22,11 +23,11 @@ namespace ErrorCodes NativeWriter::NativeWriter( WriteBuffer & ostr_, UInt64 client_revision_, const Block & header_, bool remove_low_cardinality_, - WriteBuffer * index_ostr_, size_t initial_size_of_file_) + IndexForNativeFormat * index_, size_t initial_size_of_file_) : ostr(ostr_), client_revision(client_revision_), header(header_), - index_ostr(index_ostr_), initial_size_of_file(initial_size_of_file_), remove_low_cardinality(remove_low_cardinality_) + index(index_), initial_size_of_file(initial_size_of_file_), remove_low_cardinality(remove_low_cardinality_) { - if (index_ostr) + if (index) { ostr_concrete = typeid_cast(&ostr); if (!ostr_concrete) @@ -80,18 +81,20 @@ void NativeWriter::write(const Block & block) /** The index has the same structure as the data stream. * But instead of column values, it contains a mark that points to the location in the data file where this part of the column is located. */ - if (index_ostr) + IndexOfBlockForNativeFormat index_block; + if (index) { - writeVarUInt(columns, *index_ostr); - writeVarUInt(rows, *index_ostr); + index_block.num_columns = columns; + index_block.num_rows = rows; + index_block.columns.resize(columns); } for (size_t i = 0; i < columns; ++i) { /// For the index. - MarkInCompressedFile mark; + MarkInCompressedFile mark{0, 0}; - if (index_ostr) + if (index) { ostr_concrete->next(); /// Finish compressed block. mark.offset_in_compressed_file = initial_size_of_file + ostr_concrete->getCompressedBytes(); @@ -125,15 +128,17 @@ void NativeWriter::write(const Block & block) if (rows) /// Zero items of data is always represented as zero number of bytes. writeData(*column.type, column.column, ostr, 0, 0); - if (index_ostr) + if (index) { - writeStringBinary(column.name, *index_ostr); - writeStringBinary(column.type->getName(), *index_ostr); - - writeBinary(mark.offset_in_compressed_file, *index_ostr); - writeBinary(mark.offset_in_decompressed_block, *index_ostr); + index_block.columns[i].name = column.name; + index_block.columns[i].type = column.type->getName(); + index_block.columns[i].location.offset_in_compressed_file = mark.offset_in_compressed_file; + index_block.columns[i].location.offset_in_decompressed_block = mark.offset_in_decompressed_block; } } + + if (index) + index->blocks.emplace_back(std::move(index_block)); } } diff --git a/src/Formats/NativeWriter.h b/src/Formats/NativeWriter.h index 67fc179b620..010a03ec722 100644 --- a/src/Formats/NativeWriter.h +++ b/src/Formats/NativeWriter.h @@ -9,7 +9,7 @@ namespace DB class WriteBuffer; class CompressedWriteBuffer; - +struct IndexForNativeFormat; /** Serializes the stream of blocks in their native binary format (with names and column types). * Designed for communication between servers. @@ -24,7 +24,7 @@ public: */ NativeWriter( WriteBuffer & ostr_, UInt64 client_revision_, const Block & header_, bool remove_low_cardinality_ = false, - WriteBuffer * index_ostr_ = nullptr, size_t initial_size_of_file_ = 0); + IndexForNativeFormat * index_ = nullptr, size_t initial_size_of_file_ = 0); Block getHeader() const { return header; } void write(const Block & block); @@ -36,7 +36,7 @@ private: WriteBuffer & ostr; UInt64 client_revision; Block header; - WriteBuffer * index_ostr; + IndexForNativeFormat * index = nullptr; size_t initial_size_of_file; /// The initial size of the data file, if `append` done. Used for the index. /// If you need to write index, then `ostr` must be a CompressedWriteBuffer. CompressedWriteBuffer * ostr_concrete = nullptr; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 50113c391cc..7229442d5ef 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -47,11 +47,13 @@ namespace ErrorCodes } +/// NOTE: The lock `StorageStripeLog::rwlock` is NOT kept locked while reading, +/// because we read ranges of data that do not change. class StripeLogSource final : public SourceWithProgress { public: static Block getHeader( - StorageStripeLog & storage, + const StorageStripeLog & storage, const StorageMetadataPtr & metadata_snapshot, const Names & column_names, IndexForNativeFormat::Blocks::const_iterator index_begin, @@ -74,19 +76,18 @@ public: } StripeLogSource( - StorageStripeLog & storage_, + const StorageStripeLog & storage_, const StorageMetadataPtr & metadata_snapshot_, const Names & column_names, ReadSettings read_settings_, - std::shared_ptr & index_, + std::shared_ptr indices_, IndexForNativeFormat::Blocks::const_iterator index_begin_, IndexForNativeFormat::Blocks::const_iterator index_end_) - : SourceWithProgress( - getHeader(storage_, metadata_snapshot_, column_names, index_begin_, index_end_)) + : SourceWithProgress(getHeader(storage_, metadata_snapshot_, column_names, index_begin_, index_end_)) , storage(storage_) , metadata_snapshot(metadata_snapshot_) , read_settings(std::move(read_settings_)) - , index(index_) + , indices(indices_) , index_begin(index_begin_) , index_end(index_end_) { @@ -109,7 +110,7 @@ protected: { block_in.reset(); data_in.reset(); - index.reset(); + indices.reset(); } } @@ -117,13 +118,14 @@ protected: } private: - StorageStripeLog & storage; + const StorageStripeLog & storage; StorageMetadataPtr metadata_snapshot; ReadSettings read_settings; - std::shared_ptr index; + std::shared_ptr indices; IndexForNativeFormat::Blocks::const_iterator index_begin; IndexForNativeFormat::Blocks::const_iterator index_end; + Block header; /** optional - to create objects only on first reading @@ -141,40 +143,45 @@ private: started = true; String data_file_path = storage.table_path + "data.bin"; - data_in.emplace(storage.disk->readFile(data_file_path, read_settings.adjustBufferSize(storage.disk->getFileSize(data_file_path)))); + + /// We cannot just use `storage.file_checker` to get the size of the file here, + /// because `storage.rwlock` is not locked at this point. + size_t data_file_size = storage.disk->getFileSize(data_file_path); + + data_in.emplace(storage.disk->readFile(data_file_path, read_settings.adjustBufferSize(data_file_size))); block_in.emplace(*data_in, 0, index_begin, index_end); } } }; +/// NOTE: The lock `StorageStripeLog::rwlock` is kept locked in exclusive mode while writing. class StripeLogSink final : public SinkToStorage { public: + using WriteLock = std::unique_lock; + explicit StripeLogSink( - StorageStripeLog & storage_, const StorageMetadataPtr & metadata_snapshot_, std::unique_lock && lock_) + StorageStripeLog & storage_, const StorageMetadataPtr & metadata_snapshot_, WriteLock && lock_) : SinkToStorage(metadata_snapshot_->getSampleBlock()) , storage(storage_) , metadata_snapshot(metadata_snapshot_) , lock(std::move(lock_)) - , data_out_file(storage.table_path + "data.bin") - , data_out_compressed(storage.disk->writeFile(data_out_file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append)) + , data_out_compressed(storage.disk->writeFile(storage.data_file_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append)) , data_out(std::make_unique( - *data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), storage.max_compress_block_size)) - , index_out_file(storage.table_path + "index.mrk") - , index_out_compressed(storage.disk->writeFile(index_out_file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append)) - , index_out(std::make_unique(*index_out_compressed)) - , block_out(*data_out, 0, metadata_snapshot->getSampleBlock(), false, index_out.get(), storage.disk->getFileSize(data_out_file)) + *data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), storage.max_compress_block_size)) { if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - if (storage.file_checker.empty()) - { - storage.file_checker.setEmpty(storage.table_path + "data.bin"); - storage.file_checker.setEmpty(storage.table_path + "index.mrk"); - storage.file_checker.save(); - } + /// Ensure that indices are loaded because we're going to update them. + storage.loadIndices(lock); + + /// If there were no files, save zero file sizes to be able to rollback in case of error. + storage.saveFileSizes(lock); + + size_t initial_data_size = storage.file_checker.getFileSize(storage.data_file_path); + block_out = std::make_unique(*data_out, 0, metadata_snapshot->getSampleBlock(), false, &storage.indices, initial_data_size); } String getName() const override { return "StripeLogSink"; } @@ -186,12 +193,16 @@ public: if (!done) { /// Rollback partial writes. + + /// No more writing. data_out.reset(); data_out_compressed.reset(); - index_out.reset(); - index_out_compressed.reset(); + /// Truncate files to the older sizes. storage.file_checker.repair(); + + /// Remove excessive indices. + storage.removeUnsavedIndices(lock); } } catch (...) @@ -202,7 +213,7 @@ public: void consume(Chunk chunk) override { - block_out.write(getHeader().cloneWithColumns(chunk.detachColumns())); + block_out->write(getHeader().cloneWithColumns(chunk.detachColumns())); } void onFinish() override @@ -213,13 +224,12 @@ public: data_out->next(); data_out_compressed->next(); data_out_compressed->finalize(); - index_out->next(); - index_out_compressed->next(); - index_out_compressed->finalize(); - storage.file_checker.update(data_out_file); - storage.file_checker.update(index_out_file); - storage.file_checker.save(); + /// Save the new indices. + storage.saveIndices(lock); + + /// Save the new file sizes. + storage.saveFileSizes(lock); done = true; @@ -232,15 +242,11 @@ public: private: StorageStripeLog & storage; StorageMetadataPtr metadata_snapshot; - std::unique_lock lock; + WriteLock lock; - String data_out_file; std::unique_ptr data_out_compressed; std::unique_ptr data_out; - String index_out_file; - std::unique_ptr index_out_compressed; - std::unique_ptr index_out; - NativeWriter block_out; + std::unique_ptr block_out; bool done = false; }; @@ -258,8 +264,10 @@ StorageStripeLog::StorageStripeLog( : IStorage(table_id_) , disk(std::move(disk_)) , table_path(relative_path_) - , max_compress_block_size(max_compress_block_size_) + , data_file_path(table_path + "data.bin") + , index_file_path(table_path + "index.mrk") , file_checker(disk, table_path + "sizes.json") + , max_compress_block_size(max_compress_block_size_) , log(&Poco::Logger::get("StorageStripeLog")) { StorageInMemoryMetadata storage_metadata; @@ -271,6 +279,13 @@ StorageStripeLog::StorageStripeLog( if (relative_path_.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); + /// Ensure the file checker is initialized. + if (file_checker.empty()) + { + file_checker.setEmpty(data_file_path); + file_checker.setEmpty(index_file_path); + } + if (!attach) { /// create directories if they do not exist @@ -290,6 +305,9 @@ StorageStripeLog::StorageStripeLog( } +StorageStripeLog::~StorageStripeLog() = default; + + void StorageStripeLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { assert(table_path != new_path_to_table_data); @@ -297,6 +315,8 @@ void StorageStripeLog::rename(const String & new_path_to_table_data, const Stora disk->moveDirectory(table_path, new_path_to_table_data); table_path = new_path_to_table_data; + data_file_path = table_path + "data.bin"; + index_file_path = table_path + "index.mrk"; file_checker.setPath(table_path + "sizes.json"); } renameInMemory(new_table_id); @@ -322,41 +342,38 @@ Pipe StorageStripeLog::read( const size_t /*max_block_size*/, unsigned num_streams) { - std::shared_lock lock(rwlock, getLockTimeout(context)); + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); + + auto lock_timeout = getLockTimeout(context); + loadIndices(lock_timeout); + + ReadLock lock{rwlock, lock_timeout}; if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); - - NameSet column_names_set(column_names.begin(), column_names.end()); - - Pipes pipes; - - String index_file = table_path + "index.mrk"; - if (file_checker.empty() || !disk->exists(index_file)) - { + if (!file_checker.getFileSize(data_file_path)) return Pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); - } - ReadSettings read_settings = context->getReadSettings(); + auto indices_for_selected_columns + = std::make_shared(indices.extractIndexForColumns(NameSet{column_names.begin(), column_names.end()})); - CompressedReadBufferFromFile index_in(disk->readFile(index_file, read_settings.adjustBufferSize(4096))); - std::shared_ptr index{std::make_shared(index_in, column_names_set)}; - - size_t size = index->blocks.size(); + size_t size = indices_for_selected_columns->blocks.size(); if (num_streams > size) num_streams = size; + ReadSettings read_settings = context->getReadSettings(); + Pipes pipes; + for (size_t stream = 0; stream < num_streams; ++stream) { - IndexForNativeFormat::Blocks::const_iterator begin = index->blocks.begin(); - IndexForNativeFormat::Blocks::const_iterator end = index->blocks.begin(); + IndexForNativeFormat::Blocks::const_iterator begin = indices_for_selected_columns->blocks.begin(); + IndexForNativeFormat::Blocks::const_iterator end = indices_for_selected_columns->blocks.begin(); std::advance(begin, stream * size / num_streams); std::advance(end, (stream + 1) * size / num_streams); pipes.emplace_back(std::make_shared( - *this, metadata_snapshot, column_names, read_settings, index, begin, end)); + *this, metadata_snapshot, column_names, read_settings, indices_for_selected_columns, begin, end)); } /// We do not keep read lock directly at the time of reading, because we read ranges of data that do not change. @@ -367,7 +384,7 @@ Pipe StorageStripeLog::read( SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) { - std::unique_lock lock(rwlock, getLockTimeout(context)); + WriteLock lock{rwlock, getLockTimeout(context)}; if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); @@ -377,17 +394,91 @@ SinkToStoragePtr StorageStripeLog::write(const ASTPtr & /*query*/, const Storage CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, ContextPtr context) { - std::shared_lock lock(rwlock, getLockTimeout(context)); + ReadLock lock{rwlock, getLockTimeout(context)}; if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); return file_checker.check(); } + void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) { disk->clearDirectory(table_path); - file_checker = FileChecker{disk, table_path + "sizes.json"}; + + indices.clear(); + file_checker.setEmpty(data_file_path); + file_checker.setEmpty(index_file_path); + + indices_loaded = true; + num_indices_saved = 0; +} + + +void StorageStripeLog::loadIndices(std::chrono::seconds lock_timeout) +{ + if (indices_loaded) + return; + + /// We load indices with an exclusive lock (i.e. the write lock) because we don't want + /// a data race between two threads trying to load indices simultaneously. + WriteLock lock{rwlock, lock_timeout}; + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + + loadIndices(lock); +} + + +void StorageStripeLog::loadIndices(const WriteLock & /* already locked exclusively */) +{ + if (indices_loaded) + return; + + if (disk->exists(index_file_path)) + { + CompressedReadBufferFromFile index_in(disk->readFile(index_file_path, ReadSettings{}.adjustBufferSize(4096))); + indices.read(index_in); + } + + indices_loaded = true; + num_indices_saved = indices.blocks.size(); +} + + +void StorageStripeLog::saveIndices(const WriteLock & /* already locked for writing */) +{ + size_t num_indices = indices.blocks.size(); + if (num_indices_saved == num_indices) + return; + + size_t start = num_indices_saved; + auto index_out_compressed = disk->writeFile(index_file_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append); + auto index_out = std::make_unique(*index_out_compressed); + + for (size_t i = start; i != num_indices; ++i) + indices.blocks[i].write(*index_out); + + index_out->next(); + index_out_compressed->next(); + index_out_compressed->finalize(); + + num_indices_saved = num_indices; +} + + +void StorageStripeLog::removeUnsavedIndices(const WriteLock & /* already locked for writing */) +{ + if (indices.blocks.size() > num_indices_saved) + indices.blocks.resize(num_indices_saved); +} + + +void StorageStripeLog::saveFileSizes(const WriteLock & /* already locked for writing */) +{ + file_checker.update(data_file_path); + file_checker.update(index_file_path); + file_checker.save(); } diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index 6ca9552cd11..7ab40f867dd 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -7,12 +7,15 @@ #include #include +#include #include #include namespace DB { +struct IndexForNativeFormat; + /** Implements a table engine that is suitable for small chunks of the log. * In doing so, stores all the columns in a single Native file, with a nearby index. */ @@ -23,6 +26,8 @@ class StorageStripeLog final : public shared_ptr_helper, publi friend struct shared_ptr_helper; public: + ~StorageStripeLog() override; + String getName() const override { return "StripeLog"; } Pipe read( @@ -57,18 +62,36 @@ protected: size_t max_compress_block_size_); private: - struct ColumnData - { - String data_file_path; - }; - using Files = std::map; /// file name -> column data + using ReadLock = std::shared_lock; + using WriteLock = std::unique_lock; - DiskPtr disk; + /// Reads the index file if it hasn't read yet. + /// It is done lazily, so that with a large number of tables, the server starts quickly. + void loadIndices(std::chrono::seconds lock_timeout); + void loadIndices(const WriteLock &); + + /// Saves the index file. + void saveIndices(const WriteLock &); + + /// Removes all unsaved indices. + void removeUnsavedIndices(const WriteLock &); + + /// Saves the sizes of the data and index files. + void saveFileSizes(const WriteLock &); + + const DiskPtr disk; String table_path; - size_t max_compress_block_size; - + String data_file_path; + String index_file_path; FileChecker file_checker; + + IndexForNativeFormat indices; + std::atomic indices_loaded = false; + size_t num_indices_saved = 0; + + const size_t max_compress_block_size; + std::shared_timed_mutex rwlock; Poco::Logger * log; From b3a8ad124fb8adc444e1a8d9de4031db3f7cf517 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 31 Oct 2021 02:03:58 +0800 Subject: [PATCH 865/919] Fix index analysis for set index --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 4 ++-- .../0_stateless/02112_skip_index_set_and_or.reference | 0 tests/queries/0_stateless/02112_skip_index_set_and_or.sql | 6 ++++++ 3 files changed, 8 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02112_skip_index_set_and_or.reference create mode 100644 tests/queries/0_stateless/02112_skip_index_set_and_or.sql diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 60b9ddae329..a08ae4499f2 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -451,9 +451,9 @@ bool MergeTreeIndexConditionSet::checkASTUseless(const ASTPtr & node, bool atomi const ASTs & args = func->arguments->children; if (func->name == "and" || func->name == "indexHint") - return checkASTUseless(args[0], atomic) && checkASTUseless(args[1], atomic); + return std::all_of(args.begin(), args.end(), [this, atomic](const auto & arg) { return checkASTUseless(arg, atomic); }); else if (func->name == "or") - return checkASTUseless(args[0], atomic) || checkASTUseless(args[1], atomic); + return std::any_of(args.begin(), args.end(), [this, atomic](const auto & arg) { return checkASTUseless(arg, atomic); }); else if (func->name == "not") return checkASTUseless(args[0], atomic); else diff --git a/tests/queries/0_stateless/02112_skip_index_set_and_or.reference b/tests/queries/0_stateless/02112_skip_index_set_and_or.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02112_skip_index_set_and_or.sql b/tests/queries/0_stateless/02112_skip_index_set_and_or.sql new file mode 100644 index 00000000000..7b52e5de9c9 --- /dev/null +++ b/tests/queries/0_stateless/02112_skip_index_set_and_or.sql @@ -0,0 +1,6 @@ +drop table if exists set_index; + +create table set_index (a Int32, b Int32, INDEX b_set b type set(0) granularity 1) engine MergeTree order by tuple(); +insert into set_index values (1, 2); + +select b from set_index where a = 1 and a = 1 and b = 1 settings force_data_skipping_indices = 'b_set', optimize_move_to_prewhere=0; From e9c606840f7525ef9952eeed992f4646c5415231 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Sun, 31 Oct 2021 10:08:20 +0300 Subject: [PATCH 866/919] Better handling of `xtables.lock` in `runner`. --- tests/integration/runner | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/runner b/tests/integration/runner index 86a254b26f5..03ea091d750 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -275,8 +275,9 @@ if __name__ == "__main__": --volume={library_bridge_bin}:/clickhouse-library-bridge --volume={bin}:/clickhouse \ --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ - --volume=/run/xtables.lock:/run/xtables.lock:ro \ + --volume=/run:/run/host:ro \ {dockerd_internal_volume} -e DOCKER_CLIENT_TIMEOUT=300 -e COMPOSE_HTTP_TIMEOUT=600 \ + -e XTABLES_LOCKFILE=/run/host/xtables.lock \ {env_tags} {env_cleanup} -e PYTEST_OPTS='{parallel} {opts} {tests_list} -vvv' {img} {command}".format( net=net, tty=tty, From 00718fed2f742434b44cb5f99abc6ae31c636119 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 31 Oct 2021 10:46:56 +0300 Subject: [PATCH 867/919] Fix release job --- .github/workflows/release.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index a7d3151d41c..a396e188327 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -1,6 +1,6 @@ name: ReleaseChecks concurrency: - group: docs-release + group: master-release cancel-in-progress: true on: # yamllint disable-line rule:truthy push: @@ -8,7 +8,6 @@ on: # yamllint disable-line rule:truthy - master jobs: DockerHubPush: - needs: CheckLabels runs-on: [self-hosted, style-checker] steps: - name: Check out repository code @@ -23,6 +22,7 @@ jobs: name: changed_images path: ${{ runner.temp }}/docker_images_check/changed_images.json DocsRelease: + needs: DockerHubPush runs: [self-hosted, func-tester] steps: - name: Check out repository code From 48ca14b895c11eab17c6569527a2e0b8f2a5d57c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 31 Oct 2021 12:33:42 +0300 Subject: [PATCH 868/919] Update Install.cpp --- programs/install/Install.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 0ae91f45697..790c19dfe04 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -809,8 +809,9 @@ int mainEntryClickHouseInstall(int argc, char ** argv) if (has_password_for_default_user) maybe_password = " --password"; - fs::path pid_file = prefix / options["pid-path"].as() / "clickhouse-server.pid"; - if (fs::exists(pid_file)) { + fs::path pid_file = pid_path / "clickhouse-server.pid"; + if (fs::exists(pid_file)) + { fmt::print( "\nClickHouse has been successfully installed.\n" "\nRestart clickhouse-server with:\n" From 4801ab58aa29d0eb5e40ec5a19fa414cc408c4ea Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 31 Oct 2021 13:41:06 +0300 Subject: [PATCH 869/919] Update usability.md --- benchmark/timescaledb/usability.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/benchmark/timescaledb/usability.md b/benchmark/timescaledb/usability.md index 207a1dfa2b6..6914dc24118 100644 --- a/benchmark/timescaledb/usability.md +++ b/benchmark/timescaledb/usability.md @@ -244,8 +244,7 @@ sudo mcedit /etc/postgresql/13/main/postgresql.conf #listen_addresses = 'localhost' ``` -Looks like I need to uncomment it.ERROR: cannot change configuration on already compressed chunks -DETAIL: There are compressed chunks that prevent changing the existing compression configuration. +Looks like I need to uncomment it. ``` sudo service postgresql restart From 0ff8ab70f6ad30497ee1f537d25db35ee17e3862 Mon Sep 17 00:00:00 2001 From: Aleksandr Shalimov Date: Sun, 31 Oct 2021 19:02:50 +0800 Subject: [PATCH 870/919] Fix spelling --- docs/ru/sql-reference/statements/select/prewhere.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/select/prewhere.md b/docs/ru/sql-reference/statements/select/prewhere.md index 4376cbeb295..4a04a56b47d 100644 --- a/docs/ru/sql-reference/statements/select/prewhere.md +++ b/docs/ru/sql-reference/statements/select/prewhere.md @@ -6,7 +6,7 @@ toc_title: PREWHERE Prewhere — это оптимизация для более эффективного применения фильтрации. Она включена по умолчанию, даже если секция `PREWHERE` явно не указана. В этом случае работает автоматическое перемещение части выражения из [WHERE](where.md) до стадии prewhere. Роль секции `PREWHERE` только для управления этой оптимизацией, если вы думаете, что знаете, как сделать перемещение условия лучше, чем это происходит по умолчанию. -При оптимизации prewhere сначала читываются только те столбцы, которые необходимы для выполнения выражения prewhere. Затем читаются другие столбцы, необходимые для выполнения остальной части запроса, но только те блоки, в которых находится выражение prewhere «верно» по крайней мере для некоторых рядов. Если есть много блоков, где выражение prewhere «ложно» для всех строк и для выражения prewhere требуется меньше столбцов, чем для других частей запроса, это часто позволяет считывать гораздо меньше данных с диска для выполнения запроса. +При оптимизации prewhere сначала читаются только те столбцы, которые необходимы для выполнения выражения prewhere. Затем читаются другие столбцы, необходимые для выполнения остальной части запроса, но только те блоки, в которых находится выражение prewhere «верно» по крайней мере для некоторых рядов. Если есть много блоков, где выражение prewhere «ложно» для всех строк и для выражения prewhere требуется меньше столбцов, чем для других частей запроса, это часто позволяет считывать гораздо меньше данных с диска для выполнения запроса. ## Управление PREWHERE вручную {#controlling-prewhere-manually} From 96ec829d87c6ddc4cb2b05efd79c413b7540b6ea Mon Sep 17 00:00:00 2001 From: laurieliyang <11391675+laurieliyang@users.noreply.github.com> Date: Sun, 31 Oct 2021 20:02:30 +0800 Subject: [PATCH 871/919] Update permissions-for-queries.md of cn Update permissions-for-queries.md of Chinese --- .../settings/permissions-for-queries.md | 41 +++++++++---------- 1 file changed, 20 insertions(+), 21 deletions(-) diff --git a/docs/zh/operations/settings/permissions-for-queries.md b/docs/zh/operations/settings/permissions-for-queries.md index a72500b76d5..93e439ae206 100644 --- a/docs/zh/operations/settings/permissions-for-queries.md +++ b/docs/zh/operations/settings/permissions-for-queries.md @@ -9,52 +9,51 @@ toc_title: "\u67E5\u8BE2\u6743\u9650" ClickHouse中的查询可以分为几种类型: -1. 读取数据查询: `SELECT`, `SHOW`, `DESCRIBE`, `EXISTS`. -2. 写入数据查询: `INSERT`, `OPTIMIZE`. -3. 更改设置查询: `SET`, `USE`. -4. [DDL](https://en.wikipedia.org/wiki/Data_definition_language) 查询: `CREATE`, `ALTER`, `RENAME`, `ATTACH`, `DETACH`, `DROP` `TRUNCATE`. -5. `KILL QUERY`. +1. 读取数据的查询: `SELECT` , `SHOW` , `DESCRIBE` , `EXISTS` 。 +2. 写入数据的查询: `INSERT` , `OPTIMIZE` 。 +3. 更改设置的查询: `SET` , `USE` 。 +4. [DDL](https://zh.wikipedia.org/zh-cn/数据定义语言) 查询: `CREATE` , `ALTER` , `RENAME` , `ATTACH` , `DETACH` , `DROP` , `TRUNCATE` 。 +5. `KILL QUERY` 。 以下设置按查询类型规范用户权限: -- [只读](#settings_readonly) — Restricts permissions for all types of queries except DDL queries. -- [allow_ddl](#settings_allow_ddl) — Restricts permissions for DDL queries. +- [readonly](#settings_readonly) — 对除 DDL 查询以外的所有类型限制权限。 +- [allow_ddl](#settings_allow_ddl) — 对 DDL 查询限制权限。 `KILL QUERY` 可以与任何设置进行。 -## 只读 {#settings_readonly} +## readonly {#settings_readonly} 限制读取数据、写入数据和更改设置查询的权限。 -查看查询如何划分为多种类型 [以上](#permissions_for_queries). +查看查询如何划分为 **[上述](#permissions_for_queries)** 的多种类型。 可能的值: -- 0 — All queries are allowed. -- 1 — Only read data queries are allowed. -- 2 — Read data and change settings queries are allowed. +- 0 — 所有查询都被允许。 +- 1 — 只有读取数据的查询被允许。 +- 2 — 读取数据以及变更设置的查询被允许。 -设置后 `readonly = 1`,用户无法更改 `readonly` 和 `allow_ddl` 当前会话中的设置。 +设置为 `readonly = 1` 后,用户无法在当前会话中更改 `readonly` 和 `allow_ddl` 设置。 -使用时 `GET` 方法中的 [HTTP接口](../../interfaces/http.md), `readonly = 1` 自动设置。 要修改数据,请使用 `POST` 方法。 +当使用 [HTTP接口](../../interfaces/http.md) 中的 `GET` 方法时,将自动设置为 `readonly = 1` 。 要修改数据,请使用 `POST` 方法。 -设置 `readonly = 1` 禁止用户更改所有设置。 有一种方法可以禁止用户 -从只更改特定设置,有关详细信息,请参阅 [对设置的限制](constraints-on-settings.md). +设置 `readonly = 1` 将禁止用户的更改任何设置。有一种方法可以只禁止用户更改特定的设置,有关详细信息,请参阅 [对设置的限制](constraints-on-settings.md)。 默认值:0 ## allow_ddl {#settings_allow_ddl} -允许或拒绝 [DDL](https://en.wikipedia.org/wiki/Data_definition_language) 查询。 +允许或拒绝 [DDL](https://zh.wikipedia.org/zh-cn/数据定义语言) 查询。 -查看查询如何划分为多种类型 [以上](#permissions_for_queries). +从 [上文](#permissions_for_queries) 查看查询是如何被划分为多种类型的。 可能的值: -- 0 — DDL queries are not allowed. -- 1 — DDL queries are allowed. +- 0 — DDL 查询不被允许。 +- 1 — DDL 查询被允许。 -你不能执行 `SET allow_ddl = 1` 如果 `allow_ddl = 0` 对于当前会话。 +如果对当前会话 `allow_ddl = 0` ,你就不能执行 `SET allow_ddl = 1` 。 默认值:1 From c38540786e1136e7ca7b63f67796a2918460c187 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 Oct 2021 15:11:00 +0300 Subject: [PATCH 872/919] Reimplement #28639 --- src/Storages/Kafka/StorageKafka.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 39688060b0a..25384db043f 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -408,7 +408,7 @@ ConsumerBufferPtr StorageKafka::createReadBuffer(const size_t consumer_number) } conf.set("client.software.name", VERSION_NAME); conf.set("client.software.version", VERSION_DESCRIBE); - conf.set("auto.offset.reset", "smallest"); // If no offset stored for this group, read all messages from the start + conf.set("auto.offset.reset", "earliest"); // If no offset stored for this group, read all messages from the start // that allows to prevent fast draining of the librdkafka queue // during building of single insert block. Improves performance From 6d7b0730362dc8a53a8554a127cb82ab2fc75c5d Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 31 Oct 2021 12:18:22 +0000 Subject: [PATCH 873/919] add test --- .../02111_modify_table_comment.reference | 4 +++ .../02111_modify_table_comment.sql | 32 +++++++++++++++++++ 2 files changed, 36 insertions(+) create mode 100644 tests/queries/0_stateless/02111_modify_table_comment.reference create mode 100644 tests/queries/0_stateless/02111_modify_table_comment.sql diff --git a/tests/queries/0_stateless/02111_modify_table_comment.reference b/tests/queries/0_stateless/02111_modify_table_comment.reference new file mode 100644 index 00000000000..bc58e6576e3 --- /dev/null +++ b/tests/queries/0_stateless/02111_modify_table_comment.reference @@ -0,0 +1,4 @@ +CREATE TABLE `02111_modify_table_comment`.t\n(\n `n` Int8\n)\nENGINE = MergeTree\nORDER BY n\nSETTINGS index_granularity = 8192\nCOMMENT \'this is a MergeTree table\' +CREATE TABLE `02111_modify_table_comment`.t\n(\n `n` Int8\n)\nENGINE = MergeTree\nORDER BY n\nSETTINGS index_granularity = 8192\nCOMMENT \'MergeTree Table\' +CREATE TABLE `02111_modify_table_comment`.t_merge\n(\n `n` Int8\n)\nENGINE = Merge(\'02111_modify_table_comment\', \'t\')\nCOMMENT \'this is a Merge table\' +CREATE TABLE `02111_modify_table_comment`.t_merge\n(\n `n` Int8\n)\nENGINE = Merge(\'02111_modify_table_comment\', \'t\')\nCOMMENT \'Merge Table\' diff --git a/tests/queries/0_stateless/02111_modify_table_comment.sql b/tests/queries/0_stateless/02111_modify_table_comment.sql new file mode 100644 index 00000000000..378b04517e8 --- /dev/null +++ b/tests/queries/0_stateless/02111_modify_table_comment.sql @@ -0,0 +1,32 @@ +DROP DATABASE IF EXISTS 02111_modify_table_comment; +CREATE DATABASE 02111_modify_table_comment; + +USE 02111_modify_table_comment; + +CREATE TABLE t +( + `n` Int8 +) +ENGINE = MergeTree +ORDER BY n +COMMENT 'this is a MergeTree table'; + +SHOW CREATE t; + +ALTER TABLE t + MODIFY COMMENT 'MergeTree Table'; + +SHOW CREATE t; + +CREATE TABLE t_merge AS t +ENGINE = Merge('02111_modify_table_comment', 't') +COMMENT 'this is a Merge table'; + +SHOW CREATE t_merge; + +ALTER TABLE t_merge + MODIFY COMMENT 'Merge Table'; + +SHOW CREATE t_merge; + +DROP DATABASE 02111_modify_table_comment; From 0710c1fb757aa35259d4d3653383220d1f4bd13c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 Oct 2021 15:43:48 +0300 Subject: [PATCH 874/919] Add benchmark for PostgreSQL --- benchmark/postgresql/benchmark.sh | 12 ++ benchmark/postgresql/instructions.md | 142 ++++++++++++++++++ benchmark/postgresql/log | 129 ++++++++++++++++ benchmark/postgresql/queries.sql | 43 ++++++ .../dbms/results/015_postgresql.json | 56 +++++++ 5 files changed, 382 insertions(+) create mode 100755 benchmark/postgresql/benchmark.sh create mode 100644 benchmark/postgresql/instructions.md create mode 100644 benchmark/postgresql/log create mode 100644 benchmark/postgresql/queries.sql create mode 100644 website/benchmark/dbms/results/015_postgresql.json diff --git a/benchmark/postgresql/benchmark.sh b/benchmark/postgresql/benchmark.sh new file mode 100755 index 00000000000..4a7139ec79d --- /dev/null +++ b/benchmark/postgresql/benchmark.sh @@ -0,0 +1,12 @@ +#!/bin/bash + +grep -v -P '^#' queries.sql | sed -e 's/{table}/hits_100m_pg/' | while read query; do + + echo 3 | sudo tee /proc/sys/vm/drop_caches + + 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; diff --git a/benchmark/postgresql/instructions.md b/benchmark/postgresql/instructions.md new file mode 100644 index 00000000000..296fa377bd1 --- /dev/null +++ b/benchmark/postgresql/instructions.md @@ -0,0 +1,142 @@ +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 new file mode 100644 index 00000000000..6a95561e6da --- /dev/null +++ b/benchmark/postgresql/log @@ -0,0 +1,129 @@ +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 new file mode 100644 index 00000000000..d7a2fe2d8bf --- /dev/null +++ b/benchmark/postgresql/queries.sql @@ -0,0 +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); diff --git a/website/benchmark/dbms/results/015_postgresql.json b/website/benchmark/dbms/results/015_postgresql.json new file mode 100644 index 00000000000..0298d9a54db --- /dev/null +++ b/website/benchmark/dbms/results/015_postgresql.json @@ -0,0 +1,56 @@ +[ + { + "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] + ] + } +] + From 02658bb52fa4c12e58ef5aa1049f90d4a2653228 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 Oct 2021 15:42:55 +0300 Subject: [PATCH 875/919] Minor clarifications --- website/benchmark/dbms/results/013_timescaledb.json | 2 +- website/benchmark/dbms/results/014_timescaledb.json | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/website/benchmark/dbms/results/013_timescaledb.json b/website/benchmark/dbms/results/013_timescaledb.json index fb829b0b040..836bf127910 100644 --- a/website/benchmark/dbms/results/013_timescaledb.json +++ b/website/benchmark/dbms/results/013_timescaledb.json @@ -3,7 +3,7 @@ "system": "TimescaleDB", "version": "2021-10-30", "data_size": 100000000, - "time": "", + "time": "2021-10-30", "comments": "", "result": [ diff --git a/website/benchmark/dbms/results/014_timescaledb.json b/website/benchmark/dbms/results/014_timescaledb.json index a1e63201c15..85f125c05a6 100644 --- a/website/benchmark/dbms/results/014_timescaledb.json +++ b/website/benchmark/dbms/results/014_timescaledb.json @@ -3,7 +3,7 @@ "system": "TimescaleDB (compressed)", "version": "2021-10-31", "data_size": 100000000, - "time": "", + "time": "2021-10-31", "comments": "", "result": [ From 0351946e922a33eee87c97481e8f3048919719dc Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 31 Oct 2021 17:34:47 +0300 Subject: [PATCH 876/919] Update 02111_modify_table_comment.sql --- tests/queries/0_stateless/02111_modify_table_comment.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02111_modify_table_comment.sql b/tests/queries/0_stateless/02111_modify_table_comment.sql index 378b04517e8..f9f864c7f35 100644 --- a/tests/queries/0_stateless/02111_modify_table_comment.sql +++ b/tests/queries/0_stateless/02111_modify_table_comment.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel + DROP DATABASE IF EXISTS 02111_modify_table_comment; CREATE DATABASE 02111_modify_table_comment; From 993f3faefd932d5177baa317a2e6bb27ff083464 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 Oct 2021 18:11:46 +0300 Subject: [PATCH 877/919] Miscellaneous --- src/Client/ClientBase.cpp | 8 +++++--- src/Common/clearPasswordFromCommandLine.cpp | 7 +++++-- .../SQLite/fetchSQLiteTableStructure.cpp | 10 +++++++--- src/Parsers/ASTFunction.cpp | 20 +++++++++++-------- src/Parsers/ExpressionListParsers.cpp | 6 +++++- src/Server/TCPHandler.cpp | 6 +++++- 6 files changed, 39 insertions(+), 18 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index dba8fc53045..6f2b178067f 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -52,6 +53,7 @@ #include namespace fs = std::filesystem; +using namespace std::literals; namespace DB @@ -1488,7 +1490,7 @@ void ClientBase::readArguments(int argc, char ** argv, Arguments & common_argume { const char * arg = argv[arg_num]; - if (0 == strcmp(arg, "--external")) + if (arg == "--external"sv) { in_external_group = true; external_tables_arguments.emplace_back(Arguments{""}); @@ -1503,8 +1505,8 @@ void ClientBase::readArguments(int argc, char ** argv, Arguments & common_argume } /// Options with value after whitespace. else if (in_external_group - && (0 == strcmp(arg, "--file") || 0 == strcmp(arg, "--name") || 0 == strcmp(arg, "--format") - || 0 == strcmp(arg, "--structure") || 0 == strcmp(arg, "--types"))) + && (arg == "--file"sv || arg == "--name"sv || arg == "--format"sv + || arg == "--structure"sv || arg == "--types"sv)) { if (arg_num + 1 < argc) { diff --git a/src/Common/clearPasswordFromCommandLine.cpp b/src/Common/clearPasswordFromCommandLine.cpp index 0ff56e25c3f..7cdf09c9b8f 100644 --- a/src/Common/clearPasswordFromCommandLine.cpp +++ b/src/Common/clearPasswordFromCommandLine.cpp @@ -1,11 +1,14 @@ #include -#include "clearPasswordFromCommandLine.h" +#include +#include + +using namespace std::literals; void clearPasswordFromCommandLine(int argc, char ** argv) { for (int arg = 1; arg < argc; ++arg) { - if (arg + 1 < argc && 0 == strcmp(argv[arg], "--password")) + if (arg + 1 < argc && argv[arg] == "--password"sv) { ++arg; memset(argv[arg], 0, strlen(argv[arg])); diff --git a/src/Databases/SQLite/fetchSQLiteTableStructure.cpp b/src/Databases/SQLite/fetchSQLiteTableStructure.cpp index c4acf5b3a3a..32341a36f3e 100644 --- a/src/Databases/SQLite/fetchSQLiteTableStructure.cpp +++ b/src/Databases/SQLite/fetchSQLiteTableStructure.cpp @@ -13,6 +13,10 @@ #include #include +#include + + +using namespace std::literals; namespace DB { @@ -58,15 +62,15 @@ std::shared_ptr fetchSQLiteTableStructure(sqlite3 * connectio for (int i = 0; i < col_num; ++i) { - if (strcmp(col_names[i], "name") == 0) + if (col_names[i] == "name"sv) { name_and_type.name = data_by_col[i]; } - else if (strcmp(col_names[i], "type") == 0) + else if (col_names[i] == "type"sv) { name_and_type.type = convertSQLiteDataType(data_by_col[i]); } - else if (strcmp(col_names[i], "notnull") == 0) + else if (col_names[i] == "notnull"sv) { is_nullable = (data_by_col[i][0] == '0'); } diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index e1a62dc9dce..ef1ede28e33 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -1,3 +1,5 @@ +#include + #include #include @@ -16,6 +18,8 @@ #include #include +using namespace std::literals; + namespace DB { @@ -339,7 +343,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format for (const char ** func = operators; *func; func += 2) { - if (0 == strcmp(name.c_str(), func[0])) + if (name == std::string_view(func[0])) { if (frame.need_parens) settings.ostr << '('; @@ -376,7 +380,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format } } - if (!written && 0 == strcmp(name.c_str(), "arrayElement")) + if (!written && name == "arrayElement"sv) { if (frame.need_parens) settings.ostr << '('; @@ -391,7 +395,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format settings.ostr << ')'; } - if (!written && 0 == strcmp(name.c_str(), "tupleElement")) + if (!written && name == "tupleElement"sv) { // fuzzer sometimes may insert tupleElement() created from ASTLiteral: // @@ -442,7 +446,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format } } - if (!written && 0 == strcmp(name.c_str(), "lambda")) + if (!written && name == "lambda"sv) { /// Special case: zero elements tuple in lhs of lambda is printed as (). /// Special case: one-element tuple in lhs of lambda is printed as its element. @@ -483,7 +487,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format for (const char ** func = operators; *func; func += 2) { - if (0 == strcmp(name.c_str(), func[0])) + if (name == std::string_view(func[0])) { if (frame.need_parens) settings.ostr << '('; @@ -500,7 +504,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format } } - if (!written && 0 == strcmp(name.c_str(), "array")) + if (!written && name == "array"sv) { settings.ostr << (settings.hilite ? hilite_operator : "") << '[' << (settings.hilite ? hilite_none : ""); for (size_t i = 0; i < arguments->children.size(); ++i) @@ -513,7 +517,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format written = true; } - if (!written && arguments->children.size() >= 2 && 0 == strcmp(name.c_str(), "tuple")) + if (!written && arguments->children.size() >= 2 && name == "tuple"sv) { settings.ostr << (settings.hilite ? hilite_operator : "") << '(' << (settings.hilite ? hilite_none : ""); for (size_t i = 0; i < arguments->children.size(); ++i) @@ -526,7 +530,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format written = true; } - if (!written && 0 == strcmp(name.c_str(), "map")) + if (!written && name == "map"sv) { settings.ostr << (settings.hilite ? hilite_operator : "") << "map(" << (settings.hilite ? hilite_none : ""); for (size_t i = 0; i < arguments->children.size(); ++i) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 3aa5c82884b..b6e80d83777 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -1,3 +1,5 @@ +#include + #include #include @@ -12,6 +14,8 @@ #include #include +using namespace std::literals; + namespace DB { @@ -345,7 +349,7 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node /** special exception for the access operator to the element of the array `x[y]`, which * contains the infix part '[' and the suffix ''] '(specified as' [') */ - if (0 == strcmp(it[0], "[")) + if (it[0] == "["sv) { if (pos->type != TokenType::ClosingSquareBracket) return false; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 3c36321b936..f85bd24a275 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -55,6 +56,9 @@ #include +using namespace std::literals; + + namespace CurrentMetrics { extern const Metric QueryThread; @@ -1844,7 +1848,7 @@ void TCPHandler::run() catch (Poco::Exception & e) { /// Timeout - not an error. - if (!strcmp(e.what(), "Timeout")) + if (e.what() == "Timeout"sv) { LOG_DEBUG(log, "Poco::Exception. Code: {}, e.code() = {}, e.displayText() = {}, e.what() = {}", ErrorCodes::POCO_EXCEPTION, e.code(), e.displayText(), e.what()); } From b9887273b181cb87307de7d6a110bfb385afdbf9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 31 Oct 2021 20:29:54 +0300 Subject: [PATCH 878/919] Fix yml --- .github/workflows/release.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index a396e188327..42969e485c8 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -23,7 +23,7 @@ jobs: path: ${{ runner.temp }}/docker_images_check/changed_images.json DocsRelease: needs: DockerHubPush - runs: [self-hosted, func-tester] + runs-on: [self-hosted, func-tester] steps: - name: Check out repository code uses: actions/checkout@v2 From 817ca26b7fadbf2c40b957f74789c509adb2a201 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 31 Oct 2021 20:33:49 +0300 Subject: [PATCH 879/919] Debug github event --- tests/ci/pr_info.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 14a97e510a2..47e25f15df1 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -10,6 +10,7 @@ DIFF_IN_DOCUMENTATION_EXT = [".html", ".md", ".yml", ".txt", ".css", ".js", ".xm class PRInfo: def __init__(self, github_event, need_orgs=False, need_changed_files=False): + print(github_event) if 'pull_request' in github_event: # pull request and other similar events self.number = github_event['number'] if 'after' in github_event: @@ -32,7 +33,7 @@ class PRInfo: diff = urllib.request.urlopen(diff_url) diff_object = PatchSet(diff, diff.headers.get_charsets()[0]) self.changed_files = { f.path for f in diff_object } - elif github_event['type'] == 'PushEvent': # push on master + else: self.number = 0 self.sha = github_event['after'] self.labels = {} @@ -43,8 +44,6 @@ class PRInfo: self.changed_files = [f['filename'] for f in diff['files']] else: self.changed_files = set([]) - else: - raise Exception("Unknown event type") def get_dict(self): return { From 2ea8b89cbbdbcb27352a1adbedff0241e0a92afe Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 31 Oct 2021 20:43:32 +0300 Subject: [PATCH 880/919] Fix strange mistake in documentation release process. --- docs/tools/release.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/release.sh b/docs/tools/release.sh index a7b51174fdb..ccc96b90cfc 100755 --- a/docs/tools/release.sh +++ b/docs/tools/release.sh @@ -41,7 +41,7 @@ then then sleep 1m # https://api.cloudflare.com/#zone-purge-files-by-cache-tags,-host-or-prefix - POST_DATA='{"hosts":["content.clickhouse.com"]}' + POST_DATA='{"hosts":["clickhouse.com"]}' curl -X POST "https://api.cloudflare.com/client/v4/zones/4fc6fb1d46e87851605aa7fa69ca6fe0/purge_cache" -H "Authorization: Bearer ${CLOUDFLARE_TOKEN}" -H "Content-Type:application/json" --data "${POST_DATA}" fi fi From 6ab92f70a17a8d586e2cb42f1f3673144227b806 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 Oct 2021 21:05:07 +0300 Subject: [PATCH 881/919] Add benchmark for DuckDB --- benchmark/duckdb/log | 15906 ++++++++++++++++ benchmark/duckdb/queries.sql | 43 + benchmark/duckdb/usability.md | 762 + .../benchmark/dbms/results/016_duckdb.json | 57 + 4 files changed, 16768 insertions(+) create mode 100644 benchmark/duckdb/log create mode 100644 benchmark/duckdb/queries.sql create mode 100644 benchmark/duckdb/usability.md create mode 100644 website/benchmark/dbms/results/016_duckdb.json diff --git a/benchmark/duckdb/log b/benchmark/duckdb/log new file mode 100644 index 00000000000..64a6ad6bbeb --- /dev/null +++ b/benchmark/duckdb/log @@ -0,0 +1,15906 @@ +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 new file mode 100644 index 00000000000..6089b47ea7c --- /dev/null +++ b/benchmark/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 sum(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 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))); diff --git a/benchmark/duckdb/usability.md b/benchmark/duckdb/usability.md new file mode 100644 index 00000000000..a032f68986a --- /dev/null +++ b/benchmark/duckdb/usability.md @@ -0,0 +1,762 @@ +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/website/benchmark/dbms/results/016_duckdb.json b/website/benchmark/dbms/results/016_duckdb.json new file mode 100644 index 00000000000..0b30e892c09 --- /dev/null +++ b/website/benchmark/dbms/results/016_duckdb.json @@ -0,0 +1,57 @@ +[ + { + "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] + ] + } +] + + From 14ea8f447703356212ed4c659ef8b3377f454d5a Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 31 Oct 2021 21:08:38 +0300 Subject: [PATCH 882/919] Fix docs release --- tests/ci/docs_release.py | 1 + tests/ci/pr_info.py | 6 ++++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/ci/docs_release.py b/tests/ci/docs_release.py index 7ce7028fbf5..2634cdf7b2e 100644 --- a/tests/ci/docs_release.py +++ b/tests/ci/docs_release.py @@ -150,4 +150,5 @@ if __name__ == "__main__": report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, lines, additional_files) print("::notice ::Report url: {report_url}") + commit = get_commit(gh, pr_info.sha) commit.create_status(context=NAME, description=description, state=status, target_url=report_url) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 47e25f15df1..b27f8217350 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -10,7 +10,6 @@ DIFF_IN_DOCUMENTATION_EXT = [".html", ".md", ".yml", ".txt", ".css", ".js", ".xm class PRInfo: def __init__(self, github_event, need_orgs=False, need_changed_files=False): - print(github_event) if 'pull_request' in github_event: # pull request and other similar events self.number = github_event['number'] if 'after' in github_event: @@ -33,7 +32,7 @@ class PRInfo: diff = urllib.request.urlopen(diff_url) diff_object = PatchSet(diff, diff.headers.get_charsets()[0]) self.changed_files = { f.path for f in diff_object } - else: + elif 'commits' in github_event: self.number = 0 self.sha = github_event['after'] self.labels = {} @@ -44,6 +43,9 @@ class PRInfo: self.changed_files = [f['filename'] for f in diff['files']] else: self.changed_files = set([]) + else: + raise Exception("Cannot detect type of event") + def get_dict(self): return { From cbab2aca69f5e44540672264c4a2923f6bce949d Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 31 Oct 2021 22:18:43 +0300 Subject: [PATCH 883/919] Trying to fix docs release --- tests/ci/docs_release.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/docs_release.py b/tests/ci/docs_release.py index 2634cdf7b2e..894ca8da4ac 100644 --- a/tests/ci/docs_release.py +++ b/tests/ci/docs_release.py @@ -106,7 +106,7 @@ if __name__ == "__main__": os.makedirs(test_output) token = os.getenv('CLOUDFLARE_TOKEN') - cmd = f"docker run --cap-add=SYS_PTRACE -e CLOUDFLARE_TOKEN={token} --volume={repo_path}:/repo_path --volume={test_output}:/output_path {docker_image}" + cmd = f"docker run --cap-add=SYS_PTRACE --volume=$SSH_AUTH_SOCK:/ssh-agent -e SSH_AUTH_SOCK=/ssh-agent -e CLOUDFLARE_TOKEN={token} --volume={repo_path}:/repo_path --volume={test_output}:/output_path {docker_image}" run_log_path = os.path.join(test_output, 'runlog.log') From a061bf1d79dc9ef082acafc0559068349638cb07 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 Oct 2021 22:26:43 +0300 Subject: [PATCH 884/919] Turn off logging in docs. release script just in case --- docs/tools/release.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/tools/release.sh b/docs/tools/release.sh index ccc96b90cfc..09044d690a1 100755 --- a/docs/tools/release.sh +++ b/docs/tools/release.sh @@ -37,6 +37,9 @@ then # Sometimes it does not work with error message "! [remote rejected] master -> master (cannot lock ref 'refs/heads/master': is at 42a0f6b6b6c7be56a469441b4bf29685c1cebac3 but expected 520e9b02c0d4678a2a5f41d2f561e6532fb98cc1)" for _ in {1..10}; do git push --force origin master && break; sleep 5; done + # Turn off logging. + set +x + if [[ ! -z "${CLOUDFLARE_TOKEN}" ]] then sleep 1m From 0da1c343a0e19eb2bfbfe36cda3b7ace719267a7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 31 Oct 2021 23:47:11 +0300 Subject: [PATCH 885/919] Fix style check --- tests/ci/docs_release.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/docs_release.py b/tests/ci/docs_release.py index 894ca8da4ac..832a3293ae2 100644 --- a/tests/ci/docs_release.py +++ b/tests/ci/docs_release.py @@ -106,7 +106,8 @@ if __name__ == "__main__": os.makedirs(test_output) token = os.getenv('CLOUDFLARE_TOKEN') - cmd = f"docker run --cap-add=SYS_PTRACE --volume=$SSH_AUTH_SOCK:/ssh-agent -e SSH_AUTH_SOCK=/ssh-agent -e CLOUDFLARE_TOKEN={token} --volume={repo_path}:/repo_path --volume={test_output}:/output_path {docker_image}" + cmd = "docker run --cap-add=SYS_PTRACE --volume=$SSH_AUTH_SOCK:/ssh-agent -e SSH_AUTH_SOCK=/ssh-agent " \ + f"-e CLOUDFLARE_TOKEN={token} --volume={repo_path}:/repo_path --volume={test_output}:/output_path {docker_image}" run_log_path = os.path.join(test_output, 'runlog.log') From d7dab834ec7492d408fe2e31b66651757d95325b Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 1 Nov 2021 00:33:03 +0300 Subject: [PATCH 886/919] Fix file progress for local --- src/Client/LocalConnection.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 55d3a2d6a5f..4f476b57c27 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -70,7 +70,10 @@ void LocalConnection::sendQuery( query_context = session.makeQueryContext(); query_context->setCurrentQueryId(query_id); if (send_progress) + { query_context->setProgressCallback([this] (const Progress & value) { return this->updateProgress(value); }); + query_context->setFileProgressCallback([this](const FileProgress & value) { this->updateProgress(Progress(value)); }); + } CurrentThread::QueryScope query_scope_holder(query_context); From 13ee782bda243363a3c30781e9116981b6dbfd13 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 1 Nov 2021 00:42:13 +0300 Subject: [PATCH 887/919] Minor improvement in universal install script --- docs/_includes/install/universal.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/_includes/install/universal.sh b/docs/_includes/install/universal.sh index eaea4cc69ed..ad0eb4979a3 100755 --- a/docs/_includes/install/universal.sh +++ b/docs/_includes/install/universal.sh @@ -50,7 +50,7 @@ URL="https://builds.clickhouse.com/master/${DIR}/clickhouse" echo echo "Will download ${URL}" echo -curl -O "${URL}" && chmod a+x clickhouse && +curl -O "${URL}" && chmod a+x clickhouse || exit 1 echo echo "Successfully downloaded the ClickHouse binary, you can run it as: ./clickhouse" From 0ff0cf6dd20e3d67118639f4f6d00fb74e034c61 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 1 Nov 2021 00:41:31 +0300 Subject: [PATCH 888/919] Fix --- tests/integration/test_odbc_interaction/test.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 4d2f70ad08c..001a46e1237 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -554,7 +554,7 @@ def test_concurrent_queries(started_cluster): busy_pool = Pool(5) p = busy_pool.map_async(node_insert, range(5)) p.wait() - assert_eq_with_retry(node1, "SELECT count() FROM test_pg_table", str(5*5*1000)) + assert_eq_with_retry(node1, "SELECT count() FROM test_pg_table", str(5*5*1000), retry_count=100) def node_insert_select(_): for i in range(5): @@ -564,7 +564,7 @@ def test_concurrent_queries(started_cluster): busy_pool = Pool(5) p = busy_pool.map_async(node_insert_select, range(5)) p.wait() - assert_eq_with_retry(node1, "SELECT count() FROM test_pg_table", str(5*5*1000*2)) + assert_eq_with_retry(node1, "SELECT count() FROM test_pg_table", str(5*5*1000*2), retry_count=100) node1.query('DROP TABLE test_pg_table;') cursor.execute('DROP TABLE clickhouse.test_pg_table;') @@ -627,4 +627,3 @@ def test_odbc_long_text(started_cluster): cursor.execute("""insert into clickhouse.test_long_text (flen, field1) values (400000, '{}')""".format(long_text)); result = node1.query("select field1 from test_long_text where flen=400000;") assert(result.strip() == long_text) - From d2363d625c9061a446f7a09ce951c0281340b5c0 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 1 Nov 2021 03:39:38 +0300 Subject: [PATCH 889/919] Fix reading from TinyLog. --- src/Storages/StorageLog.cpp | 119 +++++++++++++++++++----------- src/Storages/StorageStripeLog.cpp | 17 ++--- 2 files changed, 82 insertions(+), 54 deletions(-) diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index c4e9826e3a6..2ce023d2424 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -7,12 +7,13 @@ #include +#include #include +#include #include +#include #include #include -#include -#include #include @@ -63,15 +64,23 @@ public: } LogSource( - size_t block_size_, const NamesAndTypesList & columns_, const StorageLog & storage_, - size_t rows_limit_, const std::vector & offsets_, ReadSettings read_settings_) - : SourceWithProgress(getHeader(columns_)), - block_size(block_size_), - columns(columns_), - storage(storage_), - rows_limit(rows_limit_), - offsets(offsets_), - read_settings(std::move(read_settings_)) + size_t block_size_, + const NamesAndTypesList & columns_, + const StorageLog & storage_, + size_t rows_limit_, + const std::vector & offsets_, + const std::vector & file_sizes_, + bool limited_by_file_sizes_, + ReadSettings read_settings_) + : SourceWithProgress(getHeader(columns_)) + , block_size(block_size_) + , columns(columns_) + , storage(storage_) + , rows_limit(rows_limit_) + , offsets(offsets_) + , file_sizes(file_sizes_) + , limited_by_file_sizes(limited_by_file_sizes_) + , read_settings(std::move(read_settings_)) { } @@ -81,30 +90,38 @@ protected: Chunk generate() override; private: - size_t block_size; - NamesAndTypesList columns; + const size_t block_size; + const NamesAndTypesList columns; const StorageLog & storage; - size_t rows_limit; /// The maximum number of rows that can be read + const size_t rows_limit; /// The maximum number of rows that can be read size_t rows_read = 0; bool is_finished = false; - std::vector offsets; - ReadSettings read_settings; + const std::vector offsets; + const std::vector file_sizes; + const bool limited_by_file_sizes; + const ReadSettings read_settings; struct Stream { - /// We use `disk->getFileSize(data_path)` to get the size of the file here. - /// We cannot just use `storage.file_checker` for that purpose, because `storage.rwlock` is not locked at this point. - - Stream(const DiskPtr & disk, const String & data_path, size_t offset, ReadSettings read_settings_) - : plain(disk->readFile(data_path, read_settings_.adjustBufferSize(disk->getFileSize(data_path)))) - , compressed(*plain) + Stream(const DiskPtr & disk, const String & data_path, size_t offset, size_t file_size, bool limited_by_file_size, ReadSettings read_settings_) { + plain = disk->readFile(data_path, read_settings_.adjustBufferSize(file_size)); + if (offset) plain->seek(offset, SEEK_SET); + + if (limited_by_file_size) + { + limited.emplace(*plain, file_size - offset, false); + compressed.emplace(*limited); + } + else + compressed.emplace(*plain); } std::unique_ptr plain; - CompressedReadBuffer compressed; + std::optional limited; + std::optional compressed; }; using FileStreams = std::map; @@ -194,9 +211,10 @@ void LogSource::readData(const NameAndTypePair & name_and_type, ColumnPtr & colu const auto & data_file = *data_file_it->second; size_t offset = stream_for_prefix ? 0 : offsets[data_file.index]; + size_t file_size = file_sizes[data_file.index]; - auto it = streams.try_emplace(data_file_name, storage.disk, data_file.path, offset, read_settings).first; - return &it->second.compressed; + auto it = streams.try_emplace(data_file_name, storage.disk, data_file.path, offset, file_size, limited_by_file_sizes, read_settings).first; + return &it->second.compressed.value(); }; }; @@ -215,20 +233,24 @@ bool LogSource::isFinished() if (is_finished) return true; - if (rows_limit == std::numeric_limits::max()) + /// Check for row limit. + if (rows_read == rows_limit) { - /// No limit for the row count, check for EOF. - if (!streams.empty() && streams.begin()->second.compressed.eof()) - is_finished = true; - } - else - { - /// There is a limit for the row count, check that limit. - if (rows_read == rows_limit) - is_finished = true; + is_finished = true; + return true; } - return is_finished; + if (limited_by_file_sizes) + { + /// Check for EOF. + if (!streams.empty() && streams.begin()->second.compressed->eof()) + { + is_finished = true; + return true; + } + } + + return false; } @@ -768,33 +790,40 @@ Pipe StorageLog::read( std::vector offsets; offsets.resize(num_data_files, 0); + std::vector file_sizes; + file_sizes.resize(num_data_files, 0); + for (const auto & data_file : data_files) + file_sizes[data_file.index] = file_checker.getFileSize(data_file.path); + + /// For TinyLog (use_marks_file == false) there is no row limit and we just read + /// the data files up to their sizes. + bool limited_by_file_sizes = !use_marks_file; + size_t row_limit = std::numeric_limits::max(); + ReadSettings read_settings = context->getReadSettings(); Pipes pipes; for (size_t stream = 0; stream < num_streams; ++stream) { - size_t start_row, end_row; if (use_marks_file) { size_t mark_begin = stream * num_marks / num_streams; size_t mark_end = (stream + 1) * num_marks / num_streams; - start_row = mark_begin ? marks_with_real_row_count[mark_begin - 1].rows : 0; - end_row = mark_end ? marks_with_real_row_count[mark_end - 1].rows : 0; + size_t start_row = mark_begin ? marks_with_real_row_count[mark_begin - 1].rows : 0; + size_t end_row = mark_end ? marks_with_real_row_count[mark_end - 1].rows : 0; + row_limit = end_row - start_row; for (const auto & data_file : data_files) offsets[data_file.index] = data_file.marks[mark_begin].offset; } - else - { - start_row = 0; - end_row = std::numeric_limits::max(); // row limit not set - } pipes.emplace_back(std::make_shared( max_block_size, all_columns, *this, - end_row - start_row, + row_limit, offsets, + file_sizes, + limited_by_file_sizes, read_settings)); } diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 7229442d5ef..da620463ffa 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -82,7 +82,8 @@ public: ReadSettings read_settings_, std::shared_ptr indices_, IndexForNativeFormat::Blocks::const_iterator index_begin_, - IndexForNativeFormat::Blocks::const_iterator index_end_) + IndexForNativeFormat::Blocks::const_iterator index_end_, + size_t file_size_) : SourceWithProgress(getHeader(storage_, metadata_snapshot_, column_names, index_begin_, index_end_)) , storage(storage_) , metadata_snapshot(metadata_snapshot_) @@ -90,6 +91,7 @@ public: , indices(indices_) , index_begin(index_begin_) , index_end(index_end_) + , file_size(file_size_) { } @@ -125,6 +127,7 @@ private: std::shared_ptr indices; IndexForNativeFormat::Blocks::const_iterator index_begin; IndexForNativeFormat::Blocks::const_iterator index_end; + size_t file_size; Block header; @@ -143,12 +146,7 @@ private: started = true; String data_file_path = storage.table_path + "data.bin"; - - /// We cannot just use `storage.file_checker` to get the size of the file here, - /// because `storage.rwlock` is not locked at this point. - size_t data_file_size = storage.disk->getFileSize(data_file_path); - - data_in.emplace(storage.disk->readFile(data_file_path, read_settings.adjustBufferSize(data_file_size))); + data_in.emplace(storage.disk->readFile(data_file_path, read_settings.adjustBufferSize(file_size))); block_in.emplace(*data_in, 0, index_begin, index_end); } } @@ -351,7 +349,8 @@ Pipe StorageStripeLog::read( if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - if (!file_checker.getFileSize(data_file_path)) + size_t data_file_size = file_checker.getFileSize(data_file_path); + if (!data_file_size) return Pipe(std::make_shared(metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); auto indices_for_selected_columns @@ -373,7 +372,7 @@ Pipe StorageStripeLog::read( std::advance(end, (stream + 1) * size / num_streams); pipes.emplace_back(std::make_shared( - *this, metadata_snapshot, column_names, read_settings, indices_for_selected_columns, begin, end)); + *this, metadata_snapshot, column_names, read_settings, indices_for_selected_columns, begin, end, data_file_size)); } /// We do not keep read lock directly at the time of reading, because we read ranges of data that do not change. From 554b47dc737a98c20264e22adb1b5b23dc9c8869 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Mon, 1 Nov 2021 09:37:59 +0800 Subject: [PATCH 890/919] Update parts_columns.md --- docs/zh/operations/system-tables/parts_columns.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/operations/system-tables/parts_columns.md b/docs/zh/operations/system-tables/parts_columns.md index cd8b2fe9526..d603f251d30 100644 --- a/docs/zh/operations/system-tables/parts_columns.md +++ b/docs/zh/operations/system-tables/parts_columns.md @@ -1,6 +1,6 @@ # system.parts_columns {#system_tables-parts_columns} -包含关于[MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md)表的部分和列的信息. +包含关于[MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)表的部分和列的信息. 每一行描述一个数据部分. From 45676e4334ca14efcc0bba9c66e409333b8ad63f Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 1 Nov 2021 11:18:59 +0800 Subject: [PATCH 891/919] update version of snappy --- contrib/snappy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/snappy b/contrib/snappy index 3f194acb57e..fb057edfed8 160000 --- a/contrib/snappy +++ b/contrib/snappy @@ -1 +1 @@ -Subproject commit 3f194acb57e0487531c96b97af61dcbd025a78a3 +Subproject commit fb057edfed820212076239fd32cb2ff23e9016bf From 619d576b6f63d9f9a7e3abb604c6f97903c29e79 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 1 Nov 2021 11:19:22 +0800 Subject: [PATCH 892/919] switch snappy to https://github.com/ClickHouse-Extras/snappy --- .gitmodules | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index dd13597c183..8ad81b5094f 100644 --- a/.gitmodules +++ b/.gitmodules @@ -76,8 +76,7 @@ url = https://github.com/ClickHouse-Extras/libcxxabi.git [submodule "contrib/snappy"] path = contrib/snappy - url = https://github.com/taiyang-li/snappy.git - branch = fix_snappy_build_error + url = https://github.com/ClickHouse-Extras/snappy.git [submodule "contrib/cppkafka"] path = contrib/cppkafka url = https://github.com/mfontanini/cppkafka.git From ca22e7acfb4162fc39a8865562da9e660dd2e315 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 28 Oct 2021 21:00:33 +0300 Subject: [PATCH 893/919] Use existing local remote_replica_path var in StorageReplicatedMergeTree::dropReplica() --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f4a50f2e553..acc81c2c229 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -852,7 +852,7 @@ void StorageReplicatedMergeTree::dropReplica(zkutil::ZooKeeperPtr zookeeper, con LOG_INFO(logger, "Removing replica {}, marking it as lost", remote_replica_path); /// Mark itself lost before removing, because the following recursive removal may fail /// and partially dropped replica may be considered as alive one (until someone will mark it lost) - zookeeper->trySet(zookeeper_path + "/replicas/" + replica + "/is_lost", "1"); + zookeeper->trySet(remote_replica_path + "/is_lost", "1"); /// It may left some garbage if replica_path subtree are concurrently modified zookeeper->tryRemoveRecursive(remote_replica_path); if (zookeeper->exists(remote_replica_path)) From 60a411581f3178c963675d6c6b70fb7eb4a5c8c2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 28 Oct 2021 21:00:33 +0300 Subject: [PATCH 894/919] Fix possible "The local set of parts of X doesn't look like the set of parts in ZooKeeper" error If during removing replica_path from zookeeper, some error occurred (zookeeper goes away), then it may not remove everything from zookeeper. And on DETACH/ATTACH (or server restart, like stress tests does in the analysis from this comment [1]), it will trigger an error: The local set of parts of table test_1.alter_table_4 doesn't look like the set of parts in ZooKeeper: [1]: https://github.com/ClickHouse/ClickHouse/pull/28296#issuecomment-915829943 Fix this, by removing "metadata" at first, and only after this everything else, this will avoid this error, since on ATTACH such table will be marked as read-only. v2: forget to remove remote_replica_path itself v3: fix test_drop_replica by adding a check for remote_replica_path existence --- src/Storages/StorageReplicatedMergeTree.cpp | 37 ++++++++++++++++++++- 1 file changed, 36 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index acc81c2c229..63bb8af9148 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -849,12 +849,47 @@ void StorageReplicatedMergeTree::dropReplica(zkutil::ZooKeeperPtr zookeeper, con throw Exception("Table was not dropped because ZooKeeper session has expired.", ErrorCodes::TABLE_WAS_NOT_DROPPED); auto remote_replica_path = zookeeper_path + "/replicas/" + replica; + LOG_INFO(logger, "Removing replica {}, marking it as lost", remote_replica_path); /// Mark itself lost before removing, because the following recursive removal may fail /// and partially dropped replica may be considered as alive one (until someone will mark it lost) zookeeper->trySet(remote_replica_path + "/is_lost", "1"); + + /// NOTE: we should check for remote_replica_path existence, + /// since otherwise DROP REPLICA will fail if the replica had been already removed. + if (!zookeeper->exists(remote_replica_path)) + { + LOG_INFO(logger, "Removing replica {} does not exist", remote_replica_path); + return; + } + + /// Analog of removeRecursive(remote_replica_path) + /// but it removes "metadata" firstly. + /// + /// This will allow to mark table as readonly + /// and skip any checks of parts between on-disk and in the zookeeper. + /// + /// Without this removeRecursive() may remove "parts" first + /// and on DETACH/ATTACH (or server restart) it will trigger the following error: + /// + /// "The local set of parts of table X doesn't look like the set of parts in ZooKeeper" + /// + { + Strings children = zookeeper->getChildren(remote_replica_path); + + if (std::find(children.begin(), children.end(), "metadata") != children.end()) + zookeeper->remove(fs::path(remote_replica_path) / "metadata"); + + for (const auto & child : children) + { + if (child != "metadata") + zookeeper->removeRecursive(fs::path(remote_replica_path) / child); + } + + zookeeper->remove(remote_replica_path); + } + /// It may left some garbage if replica_path subtree are concurrently modified - zookeeper->tryRemoveRecursive(remote_replica_path); if (zookeeper->exists(remote_replica_path)) LOG_ERROR(logger, "Replica was not completely removed from ZooKeeper, {} still exists and may contain some garbage.", remote_replica_path); From 54e404de32ccc46c99c37e0eb752cc6cf6a5665e Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Nov 2021 10:33:52 +0300 Subject: [PATCH 895/919] Add github to known hosts in docs release --- docker/docs/release/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/docs/release/run.sh b/docker/docs/release/run.sh index 3ecfd26cb44..e465098c159 100644 --- a/docker/docs/release/run.sh +++ b/docker/docs/release/run.sh @@ -6,4 +6,5 @@ mkdir venv virtualenv -p $(which python3) venv source venv/bin/activate python3 -m pip install --ignore-installed -r requirements.txt +mkdir -p ~/.ssh && ssh-keyscan -t rsa github.com >> ~/.ssh/known_hosts ./release.sh 2>&1 | tee tee $OUTPUT_PATH/output.log From c11cf7ad9b5cb459426920ee8015bf50071e3c7d Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Nov 2021 11:04:28 +0300 Subject: [PATCH 896/919] Fixes in workflows --- .github/workflows/main.yml | 2 +- .github/workflows/release.yml | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 3d1c9730f99..3d651f43b4d 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -231,7 +231,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH FinishCheck: - needs: [StyleCheck, DockerHubPush, CheckLabels, BuilderReport, FastTest, FunctionalStatelessTestDebug, FunctionalStatefulTestDebug] + needs: [StyleCheck, DockerHubPush, CheckLabels, BuilderReport, FastTest, FunctionalStatelessTestDebug, FunctionalStatefulTestDebug, DocsCheck] runs-on: [self-hosted, style-checker] steps: - name: Check out repository code diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 42969e485c8..c8c7cd7b988 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -6,6 +6,9 @@ on: # yamllint disable-line rule:truthy push: branches: - master + paths: + - 'docs/**' + - 'website/**' jobs: DockerHubPush: runs-on: [self-hosted, style-checker] From ecf874c29bad78c0501d758dadaf15be944e88b6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Nov 2021 11:05:55 +0300 Subject: [PATCH 897/919] Better --- .github/workflows/release.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index c8c7cd7b988..63652ee633e 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -9,6 +9,7 @@ on: # yamllint disable-line rule:truthy paths: - 'docs/**' - 'website/**' + - 'benchmark/**' jobs: DockerHubPush: runs-on: [self-hosted, style-checker] From 9b87877898cab37a6176724d7c8679c0b75b1bed Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Nov 2021 11:07:42 +0300 Subject: [PATCH 898/919] Better name --- .github/workflows/release.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 63652ee633e..1c97d3f1d96 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -1,4 +1,4 @@ -name: ReleaseChecks +name: DocsReleaseChecks concurrency: group: master-release cancel-in-progress: true From a857b895d48f78122a3e92e9999249c2ff9be9a0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Nov 2021 11:18:02 +0300 Subject: [PATCH 899/919] Also fixes in docker --- .github/workflows/release.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 1c97d3f1d96..dd576b04c8c 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -10,6 +10,7 @@ on: # yamllint disable-line rule:truthy - 'docs/**' - 'website/**' - 'benchmark/**' + - 'docker/**' jobs: DockerHubPush: runs-on: [self-hosted, style-checker] From 19e690f86acabcc34da0be99ffa05685aa7c0f63 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Nov 2021 11:22:35 +0300 Subject: [PATCH 900/919] Parse json from response --- tests/ci/pr_info.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index b27f8217350..46499514027 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -38,11 +38,16 @@ class PRInfo: self.labels = {} if need_changed_files: commit_before = github_event['before'] - diff = requests.get(f'https://api.github.com/repos/ClickHouse/ClickHouse/compare/{commit_before}...{self.sha}') + response = requests.get(f'https://api.github.com/repos/ClickHouse/ClickHouse/compare/{commit_before}...{self.sha}') + response.raise_for_status() + diff = response.json() + if 'files' in diff: self.changed_files = [f['filename'] for f in diff['files']] else: self.changed_files = set([]) + else: + self.changed_files = set([]) else: raise Exception("Cannot detect type of event") From 31a8bcd3e18750baa3c2f5acb9baf87036b53e0d Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 1 Nov 2021 11:25:24 +0300 Subject: [PATCH 901/919] fix test_backward_compatibility --- .../test_backward_compatibility/test_aggregate_fixed_key.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py index 8819be527fd..98658ec81d0 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="aggregate_fixed_key") node1 = cluster.add_instance('node1', with_zookeeper=True, image='yandex/clickhouse-server', tag='21.3', with_installed_binary=True) node2 = cluster.add_instance('node2', with_zookeeper=True, image='yandex/clickhouse-server') node3 = cluster.add_instance('node3', with_zookeeper=True, image='yandex/clickhouse-server') From a1f29d31ea081cf055238be9d4a253327068db1d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 26 Oct 2021 12:48:31 +0300 Subject: [PATCH 902/919] Support BACKUP & RESTORE for log family. --- src/Common/FileChecker.cpp | 5 + src/Common/FileChecker.h | 2 + src/Storages/IStorage.cpp | 2 +- src/Storages/IStorage.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 4 +- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageLog.cpp | 163 ++++++++++++++++++ src/Storages/StorageLog.h | 3 + src/Storages/StorageStripeLog.cpp | 138 +++++++++++++++ src/Storages/StorageStripeLog.h | 5 +- .../test_backup_restore_new/test.py | 17 +- 11 files changed, 331 insertions(+), 12 deletions(-) diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index 64c13ceb69c..4de5a92a1b8 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -38,6 +38,11 @@ void FileChecker::setPath(const String & file_info_path_) files_info_path = file_info_path_; } +String FileChecker::getPath() const +{ + return files_info_path; +} + void FileChecker::update(const String & full_file_path) { bool exists = disk->exists(full_file_path); diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index 325e9325267..a0ea449393e 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -13,7 +13,9 @@ class FileChecker { public: FileChecker(DiskPtr disk_, const String & file_info_path_); + void setPath(const String & file_info_path_); + String getPath() const; void update(const String & full_file_path); void setEmpty(const String & full_file_path); diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index dd851f19906..021335fea1f 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -218,7 +218,7 @@ bool IStorage::isStaticStorage() const return false; } -BackupEntries IStorage::backup(const ASTs &, ContextPtr) const +BackupEntries IStorage::backup(const ASTs &, ContextPtr) { throw Exception("Table engine " + getName() + " doesn't support backups", ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 6c4bb44b63b..fa5f2c28b06 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -205,7 +205,7 @@ public: NameDependencies getDependentViewsByColumn(ContextPtr context) const; /// Prepares entries to backup data of the storage. - virtual BackupEntries backup(const ASTs & partitions, ContextPtr context) const; + virtual BackupEntries backup(const ASTs & partitions, ContextPtr context); /// Extract data from the backup and put it to the storage. virtual RestoreDataTasks restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index daa496fa517..c5c14cd24a8 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3500,7 +3500,7 @@ Pipe MergeTreeData::alterPartition( } -BackupEntries MergeTreeData::backup(const ASTs & partitions, ContextPtr local_context) const +BackupEntries MergeTreeData::backup(const ASTs & partitions, ContextPtr local_context) { DataPartsVector data_parts; if (partitions.empty()) @@ -3522,7 +3522,7 @@ BackupEntries MergeTreeData::backupDataParts(const DataPartsVector & data_parts) 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/backup_")).first; auto temp_dir_owner = temp_dir_it->second; fs::path temp_dir = temp_dir_owner->getPath(); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 22ec7ce6f53..1b617a2ec71 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -630,7 +630,7 @@ public: TableLockHolder & table_lock_holder); /// Prepares entries to backup data of the storage. - BackupEntries backup(const ASTs & partitions, ContextPtr context) const override; + BackupEntries backup(const ASTs & partitions, ContextPtr context) override; static BackupEntries backupDataParts(const DataPartsVector & data_parts); /// Extract data from the backup and put it to the storage. diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 2ce023d2424..e59579c5a33 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include @@ -27,6 +28,11 @@ #include #include +#include +#include +#include +#include + #include #include @@ -46,6 +52,7 @@ namespace ErrorCodes extern const int SIZES_OF_MARKS_FILES_ARE_INCONSISTENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_FILE_NAME; + extern const int NOT_IMPLEMENTED; } /// NOTE: The lock `StorageLog::rwlock` is NOT kept locked while reading, @@ -879,6 +886,162 @@ IStorage::ColumnSizeByName StorageLog::getColumnSizes() const } +BackupEntries StorageLog::backup(const ASTs & partitions, ContextPtr context) +{ + if (!partitions.empty()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); + + auto lock_timeout = getLockTimeout(context); + loadMarks(lock_timeout); + + ReadLock lock{rwlock, lock_timeout}; + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + + if (!num_data_files || !file_checker.getFileSize(data_files[INDEX_WITH_REAL_ROW_COUNT].path)) + return {}; + + auto temp_dir_owner = std::make_shared(disk, "tmp/backup_"); + auto temp_dir = temp_dir_owner->getPath(); + disk->createDirectories(temp_dir); + + BackupEntries backup_entries; + + /// *.bin + for (const auto & data_file : data_files) + { + /// We make a copy of the data file because it can be changed later in write() or in truncate(). + String data_file_name = fileName(data_file.path); + String temp_file_path = temp_dir + "/" + data_file_name; + disk->copy(data_file.path, disk, temp_file_path); + backup_entries.emplace_back( + data_file_name, + std::make_unique( + disk, temp_file_path, file_checker.getFileSize(data_file.path), std::nullopt, temp_dir_owner)); + } + + /// __marks.mrk + if (use_marks_file) + { + /// We make a copy of the data file because it can be changed later in write() or in truncate(). + String marks_file_name = fileName(marks_file_path); + String temp_file_path = temp_dir + "/" + marks_file_name; + disk->copy(marks_file_path, disk, temp_file_path); + backup_entries.emplace_back( + marks_file_name, + std::make_unique( + disk, temp_file_path, file_checker.getFileSize(marks_file_path), std::nullopt, temp_dir_owner)); + } + + /// sizes.json + String files_info_path = file_checker.getPath(); + backup_entries.emplace_back(fileName(files_info_path), std::make_unique(disk, files_info_path)); + + /// columns.txt + backup_entries.emplace_back( + "columns.txt", std::make_unique(getInMemoryMetadata().getColumns().getAllPhysical().toString())); + + /// count.txt + if (use_marks_file) + { + size_t num_rows = data_files[INDEX_WITH_REAL_ROW_COUNT].marks.empty() ? 0 : data_files[INDEX_WITH_REAL_ROW_COUNT].marks.back().rows; + backup_entries.emplace_back("count.txt", std::make_unique(toString(num_rows))); + } + + return backup_entries; +} + +RestoreDataTasks StorageLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) +{ + if (!partitions.empty()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); + + auto restore_task = [this, backup, data_path_in_backup, context]() + { + auto lock_timeout = getLockTimeout(context); + WriteLock lock{rwlock, lock_timeout}; + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + + if (!num_data_files) + return; + + /// Load the marks if not loaded yet. We have to do that now because we're going to update these marks. + loadMarks(lock); + + /// If there were no files, save zero file sizes to be able to rollback in case of error. + saveFileSizes(lock); + + try + { + /// Append data files. + for (const auto & data_file : data_files) + { + String file_path_in_backup = data_path_in_backup + fileName(data_file.path); + auto backup_entry = backup->read(file_path_in_backup); + auto in = backup_entry->getReadBuffer(); + auto out = disk->writeFile(data_file.path, max_compress_block_size, WriteMode::Append); + copyData(*in, *out); + } + + if (use_marks_file) + { + /// Append marks. + size_t num_extra_marks = 0; + String file_path_in_backup = data_path_in_backup + fileName(marks_file_path); + size_t file_size = backup->getSize(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); + + num_extra_marks = file_size / (num_data_files * sizeof(Mark)); + + size_t num_marks = data_files[0].marks.size(); + for (auto & data_file : data_files) + data_file.marks.reserve(num_marks + num_extra_marks); + + std::vector old_data_sizes; + std::vector old_num_rows; + old_data_sizes.resize(num_data_files); + old_num_rows.resize(num_data_files); + for (size_t i = 0; i != num_data_files; ++i) + { + old_data_sizes[i] = file_checker.getFileSize(data_files[i].path); + old_num_rows[i] = num_marks ? data_files[i].marks[num_marks - 1].rows : 0; + } + + auto backup_entry = backup->read(file_path_in_backup); + auto marks_rb = backup_entry->getReadBuffer(); + + for (size_t i = 0; i != num_extra_marks; ++i) + { + for (size_t j = 0; j != num_data_files; ++j) + { + Mark mark; + mark.read(*marks_rb); + mark.rows += old_num_rows[j]; /// Adjust the number of rows. + mark.offset += old_data_sizes[j]; /// Adjust the offset. + data_files[j].marks.push_back(mark); + } + } + } + + /// Finish writing. + saveMarks(lock); + saveFileSizes(lock); + } + catch (...) + { + /// Rollback partial writes. + file_checker.repair(); + removeUnsavedMarks(lock); + throw; + } + + }; + return {restore_task}; +} + + void registerStorageLog(StorageFactory & factory) { StorageFactory::StorageFeatures features{ diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index ca87d7dcf3e..8b2ef0ccac1 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -51,6 +51,9 @@ public: bool supportsSubcolumns() const override { return true; } ColumnSizeByName getColumnSizes() const override; + BackupEntries backup(const ASTs & partitions, ContextPtr context) override; + RestoreDataTasks restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; + protected: /** Attach the table with the appropriate name, along the appropriate path (with / at the end), * (the correctness of names and paths is not verified) diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index da620463ffa..92664a29767 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -9,10 +9,12 @@ #include #include +#include #include #include #include #include +#include #include #include @@ -33,6 +35,13 @@ #include #include +#include +#include +#include +#include + +#include + #include @@ -44,6 +53,7 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_FILE_NAME; extern const int TIMEOUT_EXCEEDED; + extern const int NOT_IMPLEMENTED; } @@ -481,6 +491,134 @@ void StorageStripeLog::saveFileSizes(const WriteLock & /* already locked for wri } +BackupEntries StorageStripeLog::backup(const ASTs & partitions, ContextPtr context) +{ + if (!partitions.empty()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); + + auto lock_timeout = getLockTimeout(context); + loadIndices(lock_timeout); + + ReadLock lock{rwlock, lock_timeout}; + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + + if (!file_checker.getFileSize(data_file_path)) + return {}; + + auto temp_dir_owner = std::make_shared(disk, "tmp/backup_"); + auto temp_dir = temp_dir_owner->getPath(); + disk->createDirectories(temp_dir); + + BackupEntries backup_entries; + + /// data.bin + { + /// We make a copy of the data file because it can be changed later in write() or in truncate(). + String data_file_name = fileName(data_file_path); + String temp_file_path = temp_dir + "/" + data_file_name; + disk->copy(data_file_path, disk, temp_file_path); + backup_entries.emplace_back( + data_file_name, + std::make_unique( + disk, temp_file_path, file_checker.getFileSize(data_file_path), std::nullopt, temp_dir_owner)); + } + + /// index.mrk + { + /// We make a copy of the data file because it can be changed later in write() or in truncate(). + String index_file_name = fileName(index_file_path); + String temp_file_path = temp_dir + "/" + index_file_name; + disk->copy(index_file_path, disk, temp_file_path); + backup_entries.emplace_back( + index_file_name, + std::make_unique( + disk, temp_file_path, file_checker.getFileSize(index_file_path), std::nullopt, temp_dir_owner)); + } + + /// sizes.json + String files_info_path = file_checker.getPath(); + backup_entries.emplace_back(fileName(files_info_path), std::make_unique(disk, files_info_path)); + + /// columns.txt + backup_entries.emplace_back( + "columns.txt", std::make_unique(getInMemoryMetadata().getColumns().getAllPhysical().toString())); + + /// count.txt + size_t num_rows = 0; + for (const auto & block : indices.blocks) + num_rows += block.num_rows; + backup_entries.emplace_back("count.txt", std::make_unique(toString(num_rows))); + + return backup_entries; +} + +RestoreDataTasks StorageStripeLog::restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) +{ + if (!partitions.empty()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Table engine {} doesn't support partitions", getName()); + + auto restore_task = [this, backup, data_path_in_backup, context]() + { + WriteLock lock{rwlock, getLockTimeout(context)}; + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + + /// Load the indices if not loaded yet. We have to do that now because we're going to update these indices. + loadIndices(lock); + + /// If there were no files, save zero file sizes to be able to rollback in case of error. + saveFileSizes(lock); + + try + { + /// Append the data file. + auto old_data_size = file_checker.getFileSize(data_file_path); + { + String file_path_in_backup = data_path_in_backup + fileName(data_file_path); + auto backup_entry = backup->read(file_path_in_backup); + auto in = backup_entry->getReadBuffer(); + auto out = disk->writeFile(data_file_path, max_compress_block_size, WriteMode::Append); + copyData(*in, *out); + } + + /// Append the index. + String index_path_in_backup = data_path_in_backup + fileName(index_file_path); + if (backup->exists(index_path_in_backup)) + { + IndexForNativeFormat extra_indices; + auto backup_entry = backup->read(index_path_in_backup); + auto index_in = backup_entry->getReadBuffer(); + CompressedReadBuffer index_compressed_in{*index_in}; + extra_indices.read(index_compressed_in); + + /// Adjust the offsets. + for (auto & block : extra_indices.blocks) + { + for (auto & column : block.columns) + column.location.offset_in_compressed_file += old_data_size; + } + + insertAtEnd(indices.blocks, std::move(extra_indices.blocks)); + } + + /// Finish writing. + saveIndices(lock); + saveFileSizes(lock); + } + catch (...) + { + /// Rollback partial writes. + file_checker.repair(); + removeUnsavedIndices(lock); + throw; + } + + }; + return {restore_task}; +} + + void registerStorageStripeLog(StorageFactory & factory) { StorageFactory::StorageFeatures features{ diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index 7ab40f867dd..579e2f991e7 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -50,6 +50,9 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder&) override; + BackupEntries backup(const ASTs & partitions, ContextPtr context) override; + RestoreDataTasks restoreFromBackup(const BackupPtr & backup, const String & data_path_in_backup, const ASTs & partitions, ContextMutablePtr context) override; + protected: StorageStripeLog( DiskPtr disk_, @@ -92,7 +95,7 @@ private: const size_t max_compress_block_size; - std::shared_timed_mutex rwlock; + mutable std::shared_timed_mutex rwlock; Poco::Logger * log; }; diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index 472ecaf608b..3ba73b3405f 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -6,9 +6,11 @@ cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance') -def create_and_fill_table(): +def create_and_fill_table(engine="MergeTree"): + if engine == "MergeTree": + engine = "MergeTree ORDER BY y PARTITION BY x%10" instance.query("CREATE DATABASE test") - instance.query("CREATE TABLE test.table(x UInt32, y String) ENGINE=MergeTree ORDER BY y PARTITION BY x%10") + instance.query(f"CREATE TABLE test.table(x UInt32, y String) ENGINE={engine}") instance.query("INSERT INTO test.table SELECT number, toString(number) FROM numbers(100)") @@ -36,9 +38,11 @@ def new_backup_name(): return f"test-backup-{backup_id_counter}" -def test_restore_table(): + +@pytest.mark.parametrize("engine", ["MergeTree", "Log", "TinyLog", "StripeLog"]) +def test_restore_table(engine): backup_name = new_backup_name() - create_and_fill_table() + create_and_fill_table(engine=engine) assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" instance.query(f"BACKUP TABLE test.table TO '{backup_name}'") @@ -50,9 +54,10 @@ def test_restore_table(): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" -def test_restore_table_into_existing_table(): +@pytest.mark.parametrize("engine", ["MergeTree", "Log", "TinyLog", "StripeLog"]) +def test_restore_table_into_existing_table(engine): backup_name = new_backup_name() - create_and_fill_table() + create_and_fill_table(engine=engine) assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" instance.query(f"BACKUP TABLE test.table TO '{backup_name}'") From 4e63311bb6918a6b48564b87b015bfca900aafad Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 1 Nov 2021 12:13:09 +0300 Subject: [PATCH 903/919] Try fix perf test --- docker/test/performance-comparison/compare.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 38595d47528..2e8517d71c9 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -634,7 +634,7 @@ create view query_display_names as select * from create view partial_query_times as select * from file('analyze/partial-query-times.tsv', TSVWithNamesAndTypes, - 'test text, query_index int, time_stddev float, time_median float') + 'test text, query_index int, time_stddev float, time_median double') ; -- Report for partial queries that we could only run on the new server (e.g. From 77edd081a58f507b94a18e92040ff315474a8130 Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Mon, 1 Nov 2021 13:00:21 +0300 Subject: [PATCH 904/919] Fix whitespace in src/Disks/S3/DiskS3.cpp --- src/Disks/S3/DiskS3.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index af7d6467582..3e99ca1a886 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -128,7 +128,6 @@ void throwIfError(const Aws::Utils::Outcome & response) throw Exception(err.GetMessage(), static_cast(err.GetErrorType())); } } - template void logIfError(Aws::Utils::Outcome & response, Fn auto && msg) { From 47b717ceff0a18c2d6408da180bf3691578e8597 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Nov 2021 13:27:46 +0300 Subject: [PATCH 905/919] Add stress test to github actions --- .github/workflows/main.yml | 31 ++- tests/ci/metrics_lambda/app.py | 2 +- tests/ci/stress_check.py | 273 ++++++++++++++++++++++++++ tests/ci/worker/init_func_tester.sh | 20 ++ tests/ci/worker/init_stress_tester.sh | 20 ++ 5 files changed, 344 insertions(+), 2 deletions(-) create mode 100644 tests/ci/stress_check.py create mode 100644 tests/ci/worker/init_func_tester.sh create mode 100644 tests/ci/worker/init_stress_tester.sh diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 3d651f43b4d..f1c0ea7cbac 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -207,6 +207,35 @@ jobs: docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH + StressTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stress_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stress tests (debug, actions)' + REPO_COPY: ${{runner.temp}}/stress_debug/ClickHouse + REQUIRED_BUILD_NUMBER: 7 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 stress_check.py "$CHECK_NAME" $REQUIRED_BUILD_NUMBER + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH FastTest: needs: DockerHubPush if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} @@ -231,7 +260,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH FinishCheck: - needs: [StyleCheck, DockerHubPush, CheckLabels, BuilderReport, FastTest, FunctionalStatelessTestDebug, FunctionalStatefulTestDebug, DocsCheck] + needs: [StyleCheck, DockerHubPush, CheckLabels, BuilderReport, FastTest, FunctionalStatelessTestDebug, FunctionalStatefulTestDebug, DocsCheck, StressTestDebug] runs-on: [self-hosted, style-checker] steps: - name: Check out repository code diff --git a/tests/ci/metrics_lambda/app.py b/tests/ci/metrics_lambda/app.py index af0e0fe07f1..d88e3e45b07 100644 --- a/tests/ci/metrics_lambda/app.py +++ b/tests/ci/metrics_lambda/app.py @@ -79,7 +79,7 @@ def list_runners(access_token): def group_runners_by_tag(listed_runners): result = {} - RUNNER_TYPE_LABELS = ['style-checker', 'builder', 'func-tester'] + RUNNER_TYPE_LABELS = ['style-checker', 'builder', 'func-tester', 'stress-tester'] for runner in listed_runners: for tag in runner.tags: if tag in RUNNER_TYPE_LABELS: diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py new file mode 100644 index 00000000000..ebac2cd86aa --- /dev/null +++ b/tests/ci/stress_check.py @@ -0,0 +1,273 @@ +#!/usr/bin/env python3 + +#!/usr/bin/env python3 + +import csv +import logging +import subprocess +import os +import json +import time +import sys + +from github import Github +import requests + +from report import create_test_html_report +from s3_helper import S3Helper +from get_robot_token import get_best_robot_token +from pr_info import PRInfo + + +DOWNLOAD_RETRIES_COUNT = 5 + +def dowload_build_with_progress(url, path): + logging.info("Downloading from %s to temp path %s", url, path) + for i in range(DOWNLOAD_RETRIES_COUNT): + try: + with open(path, 'wb') as f: + response = requests.get(url, stream=True) + response.raise_for_status() + total_length = response.headers.get('content-length') + if total_length is None or int(total_length) == 0: + logging.info("No content-length, will download file without progress") + f.write(response.content) + else: + dl = 0 + total_length = int(total_length) + logging.info("Content length is %ld bytes", total_length) + for data in response.iter_content(chunk_size=4096): + dl += len(data) + f.write(data) + if sys.stdout.isatty(): + done = int(50 * dl / total_length) + percent = int(100 * float(dl) / total_length) + eq_str = '=' * done + space_str = ' ' * (50 - done) + sys.stdout.write(f"\r[{eq_str}{space_str}] {percent}%") + sys.stdout.flush() + break + except Exception as ex: + sys.stdout.write("\n") + time.sleep(3) + logging.info("Exception while downloading %s, retry %s", ex, i + 1) + if os.path.exists(path): + os.remove(path) + else: + raise Exception(f"Cannot download dataset from {url}, all retries exceeded") + + sys.stdout.write("\n") + logging.info("Downloading finished") + + +def download_builds(result_path, build_urls): + for url in build_urls: + if url.endswith('.deb'): + fname = os.path.basename(url) + logging.info("Will download %s to %s", fname, result_path) + dowload_build_with_progress(url, os.path.join(result_path, fname)) + +def get_build_config(build_number, repo_path): + ci_config_path = os.path.join(repo_path, "tests/ci/ci_config.json") + with open(ci_config_path, 'r', encoding='utf-8') as ci_config: + config_dict = json.load(ci_config) + return config_dict['build_config'][build_number] + +def get_build_urls(build_config_str, reports_path): + for root, _, files in os.walk(reports_path): + for f in files: + if build_config_str in f : + logging.info("Found build report json %s", f) + with open(os.path.join(root, f), 'r', encoding='utf-8') as file_handler: + build_report = json.load(file_handler) + return build_report['build_urls'] + return [] + +def build_config_to_string(build_config): + if build_config["package-type"] == "performance": + return "performance" + + return "_".join([ + build_config['compiler'], + build_config['build-type'] if build_config['build-type'] else "relwithdebuginfo", + build_config['sanitizer'] if build_config['sanitizer'] else "none", + build_config['bundled'], + build_config['splitted'], + "tidy" if build_config['tidy'] == "enable" else "notidy", + "with_coverage" if build_config['with_coverage'] else "without_coverage", + build_config['package-type'], + ]) + + +def get_run_command(build_path, result_folder, server_log_folder, image): + cmd = "docker run " + \ + f"--volume={build_path}:/package_folder " \ + f"--volume={result_folder}:/test_output " \ + f"--volume={server_log_folder}:/var/log/clickhouse-server {image}" + + return cmd + +def process_logs(s3_client, additional_logs, s3_path_prefix): + additional_urls = [] + for log_path in additional_logs: + if log_path: + additional_urls.append( + s3_client.upload_test_report_to_s3( + log_path, + s3_path_prefix + "/" + os.path.basename(log_path))) + + return additional_urls + +def upload_results(s3_client, pr_number, commit_sha, test_results, raw_log, additional_files, check_name): + additional_files = [raw_log] + additional_files + s3_path_prefix = f"{pr_number}/{commit_sha}/" + check_name.lower().replace(' ', '_').replace('(', '_').replace(')', '_').replace(',', '_') + additional_urls = process_logs(s3_client, additional_files, s3_path_prefix) + + branch_url = "https://github.com/ClickHouse/ClickHouse/commits/master" + branch_name = "master" + if pr_number != 0: + branch_name = f"PR #{pr_number}" + branch_url = f"https://github.com/ClickHouse/ClickHouse/pull/{pr_number}" + commit_url = f"https://github.com/ClickHouse/ClickHouse/commit/{commit_sha}" + + task_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + + raw_log_url = additional_urls[0] + additional_urls.pop(0) + + html_report = create_test_html_report(check_name, test_results, raw_log_url, task_url, branch_url, branch_name, commit_url, additional_urls, True) + with open('report.html', 'w', encoding='utf-8') as f: + f.write(html_report) + + url = s3_client.upload_test_report_to_s3('report.html', s3_path_prefix + ".html") + logging.info("Search result in url %s", url) + return url + +def get_commit(gh, commit_sha): + repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) + commit = repo.get_commit(commit_sha) + return commit + +def process_results(result_folder, server_log_path): + test_results = [] + additional_files = [] + # Just upload all files from result_folder. + # If task provides processed results, then it's responsible for content of result_folder. + if os.path.exists(result_folder): + test_files = [f for f in os.listdir(result_folder) if os.path.isfile(os.path.join(result_folder, f))] + additional_files = [os.path.join(result_folder, f) for f in test_files] + + if os.path.exists(server_log_path): + server_log_files = [f for f in os.listdir(server_log_path) if os.path.isfile(os.path.join(server_log_path, f))] + additional_files = additional_files + [os.path.join(server_log_path, f) for f in server_log_files] + + status_path = os.path.join(result_folder, "check_status.tsv") + if not os.path.exists(status_path): + return "failure", "check_status.tsv doesn't exists", test_results, additional_files + + logging.info("Found check_status.tsv") + with open(status_path, 'r', encoding='utf-8') as status_file: + status = list(csv.reader(status_file, delimiter='\t')) + + if len(status) != 1 or len(status[0]) != 2: + return "error", "Invalid check_status.tsv", test_results, additional_files + state, description = status[0][0], status[0][1] + + results_path = os.path.join(result_folder, "test_results.tsv") + with open(results_path, 'r', encoding='utf-8') as results_file: + test_results = list(csv.reader(results_file, delimiter='\t')) + if len(test_results) == 0: + raise Exception("Empty results") + + return state, description, test_results, additional_files + + +if __name__ == "__main__": + logging.basicConfig(level=logging.INFO) + temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) + repo_path = os.getenv("REPO_COPY", os.path.abspath("../../")) + reports_path = os.getenv("REPORTS_PATH", "./reports") + + check_name = sys.argv[1] + build_number = int(sys.argv[2]) + + if not os.path.exists(temp_path): + os.makedirs(temp_path) + + with open(os.getenv('GITHUB_EVENT_PATH'), 'r', encoding='utf-8') as event_file: + event = json.load(event_file) + + pr_info = PRInfo(event) + + gh = Github(get_best_robot_token()) + + for root, _, files in os.walk(reports_path): + for f in files: + if f == 'changed_images.json': + images_path = os.path.join(root, 'changed_images.json') + break + + image_name = "clickhouse/stress-test" + + docker_image = image_name + if images_path and os.path.exists(images_path): + logging.info("Images file exists") + with open(images_path, 'r', encoding='utf-8') as images_fd: + images = json.load(images_fd) + logging.info("Got images %s", images) + if image_name in images: + docker_image += ':' + images[image_name] + else: + logging.info("Images file not found") + + for i in range(10): + try: + logging.info("Pulling image %s", docker_image) + subprocess.check_output(f"docker pull {docker_image}", stderr=subprocess.STDOUT, shell=True) + break + except Exception as ex: + time.sleep(i * 3) + logging.info("Got execption pulling docker %s", ex) + else: + raise Exception(f"Cannot pull dockerhub for image docker pull {docker_image}") + + build_config = get_build_config(build_number, repo_path) + build_config_str = build_config_to_string(build_config) + urls = get_build_urls(build_config_str, reports_path) + if not urls: + raise Exception("No build URLs found") + + packages_path = os.path.join(temp_path, "packages") + if not os.path.exists(packages_path): + os.makedirs(packages_path) + + server_log_path = os.path.join(temp_path, "server_log") + if not os.path.exists(server_log_path): + os.makedirs(server_log_path) + + result_path = os.path.join(temp_path, "result_path") + if not os.path.exists(result_path): + os.makedirs(result_path) + + run_log_path = os.path.join(result_path, "runlog.log") + + download_builds(packages_path, urls) + run_command = get_run_command(packages_path, result_path, server_log_path, docker_image) + logging.info("Going to run func tests: %s", run_command) + + with open(run_log_path, 'w', encoding='utf-8') as log: + with subprocess.Popen(run_command, shell=True, stderr=log, stdout=log) as process: + retcode = process.wait() + if retcode == 0: + logging.info("Run successfully") + else: + logging.info("Run failed") + + subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) + + s3_helper = S3Helper('https://s3.amazonaws.com') + state, description, test_results, additional_logs = process_results(result_path, server_log_path) + report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, run_log_path, additional_logs, check_name) + print(f"::notice ::Report url: {report_url}") + commit = get_commit(gh, pr_info.sha) + commit.create_status(context=check_name, description=description, state=state, target_url=report_url) diff --git a/tests/ci/worker/init_func_tester.sh b/tests/ci/worker/init_func_tester.sh new file mode 100644 index 00000000000..b117f11556d --- /dev/null +++ b/tests/ci/worker/init_func_tester.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash +set -euo pipefail + +echo "Running init script" +export DEBIAN_FRONTEND=noninteractive +export RUNNER_HOME=/home/ubuntu/actions-runner + +echo "Receiving token" +export RUNNER_TOKEN=`/usr/local/bin/aws ssm get-parameter --name github_runner_registration_token --with-decryption --output text --query Parameter.Value` +export RUNNER_URL="https://github.com/ClickHouse" +# Funny fact, but metadata service has fixed IP +export INSTANCE_ID=`curl -s http://169.254.169.254/latest/meta-data/instance-id` + +cd $RUNNER_HOME + +echo "Going to configure runner" +sudo -u ubuntu ./config.sh --url $RUNNER_URL --token $RUNNER_TOKEN --name $INSTANCE_ID --runnergroup Default --labels 'self-hosted,Linux,X64,func-tester' --work _work + +echo "Run" +sudo -u ubuntu ./run.sh diff --git a/tests/ci/worker/init_stress_tester.sh b/tests/ci/worker/init_stress_tester.sh new file mode 100644 index 00000000000..54ed944b274 --- /dev/null +++ b/tests/ci/worker/init_stress_tester.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash +set -euo pipefail + +echo "Running init script" +export DEBIAN_FRONTEND=noninteractive +export RUNNER_HOME=/home/ubuntu/actions-runner + +echo "Receiving token" +export RUNNER_TOKEN=`/usr/local/bin/aws ssm get-parameter --name github_runner_registration_token --with-decryption --output text --query Parameter.Value` +export RUNNER_URL="https://github.com/ClickHouse" +# Funny fact, but metadata service has fixed IP +export INSTANCE_ID=`curl -s http://169.254.169.254/latest/meta-data/instance-id` + +cd $RUNNER_HOME + +echo "Going to configure runner" +sudo -u ubuntu ./config.sh --url $RUNNER_URL --token $RUNNER_TOKEN --name $INSTANCE_ID --runnergroup Default --labels 'self-hosted,Linux,X64,stress-tester' --work _work + +echo "Run" +sudo -u ubuntu ./run.sh From ae7bc00dab9230df3d6032f9649f13d06cf7e37a Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Nov 2021 13:28:44 +0300 Subject: [PATCH 906/919] Fix step name --- .github/workflows/main.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index f1c0ea7cbac..f88075b1d71 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -217,7 +217,7 @@ jobs: path: ${{runner.temp}}/reports_dir - name: Check out repository code uses: actions/checkout@v2 - - name: Functional test + - name: Stress test env: TEMP_PATH: ${{runner.temp}}/stress_debug REPORTS_PATH: ${{runner.temp}}/reports_dir From dbfa17aabf4878f6d7e00a8e13334725fc2f7b28 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Nov 2021 13:32:56 +0300 Subject: [PATCH 907/919] Add correct S3 --- docker/test/stress/Dockerfile | 2 ++ docker/test/stress/run.sh | 2 +- tests/ci/stress_check.py | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docker/test/stress/Dockerfile b/docker/test/stress/Dockerfile index 3fe1b790d5a..e28d25c9485 100644 --- a/docker/test/stress/Dockerfile +++ b/docker/test/stress/Dockerfile @@ -26,4 +26,6 @@ COPY ./stress /stress COPY run.sh / ENV DATASETS="hits visits" +ENV S3_URL="https://clickhouse-datasets.s3.yandex.net" + CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index a4e2f797210..04845f2a4d1 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -112,7 +112,7 @@ configure start # shellcheck disable=SC2086 # No quotes because I want to split it into words. -/s3downloader --dataset-names $DATASETS +/s3downloader --url-prefix "$S3_URL" --dataset-names $DATASETS chmod 777 -R /var/lib/clickhouse clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index ebac2cd86aa..8fb099af132 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -100,7 +100,7 @@ def build_config_to_string(build_config): def get_run_command(build_path, result_folder, server_log_folder, image): - cmd = "docker run " + \ + cmd = "docker run -e S3_URL='https://clickhouse-datasets.s3.amazonaws.com' " + \ f"--volume={build_path}:/package_folder " \ f"--volume={result_folder}:/test_output " \ f"--volume={server_log_folder}:/var/log/clickhouse-server {image}" From 8a390448783897dab666682e95525619ac332d67 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Nov 2021 14:11:59 +0300 Subject: [PATCH 908/919] Fix stress tester --- .github/workflows/main.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index f88075b1d71..0a4beb50708 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -209,7 +209,7 @@ jobs: sudo rm -fr $TEMP_PATH StressTestDebug: needs: [BuilderDebDebug] - runs-on: [self-hosted, func-tester] + runs-on: [self-hosted, stress-tester] steps: - name: Download json reports uses: actions/download-artifact@v2 From 4e97d666c28bc9e5908a667392b62469829d1f10 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 1 Nov 2021 14:14:28 +0300 Subject: [PATCH 909/919] Update settings.md --- docs/en/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 9df1951972f..8ea967eac9c 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1406,7 +1406,7 @@ Possible values: - 0 — Aggregation is done without JIT compilation. - 1 — Aggregation is done using JIT compilation. -Default value: `0`. +Default value: `1`. **See Also** @@ -1421,7 +1421,7 @@ Possible values: - Positive integer. - 0 — Identical aggregate expressions are always JIT-compiled. -Default value: `0`. +Default value: `3`. ## output_format_json_quote_64bit_integers {#session_settings-output_format_json_quote_64bit_integers} From d8249d88b2e20c52dff1f06d892e243d5ac8606c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 1 Nov 2021 14:15:01 +0300 Subject: [PATCH 910/919] Update settings.md --- docs/ru/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index d5b102e53ab..37d6c2325ac 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1370,7 +1370,7 @@ load_balancing = round_robin - 0 — агрегатные функции не компилируются в нативный код. - 1 — агрегатные функции компилируются в нативный код в процессе выполнения запроса. -Значение по умолчанию: `0`. +Значение по умолчанию: `1`. **См. также** @@ -1385,7 +1385,7 @@ load_balancing = round_robin - Целое положительное число. - 0 — агрегатные функциии всегда компилируются в ходе выполнения запроса. -Значение по умолчанию: `0`. +Значение по умолчанию: `3`. ## input_format_skip_unknown_fields {#input-format-skip-unknown-fields} From 259c8061707c3fb56d5c377aa2d412ee8d6b24a0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Nov 2021 16:27:55 +0300 Subject: [PATCH 911/919] Don't check own run file --- tests/ci/stress_check.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 8fb099af132..a7730285025 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -148,7 +148,7 @@ def get_commit(gh, commit_sha): commit = repo.get_commit(commit_sha) return commit -def process_results(result_folder, server_log_path): +def process_results(result_folder, server_log_path, run_log_path): test_results = [] additional_files = [] # Just upload all files from result_folder. @@ -161,6 +161,8 @@ def process_results(result_folder, server_log_path): server_log_files = [f for f in os.listdir(server_log_path) if os.path.isfile(os.path.join(server_log_path, f))] additional_files = additional_files + [os.path.join(server_log_path, f) for f in server_log_files] + additional_files.append(run_log_path) + status_path = os.path.join(result_folder, "check_status.tsv") if not os.path.exists(status_path): return "failure", "check_status.tsv doesn't exists", test_results, additional_files @@ -249,7 +251,7 @@ if __name__ == "__main__": if not os.path.exists(result_path): os.makedirs(result_path) - run_log_path = os.path.join(result_path, "runlog.log") + run_log_path = os.path.join(temp_path, "runlog.log") download_builds(packages_path, urls) run_command = get_run_command(packages_path, result_path, server_log_path, docker_image) @@ -266,7 +268,7 @@ if __name__ == "__main__": subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) s3_helper = S3Helper('https://s3.amazonaws.com') - state, description, test_results, additional_logs = process_results(result_path, server_log_path) + state, description, test_results, additional_logs = process_results(result_path, server_log_path, run_log_path) report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, run_log_path, additional_logs, check_name) print(f"::notice ::Report url: {report_url}") commit = get_commit(gh, pr_info.sha) From 0a83c391c13a22149000d4fd2e7612697fcc734d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 1 Nov 2021 16:56:09 +0300 Subject: [PATCH 912/919] Update 02026_storage_filelog_largefile.sh --- tests/queries/0_stateless/02026_storage_filelog_largefile.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02026_storage_filelog_largefile.sh b/tests/queries/0_stateless/02026_storage_filelog_largefile.sh index 6babcc1e4f1..acd1c464334 100755 --- a/tests/queries/0_stateless/02026_storage_filelog_largefile.sh +++ b/tests/queries/0_stateless/02026_storage_filelog_largefile.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long +# Tags: long, no-parallel set -eu From 279e1eb6e91bcbd09fcd1045ad0fd975cdde7c00 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Mon, 1 Nov 2021 18:58:08 +0300 Subject: [PATCH 913/919] Update docs/en/sql-reference/statements/create/database.md Co-authored-by: tavplubix --- docs/en/sql-reference/statements/create/database.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/create/database.md b/docs/en/sql-reference/statements/create/database.md index 5a6d360f1ee..44c9153175c 100644 --- a/docs/en/sql-reference/statements/create/database.md +++ b/docs/en/sql-reference/statements/create/database.md @@ -26,7 +26,7 @@ ClickHouse creates the `db_name` database on all the servers of a specified clus ### ENGINE {#engine} -[MySQL](../../../engines/database-engines/mysql.md) allows you to retrieve data from the remote MySQL server. By default, ClickHouse uses its own [database engine](../../../engines/database-engines/index.md). There is also a [lazy](../../../engines/database-engines/lazy.md) engine. +By default, ClickHouse uses its own [Atomic](../../../engines/database-engines/atomic.md) database engine. There are also [Lazy](../../../engines/database-engines/lazy.md), MySQL, PostgresSQL, MaterializedMySQL, MaterializedPostgreSQL, Dictionary, Replicated . ### COMMENT {#comment} From 3ed7f8f0b30a681ee11fbf2bd093c1c0d98dc8c8 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 31 Oct 2021 11:51:20 +0300 Subject: [PATCH 914/919] Move access-rights' source files needed for parser to a separate target. --- src/Access/AccessEntityIO.cpp | 40 +++++++------- src/Access/AccessRights.h | 2 +- src/Access/{ => Common}/AccessFlags.h | 2 +- .../{ => Common}/AccessRightsElement.cpp | 2 +- src/Access/{ => Common}/AccessRightsElement.h | 2 +- src/Access/{ => Common}/AccessType.h | 0 .../{ => Common}/AllowedClientHosts.cpp | 2 +- src/Access/{ => Common}/AllowedClientHosts.h | 0 src/Access/{ => Common}/Authentication.cpp | 2 +- src/Access/{ => Common}/Authentication.h | 0 src/Access/Common/CMakeLists.txt | 5 ++ src/Access/DiskAccessStorage.cpp | 18 +++---- src/Access/RolesOrUsersSet.cpp | 4 +- src/Access/SettingsProfileElement.cpp | 2 +- src/Access/User.h | 4 +- src/Bridge/XDBCBridgeHelper.h | 2 +- src/CMakeLists.txt | 2 + src/Functions/FunctionsExternalDictionaries.h | 2 +- src/Functions/addressToLine.cpp | 2 +- src/Functions/addressToSymbol.cpp | 2 +- src/Functions/demange.cpp | 2 +- .../InterpreterCreateQuotaQuery.cpp | 10 ++-- .../InterpreterCreateQuotaQuery.h | 0 .../InterpreterCreateRoleQuery.cpp | 8 +-- .../{ => Access}/InterpreterCreateRoleQuery.h | 0 .../InterpreterCreateRowPolicyQuery.cpp | 12 ++--- .../InterpreterCreateRowPolicyQuery.h | 0 .../InterpreterCreateSettingsProfileQuery.cpp | 12 ++--- .../InterpreterCreateSettingsProfileQuery.h | 0 .../InterpreterCreateUserQuery.cpp | 16 +++--- .../{ => Access}/InterpreterCreateUserQuery.h | 0 .../InterpreterDropAccessEntityQuery.cpp | 12 ++--- .../InterpreterDropAccessEntityQuery.h | 0 .../{ => Access}/InterpreterGrantQuery.cpp | 14 ++--- .../{ => Access}/InterpreterGrantQuery.h | 0 .../{ => Access}/InterpreterSetRoleQuery.cpp | 8 +-- .../{ => Access}/InterpreterSetRoleQuery.h | 0 .../InterpreterShowAccessEntitiesQuery.cpp | 6 +-- .../InterpreterShowAccessEntitiesQuery.h | 0 .../InterpreterShowAccessQuery.cpp | 8 +-- .../{ => Access}/InterpreterShowAccessQuery.h | 0 ...InterpreterShowCreateAccessEntityQuery.cpp | 28 +++++----- .../InterpreterShowCreateAccessEntityQuery.h | 0 .../InterpreterShowGrantsQuery.cpp | 18 +++---- .../{ => Access}/InterpreterShowGrantsQuery.h | 0 .../InterpreterShowPrivilegesQuery.cpp | 2 +- .../InterpreterShowPrivilegesQuery.h | 0 src/Interpreters/AsynchronousInsertQueue.cpp | 2 +- src/Interpreters/InterpreterAlterQuery.cpp | 2 +- src/Interpreters/InterpreterCheckQuery.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.h | 2 +- src/Interpreters/InterpreterDescribeQuery.cpp | 2 +- src/Interpreters/InterpreterDropQuery.cpp | 2 +- src/Interpreters/InterpreterExistsQuery.cpp | 2 +- src/Interpreters/InterpreterFactory.cpp | 54 ++++++++++--------- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- src/Interpreters/InterpreterOptimizeQuery.cpp | 2 +- src/Interpreters/InterpreterRenameQuery.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- .../InterpreterShowCreateQuery.cpp | 2 +- src/Interpreters/InterpreterSystemQuery.cpp | 2 +- src/Interpreters/InterpreterUseQuery.cpp | 2 +- src/Interpreters/InterpreterWatchQuery.cpp | 2 +- src/Interpreters/Session.h | 2 +- src/Interpreters/SessionLog.h | 2 +- src/Interpreters/executeDDLQueryOnCluster.cpp | 2 +- .../{ => Access}/ASTCreateQuotaQuery.cpp | 4 +- .../{ => Access}/ASTCreateQuotaQuery.h | 0 .../{ => Access}/ASTCreateRoleQuery.cpp | 4 +- src/Parsers/{ => Access}/ASTCreateRoleQuery.h | 0 .../{ => Access}/ASTCreateRowPolicyQuery.cpp | 8 +-- .../{ => Access}/ASTCreateRowPolicyQuery.h | 0 .../ASTCreateSettingsProfileQuery.cpp | 6 +-- .../ASTCreateSettingsProfileQuery.h | 0 .../{ => Access}/ASTCreateUserQuery.cpp | 8 +-- src/Parsers/{ => Access}/ASTCreateUserQuery.h | 4 +- .../{ => Access}/ASTDropAccessEntityQuery.cpp | 4 +- .../{ => Access}/ASTDropAccessEntityQuery.h | 0 src/Parsers/{ => Access}/ASTGrantQuery.cpp | 4 +- src/Parsers/{ => Access}/ASTGrantQuery.h | 2 +- .../{ => Access}/ASTRolesOrUsersSet.cpp | 2 +- src/Parsers/{ => Access}/ASTRolesOrUsersSet.h | 0 src/Parsers/{ => Access}/ASTRowPolicyName.cpp | 2 +- src/Parsers/{ => Access}/ASTRowPolicyName.h | 0 src/Parsers/{ => Access}/ASTSetRoleQuery.cpp | 4 +- src/Parsers/{ => Access}/ASTSetRoleQuery.h | 0 .../ASTSettingsProfileElement.cpp | 2 +- .../{ => Access}/ASTSettingsProfileElement.h | 0 .../ASTShowAccessEntitiesQuery.cpp | 2 +- .../{ => Access}/ASTShowAccessEntitiesQuery.h | 0 src/Parsers/{ => Access}/ASTShowAccessQuery.h | 0 .../ASTShowCreateAccessEntityQuery.cpp | 4 +- .../ASTShowCreateAccessEntityQuery.h | 0 .../{ => Access}/ASTShowGrantsQuery.cpp | 4 +- src/Parsers/{ => Access}/ASTShowGrantsQuery.h | 0 .../{ => Access}/ASTShowPrivilegesQuery.h | 0 .../{ => Access}/ASTUserNameWithHost.cpp | 2 +- .../{ => Access}/ASTUserNameWithHost.h | 0 .../{ => Access}/ParserCreateQuotaQuery.cpp | 16 +++--- .../{ => Access}/ParserCreateQuotaQuery.h | 0 .../{ => Access}/ParserCreateRoleQuery.cpp | 12 ++--- .../{ => Access}/ParserCreateRoleQuery.h | 0 .../ParserCreateRowPolicyQuery.cpp | 20 +++---- .../{ => Access}/ParserCreateRowPolicyQuery.h | 0 .../ParserCreateSettingsProfileQuery.cpp | 14 ++--- .../ParserCreateSettingsProfileQuery.h | 0 .../{ => Access}/ParserCreateUserQuery.cpp | 22 ++++---- .../{ => Access}/ParserCreateUserQuery.h | 0 .../ParserDropAccessEntityQuery.cpp | 10 ++-- .../ParserDropAccessEntityQuery.h | 0 src/Parsers/{ => Access}/ParserGrantQuery.cpp | 8 +-- src/Parsers/{ => Access}/ParserGrantQuery.h | 0 .../{ => Access}/ParserRolesOrUsersSet.cpp | 8 +-- .../{ => Access}/ParserRolesOrUsersSet.h | 0 .../{ => Access}/ParserRowPolicyName.cpp | 8 +-- .../{ => Access}/ParserRowPolicyName.h | 0 .../{ => Access}/ParserSetRoleQuery.cpp | 8 +-- src/Parsers/{ => Access}/ParserSetRoleQuery.h | 0 .../ParserSettingsProfileElement.cpp | 8 +-- .../ParserSettingsProfileElement.h | 0 .../ParserShowAccessEntitiesQuery.cpp | 4 +- .../ParserShowAccessEntitiesQuery.h | 0 .../{ => Access}/ParserShowAccessQuery.h | 2 +- .../ParserShowCreateAccessEntityQuery.cpp | 10 ++-- .../ParserShowCreateAccessEntityQuery.h | 0 .../{ => Access}/ParserShowGrantsQuery.cpp | 10 ++-- .../{ => Access}/ParserShowGrantsQuery.h | 0 .../ParserShowPrivilegesQuery.cpp | 4 +- .../{ => Access}/ParserShowPrivilegesQuery.h | 0 .../{ => Access}/ParserUserNameWithHost.cpp | 4 +- .../{ => Access}/ParserUserNameWithHost.h | 0 src/Parsers/{ => Access}/parseUserName.cpp | 6 +-- src/Parsers/{ => Access}/parseUserName.h | 0 src/Parsers/CMakeLists.txt | 4 +- src/Parsers/ParserQuery.cpp | 17 +++--- src/Parsers/ParserQueryWithOutput.cpp | 10 ++-- src/Server/HTTPHandler.cpp | 2 +- src/Storages/LiveView/StorageLiveView.cpp | 2 +- src/Storages/StorageFactory.h | 2 +- src/Storages/StorageMaterializedView.cpp | 2 +- src/Storages/System/StorageSystemGrants.cpp | 2 +- .../System/StorageSystemPrivileges.cpp | 16 +++--- .../System/StorageSystemQuotaLimits.cpp | 2 +- .../System/StorageSystemQuotaUsage.cpp | 2 +- src/Storages/System/StorageSystemQuotas.cpp | 14 ++--- .../System/StorageSystemQuotasUsage.cpp | 2 +- src/Storages/System/StorageSystemRoles.cpp | 2 +- .../System/StorageSystemRowPolicies.cpp | 16 +++--- .../System/StorageSystemSettingsProfiles.cpp | 14 ++--- src/Storages/System/StorageSystemUsers.cpp | 8 +-- src/TableFunctions/ITableFunction.cpp | 2 +- src/TableFunctions/TableFunctionFile.cpp | 2 +- src/TableFunctions/TableFunctionURL.cpp | 2 +- 154 files changed, 359 insertions(+), 347 deletions(-) rename src/Access/{ => Common}/AccessFlags.h (99%) rename src/Access/{ => Common}/AccessRightsElement.cpp (99%) rename src/Access/{ => Common}/AccessRightsElement.h (99%) rename src/Access/{ => Common}/AccessType.h (100%) rename src/Access/{ => Common}/AllowedClientHosts.cpp (99%) rename src/Access/{ => Common}/AllowedClientHosts.h (100%) rename src/Access/{ => Common}/Authentication.cpp (99%) rename src/Access/{ => Common}/Authentication.h (100%) create mode 100644 src/Access/Common/CMakeLists.txt rename src/Interpreters/{ => Access}/InterpreterCreateQuotaQuery.cpp (95%) rename src/Interpreters/{ => Access}/InterpreterCreateQuotaQuery.h (100%) rename src/Interpreters/{ => Access}/InterpreterCreateRoleQuery.cpp (96%) rename src/Interpreters/{ => Access}/InterpreterCreateRoleQuery.h (100%) rename src/Interpreters/{ => Access}/InterpreterCreateRowPolicyQuery.cpp (93%) rename src/Interpreters/{ => Access}/InterpreterCreateRowPolicyQuery.h (100%) rename src/Interpreters/{ => Access}/InterpreterCreateSettingsProfileQuery.cpp (94%) rename src/Interpreters/{ => Access}/InterpreterCreateSettingsProfileQuery.h (100%) rename src/Interpreters/{ => Access}/InterpreterCreateUserQuery.cpp (95%) rename src/Interpreters/{ => Access}/InterpreterCreateUserQuery.h (100%) rename src/Interpreters/{ => Access}/InterpreterDropAccessEntityQuery.cpp (91%) rename src/Interpreters/{ => Access}/InterpreterDropAccessEntityQuery.h (100%) rename src/Interpreters/{ => Access}/InterpreterGrantQuery.cpp (99%) rename src/Interpreters/{ => Access}/InterpreterGrantQuery.h (100%) rename src/Interpreters/{ => Access}/InterpreterSetRoleQuery.cpp (94%) rename src/Interpreters/{ => Access}/InterpreterSetRoleQuery.h (100%) rename src/Interpreters/{ => Access}/InterpreterShowAccessEntitiesQuery.cpp (96%) rename src/Interpreters/{ => Access}/InterpreterShowAccessEntitiesQuery.h (100%) rename src/Interpreters/{ => Access}/InterpreterShowAccessQuery.cpp (91%) rename src/Interpreters/{ => Access}/InterpreterShowAccessQuery.h (100%) rename src/Interpreters/{ => Access}/InterpreterShowCreateAccessEntityQuery.cpp (95%) rename src/Interpreters/{ => Access}/InterpreterShowCreateAccessEntityQuery.h (100%) rename src/Interpreters/{ => Access}/InterpreterShowGrantsQuery.cpp (96%) rename src/Interpreters/{ => Access}/InterpreterShowGrantsQuery.h (100%) rename src/Interpreters/{ => Access}/InterpreterShowPrivilegesQuery.cpp (84%) rename src/Interpreters/{ => Access}/InterpreterShowPrivilegesQuery.h (100%) rename src/Parsers/{ => Access}/ASTCreateQuotaQuery.cpp (98%) rename src/Parsers/{ => Access}/ASTCreateQuotaQuery.h (100%) rename src/Parsers/{ => Access}/ASTCreateRoleQuery.cpp (95%) rename src/Parsers/{ => Access}/ASTCreateRoleQuery.h (100%) rename src/Parsers/{ => Access}/ASTCreateRowPolicyQuery.cpp (97%) rename src/Parsers/{ => Access}/ASTCreateRowPolicyQuery.h (100%) rename src/Parsers/{ => Access}/ASTCreateSettingsProfileQuery.cpp (94%) rename src/Parsers/{ => Access}/ASTCreateSettingsProfileQuery.h (100%) rename src/Parsers/{ => Access}/ASTCreateUserQuery.cpp (98%) rename src/Parsers/{ => Access}/ASTCreateUserQuery.h (96%) rename src/Parsers/{ => Access}/ASTDropAccessEntityQuery.cpp (93%) rename src/Parsers/{ => Access}/ASTDropAccessEntityQuery.h (100%) rename src/Parsers/{ => Access}/ASTGrantQuery.cpp (98%) rename src/Parsers/{ => Access}/ASTGrantQuery.h (97%) rename src/Parsers/{ => Access}/ASTRolesOrUsersSet.cpp (98%) rename src/Parsers/{ => Access}/ASTRolesOrUsersSet.h (100%) rename src/Parsers/{ => Access}/ASTRowPolicyName.cpp (98%) rename src/Parsers/{ => Access}/ASTRowPolicyName.h (100%) rename src/Parsers/{ => Access}/ASTSetRoleQuery.cpp (91%) rename src/Parsers/{ => Access}/ASTSetRoleQuery.h (100%) rename src/Parsers/{ => Access}/ASTSettingsProfileElement.cpp (98%) rename src/Parsers/{ => Access}/ASTSettingsProfileElement.h (100%) rename src/Parsers/{ => Access}/ASTShowAccessEntitiesQuery.cpp (96%) rename src/Parsers/{ => Access}/ASTShowAccessEntitiesQuery.h (100%) rename src/Parsers/{ => Access}/ASTShowAccessQuery.h (100%) rename src/Parsers/{ => Access}/ASTShowCreateAccessEntityQuery.cpp (96%) rename src/Parsers/{ => Access}/ASTShowCreateAccessEntityQuery.h (100%) rename src/Parsers/{ => Access}/ASTShowGrantsQuery.cpp (90%) rename src/Parsers/{ => Access}/ASTShowGrantsQuery.h (100%) rename src/Parsers/{ => Access}/ASTShowPrivilegesQuery.h (100%) rename src/Parsers/{ => Access}/ASTUserNameWithHost.cpp (97%) rename src/Parsers/{ => Access}/ASTUserNameWithHost.h (100%) rename src/Parsers/{ => Access}/ParserCreateQuotaQuery.cpp (98%) rename src/Parsers/{ => Access}/ParserCreateQuotaQuery.h (100%) rename src/Parsers/{ => Access}/ParserCreateRoleQuery.cpp (93%) rename src/Parsers/{ => Access}/ParserCreateRoleQuery.h (100%) rename src/Parsers/{ => Access}/ParserCreateRowPolicyQuery.cpp (96%) rename src/Parsers/{ => Access}/ParserCreateRowPolicyQuery.h (100%) rename src/Parsers/{ => Access}/ParserCreateSettingsProfileQuery.cpp (93%) rename src/Parsers/{ => Access}/ParserCreateSettingsProfileQuery.h (100%) rename src/Parsers/{ => Access}/ParserCreateUserQuery.cpp (97%) rename src/Parsers/{ => Access}/ParserCreateUserQuery.h (100%) rename src/Parsers/{ => Access}/ParserDropAccessEntityQuery.cpp (91%) rename src/Parsers/{ => Access}/ParserDropAccessEntityQuery.h (100%) rename src/Parsers/{ => Access}/ParserGrantQuery.cpp (98%) rename src/Parsers/{ => Access}/ParserGrantQuery.h (100%) rename src/Parsers/{ => Access}/ParserRolesOrUsersSet.cpp (96%) rename src/Parsers/{ => Access}/ParserRolesOrUsersSet.h (100%) rename src/Parsers/{ => Access}/ParserRowPolicyName.cpp (98%) rename src/Parsers/{ => Access}/ParserRowPolicyName.h (100%) rename src/Parsers/{ => Access}/ParserSetRoleQuery.cpp (92%) rename src/Parsers/{ => Access}/ParserSetRoleQuery.h (100%) rename src/Parsers/{ => Access}/ParserSettingsProfileElement.cpp (98%) rename src/Parsers/{ => Access}/ParserSettingsProfileElement.h (100%) rename src/Parsers/{ => Access}/ParserShowAccessEntitiesQuery.cpp (96%) rename src/Parsers/{ => Access}/ParserShowAccessEntitiesQuery.h (100%) rename src/Parsers/{ => Access}/ParserShowAccessQuery.h (92%) rename src/Parsers/{ => Access}/ParserShowCreateAccessEntityQuery.cpp (95%) rename src/Parsers/{ => Access}/ParserShowCreateAccessEntityQuery.h (100%) rename src/Parsers/{ => Access}/ParserShowGrantsQuery.cpp (79%) rename src/Parsers/{ => Access}/ParserShowGrantsQuery.h (100%) rename src/Parsers/{ => Access}/ParserShowPrivilegesQuery.cpp (76%) rename src/Parsers/{ => Access}/ParserShowPrivilegesQuery.h (100%) rename src/Parsers/{ => Access}/ParserUserNameWithHost.cpp (95%) rename src/Parsers/{ => Access}/ParserUserNameWithHost.h (100%) rename src/Parsers/{ => Access}/parseUserName.cpp (88%) rename src/Parsers/{ => Access}/parseUserName.h (100%) diff --git a/src/Access/AccessEntityIO.cpp b/src/Access/AccessEntityIO.cpp index 2160f3e9db8..199b3b22efc 100644 --- a/src/Access/AccessEntityIO.cpp +++ b/src/Access/AccessEntityIO.cpp @@ -7,26 +7,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 +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include diff --git a/src/Access/AccessRights.h b/src/Access/AccessRights.h index 2657b66d824..32a4462d212 100644 --- a/src/Access/AccessRights.h +++ b/src/Access/AccessRights.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include diff --git a/src/Access/AccessFlags.h b/src/Access/Common/AccessFlags.h similarity index 99% rename from src/Access/AccessFlags.h rename to src/Access/Common/AccessFlags.h index b107248e02b..87dc17522ab 100644 --- a/src/Access/AccessFlags.h +++ b/src/Access/Common/AccessFlags.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include diff --git a/src/Access/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp similarity index 99% rename from src/Access/AccessRightsElement.cpp rename to src/Access/Common/AccessRightsElement.cpp index 823019ffebd..5d62de69e40 100644 --- a/src/Access/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Access/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h similarity index 99% rename from src/Access/AccessRightsElement.h rename to src/Access/Common/AccessRightsElement.h index c46a4b54e6e..012c8aa0d12 100644 --- a/src/Access/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB diff --git a/src/Access/AccessType.h b/src/Access/Common/AccessType.h similarity index 100% rename from src/Access/AccessType.h rename to src/Access/Common/AccessType.h diff --git a/src/Access/AllowedClientHosts.cpp b/src/Access/Common/AllowedClientHosts.cpp similarity index 99% rename from src/Access/AllowedClientHosts.cpp rename to src/Access/Common/AllowedClientHosts.cpp index f306b1386dd..c5e87844376 100644 --- a/src/Access/AllowedClientHosts.cpp +++ b/src/Access/Common/AllowedClientHosts.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/Access/AllowedClientHosts.h b/src/Access/Common/AllowedClientHosts.h similarity index 100% rename from src/Access/AllowedClientHosts.h rename to src/Access/Common/AllowedClientHosts.h diff --git a/src/Access/Authentication.cpp b/src/Access/Common/Authentication.cpp similarity index 99% rename from src/Access/Authentication.cpp rename to src/Access/Common/Authentication.cpp index f07bdb10578..886b58202fd 100644 --- a/src/Access/Authentication.cpp +++ b/src/Access/Common/Authentication.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/Access/Authentication.h b/src/Access/Common/Authentication.h similarity index 100% rename from src/Access/Authentication.h rename to src/Access/Common/Authentication.h diff --git a/src/Access/Common/CMakeLists.txt b/src/Access/Common/CMakeLists.txt new file mode 100644 index 00000000000..6a7682ec4bd --- /dev/null +++ b/src/Access/Common/CMakeLists.txt @@ -0,0 +1,5 @@ +include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") + +add_headers_and_sources(clickhouse_common_access .) +add_library(clickhouse_common_access ${clickhouse_common_access_headers} ${clickhouse_common_access_sources}) +target_link_libraries(clickhouse_common_access PUBLIC clickhouse_common_io) diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index e3d773f4d91..247bcc1ee89 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -1,19 +1,19 @@ #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 diff --git a/src/Access/RolesOrUsersSet.cpp b/src/Access/RolesOrUsersSet.cpp index ebd4f0f7a40..97acd5c1a76 100644 --- a/src/Access/RolesOrUsersSet.cpp +++ b/src/Access/RolesOrUsersSet.cpp @@ -1,9 +1,9 @@ #include +#include +#include #include #include #include -#include -#include #include #include #include diff --git a/src/Access/SettingsProfileElement.cpp b/src/Access/SettingsProfileElement.cpp index b42bcd1c279..a5d6dbf60f7 100644 --- a/src/Access/SettingsProfileElement.cpp +++ b/src/Access/SettingsProfileElement.cpp @@ -2,11 +2,11 @@ #include #include #include -#include #include #include #include #include +#include #include diff --git a/src/Access/User.h b/src/Access/User.h index 6b61d5afdea..4bde967ba2c 100644 --- a/src/Access/User.h +++ b/src/Access/User.h @@ -2,8 +2,8 @@ #include #include -#include -#include +#include +#include #include #include #include diff --git a/src/Bridge/XDBCBridgeHelper.h b/src/Bridge/XDBCBridgeHelper.h index 80d9610f837..d321c1f23de 100644 --- a/src/Bridge/XDBCBridgeHelper.h +++ b/src/Bridge/XDBCBridgeHelper.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 87e6cc86d94..0a7c3123f9d 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -166,6 +166,7 @@ endif() target_link_libraries (clickhouse_common_io PRIVATE jemalloc) +add_subdirectory(Access/Common) add_subdirectory(Common/ZooKeeper) add_subdirectory(Common/Config) @@ -197,6 +198,7 @@ add_object_library(clickhouse_databases Databases) add_object_library(clickhouse_databases_mysql Databases/MySQL) add_object_library(clickhouse_disks Disks) add_object_library(clickhouse_interpreters Interpreters) +add_object_library(clickhouse_interpreters_access Interpreters/Access) add_object_library(clickhouse_interpreters_mysql Interpreters/MySQL) add_object_library(clickhouse_interpreters_clusterproxy Interpreters/ClusterProxy) add_object_library(clickhouse_interpreters_jit Interpreters/JIT) diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 7697d86dc34..7dc47b54bea 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -21,7 +21,7 @@ #include #include -#include +#include #include #include diff --git a/src/Functions/addressToLine.cpp b/src/Functions/addressToLine.cpp index d513a8767dc..6ef1800d913 100644 --- a/src/Functions/addressToLine.cpp +++ b/src/Functions/addressToLine.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Functions/addressToSymbol.cpp b/src/Functions/addressToSymbol.cpp index 1561e0ee506..0fd25503ec7 100644 --- a/src/Functions/addressToSymbol.cpp +++ b/src/Functions/addressToSymbol.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Functions/demange.cpp b/src/Functions/demange.cpp index 0f50eb5e141..ecf6661d20d 100644 --- a/src/Functions/demange.cpp +++ b/src/Functions/demange.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include diff --git a/src/Interpreters/InterpreterCreateQuotaQuery.cpp b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp similarity index 95% rename from src/Interpreters/InterpreterCreateQuotaQuery.cpp rename to src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp index b4f61e43186..92b574c0036 100644 --- a/src/Interpreters/InterpreterCreateQuotaQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp @@ -1,10 +1,10 @@ -#include -#include -#include +#include +#include +#include +#include +#include #include #include -#include -#include #include #include #include diff --git a/src/Interpreters/InterpreterCreateQuotaQuery.h b/src/Interpreters/Access/InterpreterCreateQuotaQuery.h similarity index 100% rename from src/Interpreters/InterpreterCreateQuotaQuery.h rename to src/Interpreters/Access/InterpreterCreateQuotaQuery.h diff --git a/src/Interpreters/InterpreterCreateRoleQuery.cpp b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp similarity index 96% rename from src/Interpreters/InterpreterCreateRoleQuery.cpp rename to src/Interpreters/Access/InterpreterCreateRoleQuery.cpp index b9debc259be..62b44449726 100644 --- a/src/Interpreters/InterpreterCreateRoleQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp @@ -1,9 +1,9 @@ -#include -#include -#include -#include +#include +#include #include #include +#include +#include namespace DB diff --git a/src/Interpreters/InterpreterCreateRoleQuery.h b/src/Interpreters/Access/InterpreterCreateRoleQuery.h similarity index 100% rename from src/Interpreters/InterpreterCreateRoleQuery.h rename to src/Interpreters/Access/InterpreterCreateRoleQuery.h diff --git a/src/Interpreters/InterpreterCreateRowPolicyQuery.cpp b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp similarity index 93% rename from src/Interpreters/InterpreterCreateRowPolicyQuery.cpp rename to src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp index 5e4b9b30e66..b07ac0a6a10 100644 --- a/src/Interpreters/InterpreterCreateRowPolicyQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp @@ -1,12 +1,12 @@ -#include -#include -#include -#include +#include +#include +#include +#include #include +#include +#include #include #include -#include -#include #include diff --git a/src/Interpreters/InterpreterCreateRowPolicyQuery.h b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.h similarity index 100% rename from src/Interpreters/InterpreterCreateRowPolicyQuery.h rename to src/Interpreters/Access/InterpreterCreateRowPolicyQuery.h diff --git a/src/Interpreters/InterpreterCreateSettingsProfileQuery.cpp b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp similarity index 94% rename from src/Interpreters/InterpreterCreateSettingsProfileQuery.cpp rename to src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp index fb5fb258b10..fa83c6cff93 100644 --- a/src/Interpreters/InterpreterCreateSettingsProfileQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp @@ -1,11 +1,11 @@ -#include -#include -#include -#include -#include +#include +#include +#include #include #include -#include +#include +#include +#include namespace DB diff --git a/src/Interpreters/InterpreterCreateSettingsProfileQuery.h b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.h similarity index 100% rename from src/Interpreters/InterpreterCreateSettingsProfileQuery.h rename to src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.h diff --git a/src/Interpreters/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp similarity index 95% rename from src/Interpreters/InterpreterCreateUserQuery.cpp rename to src/Interpreters/Access/InterpreterCreateUserQuery.cpp index 6f963a3b338..e213ee59256 100644 --- a/src/Interpreters/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -1,14 +1,14 @@ -#include -#include -#include -#include -#include -#include -#include +#include +#include +#include +#include #include #include -#include #include +#include +#include +#include +#include #include diff --git a/src/Interpreters/InterpreterCreateUserQuery.h b/src/Interpreters/Access/InterpreterCreateUserQuery.h similarity index 100% rename from src/Interpreters/InterpreterCreateUserQuery.h rename to src/Interpreters/Access/InterpreterCreateUserQuery.h diff --git a/src/Interpreters/InterpreterDropAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp similarity index 91% rename from src/Interpreters/InterpreterDropAccessEntityQuery.cpp rename to src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp index a9b8db6d74e..04937aec46a 100644 --- a/src/Interpreters/InterpreterDropAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp @@ -1,15 +1,15 @@ -#include -#include -#include -#include -#include +#include +#include +#include #include -#include +#include #include #include #include #include #include +#include +#include namespace DB diff --git a/src/Interpreters/InterpreterDropAccessEntityQuery.h b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.h similarity index 100% rename from src/Interpreters/InterpreterDropAccessEntityQuery.h rename to src/Interpreters/Access/InterpreterDropAccessEntityQuery.h diff --git a/src/Interpreters/InterpreterGrantQuery.cpp b/src/Interpreters/Access/InterpreterGrantQuery.cpp similarity index 99% rename from src/Interpreters/InterpreterGrantQuery.cpp rename to src/Interpreters/Access/InterpreterGrantQuery.cpp index 506ab8a3387..f748f9ac02e 100644 --- a/src/Interpreters/InterpreterGrantQuery.cpp +++ b/src/Interpreters/Access/InterpreterGrantQuery.cpp @@ -1,14 +1,14 @@ -#include -#include -#include -#include -#include -#include +#include +#include +#include #include #include +#include #include #include -#include +#include +#include +#include #include #include diff --git a/src/Interpreters/InterpreterGrantQuery.h b/src/Interpreters/Access/InterpreterGrantQuery.h similarity index 100% rename from src/Interpreters/InterpreterGrantQuery.h rename to src/Interpreters/Access/InterpreterGrantQuery.h diff --git a/src/Interpreters/InterpreterSetRoleQuery.cpp b/src/Interpreters/Access/InterpreterSetRoleQuery.cpp similarity index 94% rename from src/Interpreters/InterpreterSetRoleQuery.cpp rename to src/Interpreters/Access/InterpreterSetRoleQuery.cpp index 057ccd447ef..82b100e9d48 100644 --- a/src/Interpreters/InterpreterSetRoleQuery.cpp +++ b/src/Interpreters/Access/InterpreterSetRoleQuery.cpp @@ -1,10 +1,10 @@ -#include -#include -#include -#include +#include +#include +#include #include #include #include +#include namespace DB diff --git a/src/Interpreters/InterpreterSetRoleQuery.h b/src/Interpreters/Access/InterpreterSetRoleQuery.h similarity index 100% rename from src/Interpreters/InterpreterSetRoleQuery.h rename to src/Interpreters/Access/InterpreterSetRoleQuery.h diff --git a/src/Interpreters/InterpreterShowAccessEntitiesQuery.cpp b/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp similarity index 96% rename from src/Interpreters/InterpreterShowAccessEntitiesQuery.cpp rename to src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp index 41b986e43a2..b0fe28e1abd 100644 --- a/src/Interpreters/InterpreterShowAccessEntitiesQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp @@ -1,10 +1,10 @@ -#include -#include +#include +#include #include -#include #include #include #include +#include namespace DB diff --git a/src/Interpreters/InterpreterShowAccessEntitiesQuery.h b/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.h similarity index 100% rename from src/Interpreters/InterpreterShowAccessEntitiesQuery.h rename to src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.h diff --git a/src/Interpreters/InterpreterShowAccessQuery.cpp b/src/Interpreters/Access/InterpreterShowAccessQuery.cpp similarity index 91% rename from src/Interpreters/InterpreterShowAccessQuery.cpp rename to src/Interpreters/Access/InterpreterShowAccessQuery.cpp index 86ab409d82b..2ecd51ff2a9 100644 --- a/src/Interpreters/InterpreterShowAccessQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowAccessQuery.cpp @@ -1,13 +1,13 @@ -#include +#include #include #include -#include -#include +#include +#include #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterShowAccessQuery.h b/src/Interpreters/Access/InterpreterShowAccessQuery.h similarity index 100% rename from src/Interpreters/InterpreterShowAccessQuery.h rename to src/Interpreters/Access/InterpreterShowAccessQuery.h diff --git a/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp similarity index 95% rename from src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp rename to src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp index 7be7032f48a..c68109002db 100644 --- a/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp @@ -1,15 +1,14 @@ -#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 @@ -20,10 +19,11 @@ #include #include #include -#include -#include #include #include +#include +#include +#include #include #include diff --git a/src/Interpreters/InterpreterShowCreateAccessEntityQuery.h b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.h similarity index 100% rename from src/Interpreters/InterpreterShowCreateAccessEntityQuery.h rename to src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.h diff --git a/src/Interpreters/InterpreterShowGrantsQuery.cpp b/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp similarity index 96% rename from src/Interpreters/InterpreterShowGrantsQuery.cpp rename to src/Interpreters/Access/InterpreterShowGrantsQuery.cpp index 7302e893cdd..930dc0dc5da 100644 --- a/src/Interpreters/InterpreterShowGrantsQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp @@ -1,16 +1,16 @@ -#include -#include -#include -#include +#include +#include +#include +#include #include -#include -#include -#include -#include #include -#include #include #include +#include +#include +#include +#include +#include #include #include diff --git a/src/Interpreters/InterpreterShowGrantsQuery.h b/src/Interpreters/Access/InterpreterShowGrantsQuery.h similarity index 100% rename from src/Interpreters/InterpreterShowGrantsQuery.h rename to src/Interpreters/Access/InterpreterShowGrantsQuery.h diff --git a/src/Interpreters/InterpreterShowPrivilegesQuery.cpp b/src/Interpreters/Access/InterpreterShowPrivilegesQuery.cpp similarity index 84% rename from src/Interpreters/InterpreterShowPrivilegesQuery.cpp rename to src/Interpreters/Access/InterpreterShowPrivilegesQuery.cpp index 201c1cfece8..05aa74d7dc4 100644 --- a/src/Interpreters/InterpreterShowPrivilegesQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowPrivilegesQuery.cpp @@ -1,4 +1,4 @@ -#include +#include #include diff --git a/src/Interpreters/InterpreterShowPrivilegesQuery.h b/src/Interpreters/Access/InterpreterShowPrivilegesQuery.h similarity index 100% rename from src/Interpreters/InterpreterShowPrivilegesQuery.h rename to src/Interpreters/Access/InterpreterShowPrivilegesQuery.h diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 2f86c8bef28..d5d2b1a722d 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -19,7 +19,7 @@ #include #include #include -#include +#include #include diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 729a495987f..b620ddf6a1e 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index 9fd318ee4cf..b52b91f47eb 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 6d38c55bd62..f1db1f771f3 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -41,7 +41,7 @@ #include #include -#include +#include #include #include diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 89d27a30555..03c4b4ae1b6 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 5370aee1096..638c671c3a3 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 4fbad7e5471..13a376dff8d 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterExistsQuery.cpp b/src/Interpreters/InterpreterExistsQuery.cpp index 24c30a8be30..5af51c61b29 100644 --- a/src/Interpreters/InterpreterExistsQuery.cpp +++ b/src/Interpreters/InterpreterExistsQuery.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/src/Interpreters/InterpreterFactory.cpp b/src/Interpreters/InterpreterFactory.cpp index fcf5f19aef6..e9ee2b0910a 100644 --- a/src/Interpreters/InterpreterFactory.cpp +++ b/src/Interpreters/InterpreterFactory.cpp @@ -2,17 +2,10 @@ #include #include #include -#include -#include -#include -#include -#include #include -#include #include #include #include -#include #include #include #include @@ -21,12 +14,6 @@ #include #include #include -#include -#include -#include -#include -#include -#include #include #include #include @@ -34,26 +21,33 @@ #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 -#include #include #include #include @@ -62,13 +56,7 @@ #include #include #include -#include -#include -#include -#include #include -#include -#include #include #include #include @@ -76,6 +64,20 @@ #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + #include #include diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 6a1a8652b23..231eb15b02f 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterOptimizeQuery.cpp b/src/Interpreters/InterpreterOptimizeQuery.cpp index 64de5ee0479..a44a49ec020 100644 --- a/src/Interpreters/InterpreterOptimizeQuery.cpp +++ b/src/Interpreters/InterpreterOptimizeQuery.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index e3d52487a52..72d7e9b1cba 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 0050df1bf52..e2233b98d7d 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -10,7 +10,7 @@ #include #include -#include +#include #include #include diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index 30a417f6fa7..84dbae0fac5 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index e34d974fa80..69bf036ae97 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -31,7 +31,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterUseQuery.cpp b/src/Interpreters/InterpreterUseQuery.cpp index 626d2f499c7..d8a5ae57470 100644 --- a/src/Interpreters/InterpreterUseQuery.cpp +++ b/src/Interpreters/InterpreterUseQuery.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include diff --git a/src/Interpreters/InterpreterWatchQuery.cpp b/src/Interpreters/InterpreterWatchQuery.cpp index e5e447562c6..4e30c3d21a4 100644 --- a/src/Interpreters/InterpreterWatchQuery.cpp +++ b/src/Interpreters/InterpreterWatchQuery.cpp @@ -14,7 +14,7 @@ limitations under the License. */ #include #include #include -#include +#include #include diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index f3cae33d752..82ba6a4a1fc 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include diff --git a/src/Interpreters/SessionLog.h b/src/Interpreters/SessionLog.h index 6d302c74d5f..794a8d3897f 100644 --- a/src/Interpreters/SessionLog.h +++ b/src/Interpreters/SessionLog.h @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB { diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index c3b8cc5c677..46ec6f776ee 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Parsers/ASTCreateQuotaQuery.cpp b/src/Parsers/Access/ASTCreateQuotaQuery.cpp similarity index 98% rename from src/Parsers/ASTCreateQuotaQuery.cpp rename to src/Parsers/Access/ASTCreateQuotaQuery.cpp index 135b25c5bdb..4e4c84f9e93 100644 --- a/src/Parsers/ASTCreateQuotaQuery.cpp +++ b/src/Parsers/Access/ASTCreateQuotaQuery.cpp @@ -1,5 +1,5 @@ -#include -#include +#include +#include #include #include #include diff --git a/src/Parsers/ASTCreateQuotaQuery.h b/src/Parsers/Access/ASTCreateQuotaQuery.h similarity index 100% rename from src/Parsers/ASTCreateQuotaQuery.h rename to src/Parsers/Access/ASTCreateQuotaQuery.h diff --git a/src/Parsers/ASTCreateRoleQuery.cpp b/src/Parsers/Access/ASTCreateRoleQuery.cpp similarity index 95% rename from src/Parsers/ASTCreateRoleQuery.cpp rename to src/Parsers/Access/ASTCreateRoleQuery.cpp index 73b523a5bfe..29e78d710cf 100644 --- a/src/Parsers/ASTCreateRoleQuery.cpp +++ b/src/Parsers/Access/ASTCreateRoleQuery.cpp @@ -1,5 +1,5 @@ -#include -#include +#include +#include #include #include diff --git a/src/Parsers/ASTCreateRoleQuery.h b/src/Parsers/Access/ASTCreateRoleQuery.h similarity index 100% rename from src/Parsers/ASTCreateRoleQuery.h rename to src/Parsers/Access/ASTCreateRoleQuery.h diff --git a/src/Parsers/ASTCreateRowPolicyQuery.cpp b/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp similarity index 97% rename from src/Parsers/ASTCreateRowPolicyQuery.cpp rename to src/Parsers/Access/ASTCreateRowPolicyQuery.cpp index 6aac008e0be..0267379d6e5 100644 --- a/src/Parsers/ASTCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp @@ -1,11 +1,11 @@ -#include -#include -#include +#include +#include +#include #include #include +#include #include #include -#include namespace DB diff --git a/src/Parsers/ASTCreateRowPolicyQuery.h b/src/Parsers/Access/ASTCreateRowPolicyQuery.h similarity index 100% rename from src/Parsers/ASTCreateRowPolicyQuery.h rename to src/Parsers/Access/ASTCreateRowPolicyQuery.h diff --git a/src/Parsers/ASTCreateSettingsProfileQuery.cpp b/src/Parsers/Access/ASTCreateSettingsProfileQuery.cpp similarity index 94% rename from src/Parsers/ASTCreateSettingsProfileQuery.cpp rename to src/Parsers/Access/ASTCreateSettingsProfileQuery.cpp index e99c40ca681..d9385e6be7b 100644 --- a/src/Parsers/ASTCreateSettingsProfileQuery.cpp +++ b/src/Parsers/Access/ASTCreateSettingsProfileQuery.cpp @@ -1,6 +1,6 @@ -#include -#include -#include +#include +#include +#include #include #include diff --git a/src/Parsers/ASTCreateSettingsProfileQuery.h b/src/Parsers/Access/ASTCreateSettingsProfileQuery.h similarity index 100% rename from src/Parsers/ASTCreateSettingsProfileQuery.h rename to src/Parsers/Access/ASTCreateSettingsProfileQuery.h diff --git a/src/Parsers/ASTCreateUserQuery.cpp b/src/Parsers/Access/ASTCreateUserQuery.cpp similarity index 98% rename from src/Parsers/ASTCreateUserQuery.cpp rename to src/Parsers/Access/ASTCreateUserQuery.cpp index 594d21f2a4b..5fcec4c8b07 100644 --- a/src/Parsers/ASTCreateUserQuery.cpp +++ b/src/Parsers/Access/ASTCreateUserQuery.cpp @@ -1,7 +1,7 @@ -#include -#include -#include -#include +#include +#include +#include +#include #include #include diff --git a/src/Parsers/ASTCreateUserQuery.h b/src/Parsers/Access/ASTCreateUserQuery.h similarity index 96% rename from src/Parsers/ASTCreateUserQuery.h rename to src/Parsers/Access/ASTCreateUserQuery.h index 9e80abcb6dd..183acb6d3fe 100644 --- a/src/Parsers/ASTCreateUserQuery.h +++ b/src/Parsers/Access/ASTCreateUserQuery.h @@ -3,8 +3,8 @@ #include #include #include -#include -#include +#include +#include namespace DB diff --git a/src/Parsers/ASTDropAccessEntityQuery.cpp b/src/Parsers/Access/ASTDropAccessEntityQuery.cpp similarity index 93% rename from src/Parsers/ASTDropAccessEntityQuery.cpp rename to src/Parsers/Access/ASTDropAccessEntityQuery.cpp index 6c19c9f8af3..19064ad9109 100644 --- a/src/Parsers/ASTDropAccessEntityQuery.cpp +++ b/src/Parsers/Access/ASTDropAccessEntityQuery.cpp @@ -1,5 +1,5 @@ -#include -#include +#include +#include #include #include diff --git a/src/Parsers/ASTDropAccessEntityQuery.h b/src/Parsers/Access/ASTDropAccessEntityQuery.h similarity index 100% rename from src/Parsers/ASTDropAccessEntityQuery.h rename to src/Parsers/Access/ASTDropAccessEntityQuery.h diff --git a/src/Parsers/ASTGrantQuery.cpp b/src/Parsers/Access/ASTGrantQuery.cpp similarity index 98% rename from src/Parsers/ASTGrantQuery.cpp rename to src/Parsers/Access/ASTGrantQuery.cpp index e2ac7658c0f..99dc119087c 100644 --- a/src/Parsers/ASTGrantQuery.cpp +++ b/src/Parsers/Access/ASTGrantQuery.cpp @@ -1,5 +1,5 @@ -#include -#include +#include +#include #include #include diff --git a/src/Parsers/ASTGrantQuery.h b/src/Parsers/Access/ASTGrantQuery.h similarity index 97% rename from src/Parsers/ASTGrantQuery.h rename to src/Parsers/Access/ASTGrantQuery.h index b0fb64cb33e..f8ea9b478fe 100644 --- a/src/Parsers/ASTGrantQuery.h +++ b/src/Parsers/Access/ASTGrantQuery.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include diff --git a/src/Parsers/ASTRolesOrUsersSet.cpp b/src/Parsers/Access/ASTRolesOrUsersSet.cpp similarity index 98% rename from src/Parsers/ASTRolesOrUsersSet.cpp rename to src/Parsers/Access/ASTRolesOrUsersSet.cpp index fc5385e4a58..dc7626b90d6 100644 --- a/src/Parsers/ASTRolesOrUsersSet.cpp +++ b/src/Parsers/Access/ASTRolesOrUsersSet.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Parsers/ASTRolesOrUsersSet.h b/src/Parsers/Access/ASTRolesOrUsersSet.h similarity index 100% rename from src/Parsers/ASTRolesOrUsersSet.h rename to src/Parsers/Access/ASTRolesOrUsersSet.h diff --git a/src/Parsers/ASTRowPolicyName.cpp b/src/Parsers/Access/ASTRowPolicyName.cpp similarity index 98% rename from src/Parsers/ASTRowPolicyName.cpp rename to src/Parsers/Access/ASTRowPolicyName.cpp index 0b69c1a46b3..c8b8107af20 100644 --- a/src/Parsers/ASTRowPolicyName.cpp +++ b/src/Parsers/Access/ASTRowPolicyName.cpp @@ -1,4 +1,4 @@ -#include +#include #include diff --git a/src/Parsers/ASTRowPolicyName.h b/src/Parsers/Access/ASTRowPolicyName.h similarity index 100% rename from src/Parsers/ASTRowPolicyName.h rename to src/Parsers/Access/ASTRowPolicyName.h diff --git a/src/Parsers/ASTSetRoleQuery.cpp b/src/Parsers/Access/ASTSetRoleQuery.cpp similarity index 91% rename from src/Parsers/ASTSetRoleQuery.cpp rename to src/Parsers/Access/ASTSetRoleQuery.cpp index e59e103b774..c886da1c8b5 100644 --- a/src/Parsers/ASTSetRoleQuery.cpp +++ b/src/Parsers/Access/ASTSetRoleQuery.cpp @@ -1,5 +1,5 @@ -#include -#include +#include +#include #include #include diff --git a/src/Parsers/ASTSetRoleQuery.h b/src/Parsers/Access/ASTSetRoleQuery.h similarity index 100% rename from src/Parsers/ASTSetRoleQuery.h rename to src/Parsers/Access/ASTSetRoleQuery.h diff --git a/src/Parsers/ASTSettingsProfileElement.cpp b/src/Parsers/Access/ASTSettingsProfileElement.cpp similarity index 98% rename from src/Parsers/ASTSettingsProfileElement.cpp rename to src/Parsers/Access/ASTSettingsProfileElement.cpp index 8f35c154a79..23dba8a926f 100644 --- a/src/Parsers/ASTSettingsProfileElement.cpp +++ b/src/Parsers/Access/ASTSettingsProfileElement.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/src/Parsers/ASTSettingsProfileElement.h b/src/Parsers/Access/ASTSettingsProfileElement.h similarity index 100% rename from src/Parsers/ASTSettingsProfileElement.h rename to src/Parsers/Access/ASTSettingsProfileElement.h diff --git a/src/Parsers/ASTShowAccessEntitiesQuery.cpp b/src/Parsers/Access/ASTShowAccessEntitiesQuery.cpp similarity index 96% rename from src/Parsers/ASTShowAccessEntitiesQuery.cpp rename to src/Parsers/Access/ASTShowAccessEntitiesQuery.cpp index 6dd53fd5cde..e2dfe031f53 100644 --- a/src/Parsers/ASTShowAccessEntitiesQuery.cpp +++ b/src/Parsers/Access/ASTShowAccessEntitiesQuery.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Parsers/ASTShowAccessEntitiesQuery.h b/src/Parsers/Access/ASTShowAccessEntitiesQuery.h similarity index 100% rename from src/Parsers/ASTShowAccessEntitiesQuery.h rename to src/Parsers/Access/ASTShowAccessEntitiesQuery.h diff --git a/src/Parsers/ASTShowAccessQuery.h b/src/Parsers/Access/ASTShowAccessQuery.h similarity index 100% rename from src/Parsers/ASTShowAccessQuery.h rename to src/Parsers/Access/ASTShowAccessQuery.h diff --git a/src/Parsers/ASTShowCreateAccessEntityQuery.cpp b/src/Parsers/Access/ASTShowCreateAccessEntityQuery.cpp similarity index 96% rename from src/Parsers/ASTShowCreateAccessEntityQuery.cpp rename to src/Parsers/Access/ASTShowCreateAccessEntityQuery.cpp index 5ff51a47002..db252db968d 100644 --- a/src/Parsers/ASTShowCreateAccessEntityQuery.cpp +++ b/src/Parsers/Access/ASTShowCreateAccessEntityQuery.cpp @@ -1,5 +1,5 @@ -#include -#include +#include +#include #include #include diff --git a/src/Parsers/ASTShowCreateAccessEntityQuery.h b/src/Parsers/Access/ASTShowCreateAccessEntityQuery.h similarity index 100% rename from src/Parsers/ASTShowCreateAccessEntityQuery.h rename to src/Parsers/Access/ASTShowCreateAccessEntityQuery.h diff --git a/src/Parsers/ASTShowGrantsQuery.cpp b/src/Parsers/Access/ASTShowGrantsQuery.cpp similarity index 90% rename from src/Parsers/ASTShowGrantsQuery.cpp rename to src/Parsers/Access/ASTShowGrantsQuery.cpp index 4011cfc522c..5d54cf45dc1 100644 --- a/src/Parsers/ASTShowGrantsQuery.cpp +++ b/src/Parsers/Access/ASTShowGrantsQuery.cpp @@ -1,5 +1,5 @@ -#include -#include +#include +#include #include #include diff --git a/src/Parsers/ASTShowGrantsQuery.h b/src/Parsers/Access/ASTShowGrantsQuery.h similarity index 100% rename from src/Parsers/ASTShowGrantsQuery.h rename to src/Parsers/Access/ASTShowGrantsQuery.h diff --git a/src/Parsers/ASTShowPrivilegesQuery.h b/src/Parsers/Access/ASTShowPrivilegesQuery.h similarity index 100% rename from src/Parsers/ASTShowPrivilegesQuery.h rename to src/Parsers/Access/ASTShowPrivilegesQuery.h diff --git a/src/Parsers/ASTUserNameWithHost.cpp b/src/Parsers/Access/ASTUserNameWithHost.cpp similarity index 97% rename from src/Parsers/ASTUserNameWithHost.cpp rename to src/Parsers/Access/ASTUserNameWithHost.cpp index b99ea5ab8d4..af84399ae45 100644 --- a/src/Parsers/ASTUserNameWithHost.cpp +++ b/src/Parsers/Access/ASTUserNameWithHost.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/src/Parsers/ASTUserNameWithHost.h b/src/Parsers/Access/ASTUserNameWithHost.h similarity index 100% rename from src/Parsers/ASTUserNameWithHost.h rename to src/Parsers/Access/ASTUserNameWithHost.h diff --git a/src/Parsers/ParserCreateQuotaQuery.cpp b/src/Parsers/Access/ParserCreateQuotaQuery.cpp similarity index 98% rename from src/Parsers/ParserCreateQuotaQuery.cpp rename to src/Parsers/Access/ParserCreateQuotaQuery.cpp index 682b345b937..0c6e1224cce 100644 --- a/src/Parsers/ParserCreateQuotaQuery.cpp +++ b/src/Parsers/Access/ParserCreateQuotaQuery.cpp @@ -1,14 +1,14 @@ -#include -#include +#include +#include +#include +#include +#include +#include #include +#include +#include #include #include -#include -#include -#include -#include -#include -#include #include #include #include diff --git a/src/Parsers/ParserCreateQuotaQuery.h b/src/Parsers/Access/ParserCreateQuotaQuery.h similarity index 100% rename from src/Parsers/ParserCreateQuotaQuery.h rename to src/Parsers/Access/ParserCreateQuotaQuery.h diff --git a/src/Parsers/ParserCreateRoleQuery.cpp b/src/Parsers/Access/ParserCreateRoleQuery.cpp similarity index 93% rename from src/Parsers/ParserCreateRoleQuery.cpp rename to src/Parsers/Access/ParserCreateRoleQuery.cpp index 5863136750f..314075cb7c0 100644 --- a/src/Parsers/ParserCreateRoleQuery.cpp +++ b/src/Parsers/Access/ParserCreateRoleQuery.cpp @@ -1,11 +1,11 @@ -#include -#include +#include +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include #include diff --git a/src/Parsers/ParserCreateRoleQuery.h b/src/Parsers/Access/ParserCreateRoleQuery.h similarity index 100% rename from src/Parsers/ParserCreateRoleQuery.h rename to src/Parsers/Access/ParserCreateRoleQuery.h diff --git a/src/Parsers/ParserCreateRowPolicyQuery.cpp b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp similarity index 96% rename from src/Parsers/ParserCreateRowPolicyQuery.cpp rename to src/Parsers/Access/ParserCreateRowPolicyQuery.cpp index d4d3db3f846..f6a33ec84a3 100644 --- a/src/Parsers/ParserCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp @@ -1,15 +1,15 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include +#include +#include +#include +#include +#include +#include #include #include -#include +#include +#include +#include #include #include diff --git a/src/Parsers/ParserCreateRowPolicyQuery.h b/src/Parsers/Access/ParserCreateRowPolicyQuery.h similarity index 100% rename from src/Parsers/ParserCreateRowPolicyQuery.h rename to src/Parsers/Access/ParserCreateRowPolicyQuery.h diff --git a/src/Parsers/ParserCreateSettingsProfileQuery.cpp b/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp similarity index 93% rename from src/Parsers/ParserCreateSettingsProfileQuery.cpp rename to src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp index 2d1e6824b50..8b5f2df2dd2 100644 --- a/src/Parsers/ParserCreateSettingsProfileQuery.cpp +++ b/src/Parsers/Access/ParserCreateSettingsProfileQuery.cpp @@ -1,12 +1,12 @@ -#include -#include +#include +#include +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include -#include #include #include diff --git a/src/Parsers/ParserCreateSettingsProfileQuery.h b/src/Parsers/Access/ParserCreateSettingsProfileQuery.h similarity index 100% rename from src/Parsers/ParserCreateSettingsProfileQuery.h rename to src/Parsers/Access/ParserCreateSettingsProfileQuery.h diff --git a/src/Parsers/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp similarity index 97% rename from src/Parsers/ParserCreateUserQuery.cpp rename to src/Parsers/Access/ParserCreateUserQuery.cpp index 7cada4b8ee8..421730b9ec5 100644 --- a/src/Parsers/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -1,18 +1,18 @@ -#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 diff --git a/src/Parsers/ParserCreateUserQuery.h b/src/Parsers/Access/ParserCreateUserQuery.h similarity index 100% rename from src/Parsers/ParserCreateUserQuery.h rename to src/Parsers/Access/ParserCreateUserQuery.h diff --git a/src/Parsers/ParserDropAccessEntityQuery.cpp b/src/Parsers/Access/ParserDropAccessEntityQuery.cpp similarity index 91% rename from src/Parsers/ParserDropAccessEntityQuery.cpp rename to src/Parsers/Access/ParserDropAccessEntityQuery.cpp index 14ef35e232c..d91cd8280a7 100644 --- a/src/Parsers/ParserDropAccessEntityQuery.cpp +++ b/src/Parsers/Access/ParserDropAccessEntityQuery.cpp @@ -1,10 +1,10 @@ -#include -#include +#include +#include +#include +#include +#include #include -#include -#include #include -#include #include diff --git a/src/Parsers/ParserDropAccessEntityQuery.h b/src/Parsers/Access/ParserDropAccessEntityQuery.h similarity index 100% rename from src/Parsers/ParserDropAccessEntityQuery.h rename to src/Parsers/Access/ParserDropAccessEntityQuery.h diff --git a/src/Parsers/ParserGrantQuery.cpp b/src/Parsers/Access/ParserGrantQuery.cpp similarity index 98% rename from src/Parsers/ParserGrantQuery.cpp rename to src/Parsers/Access/ParserGrantQuery.cpp index 85a6c9c71d4..8dd3e171237 100644 --- a/src/Parsers/ParserGrantQuery.cpp +++ b/src/Parsers/Access/ParserGrantQuery.cpp @@ -1,11 +1,11 @@ -#include -#include -#include +#include +#include +#include +#include #include #include #include #include -#include #include #include #include diff --git a/src/Parsers/ParserGrantQuery.h b/src/Parsers/Access/ParserGrantQuery.h similarity index 100% rename from src/Parsers/ParserGrantQuery.h rename to src/Parsers/Access/ParserGrantQuery.h diff --git a/src/Parsers/ParserRolesOrUsersSet.cpp b/src/Parsers/Access/ParserRolesOrUsersSet.cpp similarity index 96% rename from src/Parsers/ParserRolesOrUsersSet.cpp rename to src/Parsers/Access/ParserRolesOrUsersSet.cpp index 41e9ee6501d..6f426d89bb3 100644 --- a/src/Parsers/ParserRolesOrUsersSet.cpp +++ b/src/Parsers/Access/ParserRolesOrUsersSet.cpp @@ -1,9 +1,9 @@ -#include +#include +#include +#include +#include #include #include -#include -#include -#include #include #include diff --git a/src/Parsers/ParserRolesOrUsersSet.h b/src/Parsers/Access/ParserRolesOrUsersSet.h similarity index 100% rename from src/Parsers/ParserRolesOrUsersSet.h rename to src/Parsers/Access/ParserRolesOrUsersSet.h diff --git a/src/Parsers/ParserRowPolicyName.cpp b/src/Parsers/Access/ParserRowPolicyName.cpp similarity index 98% rename from src/Parsers/ParserRowPolicyName.cpp rename to src/Parsers/Access/ParserRowPolicyName.cpp index a3e12009c9a..aa159532754 100644 --- a/src/Parsers/ParserRowPolicyName.cpp +++ b/src/Parsers/Access/ParserRowPolicyName.cpp @@ -1,10 +1,10 @@ -#include -#include -#include -#include +#include +#include #include #include #include +#include +#include #include diff --git a/src/Parsers/ParserRowPolicyName.h b/src/Parsers/Access/ParserRowPolicyName.h similarity index 100% rename from src/Parsers/ParserRowPolicyName.h rename to src/Parsers/Access/ParserRowPolicyName.h diff --git a/src/Parsers/ParserSetRoleQuery.cpp b/src/Parsers/Access/ParserSetRoleQuery.cpp similarity index 92% rename from src/Parsers/ParserSetRoleQuery.cpp rename to src/Parsers/Access/ParserSetRoleQuery.cpp index 678474af040..50ccc67a372 100644 --- a/src/Parsers/ParserSetRoleQuery.cpp +++ b/src/Parsers/Access/ParserSetRoleQuery.cpp @@ -1,8 +1,8 @@ -#include -#include +#include +#include +#include +#include #include -#include -#include namespace DB diff --git a/src/Parsers/ParserSetRoleQuery.h b/src/Parsers/Access/ParserSetRoleQuery.h similarity index 100% rename from src/Parsers/ParserSetRoleQuery.h rename to src/Parsers/Access/ParserSetRoleQuery.h diff --git a/src/Parsers/ParserSettingsProfileElement.cpp b/src/Parsers/Access/ParserSettingsProfileElement.cpp similarity index 98% rename from src/Parsers/ParserSettingsProfileElement.cpp rename to src/Parsers/Access/ParserSettingsProfileElement.cpp index d7d982efe23..2c58bd0e623 100644 --- a/src/Parsers/ParserSettingsProfileElement.cpp +++ b/src/Parsers/Access/ParserSettingsProfileElement.cpp @@ -1,10 +1,10 @@ -#include +#include +#include +#include +#include #include #include #include -#include -#include -#include #include #include diff --git a/src/Parsers/ParserSettingsProfileElement.h b/src/Parsers/Access/ParserSettingsProfileElement.h similarity index 100% rename from src/Parsers/ParserSettingsProfileElement.h rename to src/Parsers/Access/ParserSettingsProfileElement.h diff --git a/src/Parsers/ParserShowAccessEntitiesQuery.cpp b/src/Parsers/Access/ParserShowAccessEntitiesQuery.cpp similarity index 96% rename from src/Parsers/ParserShowAccessEntitiesQuery.cpp rename to src/Parsers/Access/ParserShowAccessEntitiesQuery.cpp index 96a275902fb..b1329735b64 100644 --- a/src/Parsers/ParserShowAccessEntitiesQuery.cpp +++ b/src/Parsers/Access/ParserShowAccessEntitiesQuery.cpp @@ -1,5 +1,5 @@ -#include -#include +#include +#include #include #include #include diff --git a/src/Parsers/ParserShowAccessEntitiesQuery.h b/src/Parsers/Access/ParserShowAccessEntitiesQuery.h similarity index 100% rename from src/Parsers/ParserShowAccessEntitiesQuery.h rename to src/Parsers/Access/ParserShowAccessEntitiesQuery.h diff --git a/src/Parsers/ParserShowAccessQuery.h b/src/Parsers/Access/ParserShowAccessQuery.h similarity index 92% rename from src/Parsers/ParserShowAccessQuery.h rename to src/Parsers/Access/ParserShowAccessQuery.h index b6483aa3d43..da0d6ff449f 100644 --- a/src/Parsers/ParserShowAccessQuery.h +++ b/src/Parsers/Access/ParserShowAccessQuery.h @@ -1,9 +1,9 @@ #pragma once #include +#include #include #include -#include namespace DB diff --git a/src/Parsers/ParserShowCreateAccessEntityQuery.cpp b/src/Parsers/Access/ParserShowCreateAccessEntityQuery.cpp similarity index 95% rename from src/Parsers/ParserShowCreateAccessEntityQuery.cpp rename to src/Parsers/Access/ParserShowCreateAccessEntityQuery.cpp index 86ee64ab778..2df04513361 100644 --- a/src/Parsers/ParserShowCreateAccessEntityQuery.cpp +++ b/src/Parsers/Access/ParserShowCreateAccessEntityQuery.cpp @@ -1,10 +1,10 @@ -#include -#include +#include +#include +#include +#include +#include #include #include -#include -#include -#include #include #include #include diff --git a/src/Parsers/ParserShowCreateAccessEntityQuery.h b/src/Parsers/Access/ParserShowCreateAccessEntityQuery.h similarity index 100% rename from src/Parsers/ParserShowCreateAccessEntityQuery.h rename to src/Parsers/Access/ParserShowCreateAccessEntityQuery.h diff --git a/src/Parsers/ParserShowGrantsQuery.cpp b/src/Parsers/Access/ParserShowGrantsQuery.cpp similarity index 79% rename from src/Parsers/ParserShowGrantsQuery.cpp rename to src/Parsers/Access/ParserShowGrantsQuery.cpp index bd9e4012771..02d85d2f90b 100644 --- a/src/Parsers/ParserShowGrantsQuery.cpp +++ b/src/Parsers/Access/ParserShowGrantsQuery.cpp @@ -1,9 +1,9 @@ -#include -#include -#include -#include +#include +#include +#include +#include +#include #include -#include namespace DB diff --git a/src/Parsers/ParserShowGrantsQuery.h b/src/Parsers/Access/ParserShowGrantsQuery.h similarity index 100% rename from src/Parsers/ParserShowGrantsQuery.h rename to src/Parsers/Access/ParserShowGrantsQuery.h diff --git a/src/Parsers/ParserShowPrivilegesQuery.cpp b/src/Parsers/Access/ParserShowPrivilegesQuery.cpp similarity index 76% rename from src/Parsers/ParserShowPrivilegesQuery.cpp rename to src/Parsers/Access/ParserShowPrivilegesQuery.cpp index 56b4327dccf..a120d4ed7c2 100644 --- a/src/Parsers/ParserShowPrivilegesQuery.cpp +++ b/src/Parsers/Access/ParserShowPrivilegesQuery.cpp @@ -1,6 +1,6 @@ -#include +#include +#include #include -#include namespace DB diff --git a/src/Parsers/ParserShowPrivilegesQuery.h b/src/Parsers/Access/ParserShowPrivilegesQuery.h similarity index 100% rename from src/Parsers/ParserShowPrivilegesQuery.h rename to src/Parsers/Access/ParserShowPrivilegesQuery.h diff --git a/src/Parsers/ParserUserNameWithHost.cpp b/src/Parsers/Access/ParserUserNameWithHost.cpp similarity index 95% rename from src/Parsers/ParserUserNameWithHost.cpp rename to src/Parsers/Access/ParserUserNameWithHost.cpp index 9cb4bb6fc97..c9c655fecc4 100644 --- a/src/Parsers/ParserUserNameWithHost.cpp +++ b/src/Parsers/Access/ParserUserNameWithHost.cpp @@ -1,5 +1,5 @@ -#include -#include +#include +#include #include #include #include diff --git a/src/Parsers/ParserUserNameWithHost.h b/src/Parsers/Access/ParserUserNameWithHost.h similarity index 100% rename from src/Parsers/ParserUserNameWithHost.h rename to src/Parsers/Access/ParserUserNameWithHost.h diff --git a/src/Parsers/parseUserName.cpp b/src/Parsers/Access/parseUserName.cpp similarity index 88% rename from src/Parsers/parseUserName.cpp rename to src/Parsers/Access/parseUserName.cpp index 1f25f51ef22..fb20d4d1e6c 100644 --- a/src/Parsers/parseUserName.cpp +++ b/src/Parsers/Access/parseUserName.cpp @@ -1,6 +1,6 @@ -#include -#include -#include +#include +#include +#include #include diff --git a/src/Parsers/parseUserName.h b/src/Parsers/Access/parseUserName.h similarity index 100% rename from src/Parsers/parseUserName.h rename to src/Parsers/Access/parseUserName.h diff --git a/src/Parsers/CMakeLists.txt b/src/Parsers/CMakeLists.txt index a20dd3567a9..d945e63589a 100644 --- a/src/Parsers/CMakeLists.txt +++ b/src/Parsers/CMakeLists.txt @@ -1,8 +1,10 @@ include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake") + add_headers_and_sources(clickhouse_parsers .) +add_headers_and_sources(clickhouse_parsers ./Access) add_headers_and_sources(clickhouse_parsers ./MySQL) add_library(clickhouse_parsers ${clickhouse_parsers_headers} ${clickhouse_parsers_sources}) -target_link_libraries(clickhouse_parsers PUBLIC clickhouse_common_io) +target_link_libraries(clickhouse_parsers PUBLIC clickhouse_common_io clickhouse_common_access) if (USE_DEBUG_HELPERS) set (INCLUDE_DEBUG_HELPERS "-I\"${ClickHouse_SOURCE_DIR}/base\" -include \"${ClickHouse_SOURCE_DIR}/src/Parsers/iostream_debug_helpers.h\"") diff --git a/src/Parsers/ParserQuery.cpp b/src/Parsers/ParserQuery.cpp index 345013b6475..7677efd9415 100644 --- a/src/Parsers/ParserQuery.cpp +++ b/src/Parsers/ParserQuery.cpp @@ -2,26 +2,27 @@ #include #include #include -#include -#include -#include -#include -#include -#include #include #include -#include #include #include #include #include #include #include -#include #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include + namespace DB { diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 4309063a736..f1e007948f9 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -13,16 +13,16 @@ #include #include #include -#include -#include -#include -#include -#include #include #include #include #include #include +#include +#include +#include +#include +#include #include "Common/Exception.h" diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index b2a00055532..32ff2d854cd 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include #include diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index ec748d4d43a..e3e6b4382f4 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -41,7 +41,7 @@ limitations under the License. */ #include #include #include -#include +#include #include diff --git a/src/Storages/StorageFactory.h b/src/Storages/StorageFactory.h index bdc57bfdc6d..20db1a44897 100644 --- a/src/Storages/StorageFactory.h +++ b/src/Storages/StorageFactory.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 361ff95984f..e7de3010c33 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/System/StorageSystemGrants.cpp b/src/Storages/System/StorageSystemGrants.cpp index 1ba5e6d96a4..42917033a28 100644 --- a/src/Storages/System/StorageSystemGrants.cpp +++ b/src/Storages/System/StorageSystemGrants.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/System/StorageSystemPrivileges.cpp b/src/Storages/System/StorageSystemPrivileges.cpp index ca369efe43a..ff3c6eecc6b 100644 --- a/src/Storages/System/StorageSystemPrivileges.cpp +++ b/src/Storages/System/StorageSystemPrivileges.cpp @@ -1,18 +1,18 @@ #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 diff --git a/src/Storages/System/StorageSystemQuotaLimits.cpp b/src/Storages/System/StorageSystemQuotaLimits.cpp index 3b64f72d621..6d837018053 100644 --- a/src/Storages/System/StorageSystemQuotaLimits.cpp +++ b/src/Storages/System/StorageSystemQuotaLimits.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/System/StorageSystemQuotaUsage.cpp b/src/Storages/System/StorageSystemQuotaUsage.cpp index 6723037cf3b..2d2c5ec2864 100644 --- a/src/Storages/System/StorageSystemQuotaUsage.cpp +++ b/src/Storages/System/StorageSystemQuotaUsage.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/System/StorageSystemQuotas.cpp b/src/Storages/System/StorageSystemQuotas.cpp index f5f57b48232..eaa2f79364b 100644 --- a/src/Storages/System/StorageSystemQuotas.cpp +++ b/src/Storages/System/StorageSystemQuotas.cpp @@ -1,17 +1,17 @@ #include +#include +#include +#include +#include +#include +#include #include #include #include #include #include -#include -#include -#include #include -#include -#include -#include -#include +#include #include diff --git a/src/Storages/System/StorageSystemQuotasUsage.cpp b/src/Storages/System/StorageSystemQuotasUsage.cpp index 363562bce19..98a77dde9f4 100644 --- a/src/Storages/System/StorageSystemQuotasUsage.cpp +++ b/src/Storages/System/StorageSystemQuotasUsage.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/System/StorageSystemRoles.cpp b/src/Storages/System/StorageSystemRoles.cpp index 42983670466..24a5b3ede4d 100644 --- a/src/Storages/System/StorageSystemRoles.cpp +++ b/src/Storages/System/StorageSystemRoles.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/System/StorageSystemRowPolicies.cpp b/src/Storages/System/StorageSystemRowPolicies.cpp index 3fa62bc784b..73323b13d79 100644 --- a/src/Storages/System/StorageSystemRowPolicies.cpp +++ b/src/Storages/System/StorageSystemRowPolicies.cpp @@ -1,18 +1,18 @@ #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 diff --git a/src/Storages/System/StorageSystemSettingsProfiles.cpp b/src/Storages/System/StorageSystemSettingsProfiles.cpp index 87847fb80bc..c81e9665d16 100644 --- a/src/Storages/System/StorageSystemSettingsProfiles.cpp +++ b/src/Storages/System/StorageSystemSettingsProfiles.cpp @@ -1,16 +1,16 @@ #include +#include +#include +#include +#include +#include +#include #include #include #include #include -#include -#include -#include #include -#include -#include -#include -#include +#include namespace DB diff --git a/src/Storages/System/StorageSystemUsers.cpp b/src/Storages/System/StorageSystemUsers.cpp index a48e12a1476..bfce1d50160 100644 --- a/src/Storages/System/StorageSystemUsers.cpp +++ b/src/Storages/System/StorageSystemUsers.cpp @@ -1,4 +1,7 @@ #include +#include +#include +#include #include #include #include @@ -8,10 +11,7 @@ #include #include #include -#include -#include -#include -#include +#include #include #include #include diff --git a/src/TableFunctions/ITableFunction.cpp b/src/TableFunctions/ITableFunction.cpp index 218d86fe4a2..fa7f6e52220 100644 --- a/src/TableFunctions/ITableFunction.cpp +++ b/src/TableFunctions/ITableFunction.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include diff --git a/src/TableFunctions/TableFunctionFile.cpp b/src/TableFunctions/TableFunctionFile.cpp index dcb91c8cc2e..d8bdb3b45c4 100644 --- a/src/TableFunctions/TableFunctionFile.cpp +++ b/src/TableFunctions/TableFunctionFile.cpp @@ -1,7 +1,7 @@ #include #include "registerTableFunctions.h" -#include +#include #include #include #include diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index bbaa1b5f048..f2947268905 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -1,7 +1,7 @@ #include #include "registerTableFunctions.h" -#include +#include #include #include #include From ab01b9afc8bcd11a6b900070eaeccac69bde799f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 1 Nov 2021 17:03:20 +0300 Subject: [PATCH 915/919] Split Authentication.h to common and main parts. --- src/Access/{Common => }/Authentication.cpp | 79 ++--- src/Access/Authentication.h | 52 +++ src/Access/Common/Authentication.h | 331 ------------------ src/Access/Common/AuthenticationData.cpp | 196 +++++++++++ src/Access/Common/AuthenticationData.h | 103 ++++++ src/Access/IAccessStorage.cpp | 5 +- src/Access/LDAPAccessStorage.cpp | 8 +- src/Access/User.cpp | 2 +- src/Access/User.h | 4 +- src/Access/UsersConfigAccessStorage.cpp | 20 +- src/Core/PostgreSQLProtocol.h | 14 +- .../Access/InterpreterCreateUserQuery.cpp | 4 +- ...InterpreterShowCreateAccessEntityQuery.cpp | 4 +- src/Interpreters/Session.cpp | 6 +- src/Interpreters/Session.h | 8 +- src/Interpreters/SessionLog.cpp | 12 +- src/Interpreters/SessionLog.h | 4 +- src/Parsers/Access/ASTCreateUserQuery.cpp | 50 +-- src/Parsers/Access/ASTCreateUserQuery.h | 4 +- src/Parsers/Access/ParserCreateUserQuery.cpp | 44 +-- src/Server/HTTPHandler.cpp | 2 +- src/Server/MySQLHandler.cpp | 2 +- src/Server/PostgreSQLHandler.h | 4 + src/Server/PostgreSQLHandlerFactory.h | 4 + src/Storages/System/StorageSystemUsers.cpp | 22 +- 25 files changed, 507 insertions(+), 477 deletions(-) rename src/Access/{Common => }/Authentication.cpp (64%) create mode 100644 src/Access/Authentication.h delete mode 100644 src/Access/Common/Authentication.h create mode 100644 src/Access/Common/AuthenticationData.cpp create mode 100644 src/Access/Common/AuthenticationData.h diff --git a/src/Access/Common/Authentication.cpp b/src/Access/Authentication.cpp similarity index 64% rename from src/Access/Common/Authentication.cpp rename to src/Access/Authentication.cpp index 886b58202fd..d0aef37c15b 100644 --- a/src/Access/Common/Authentication.cpp +++ b/src/Access/Authentication.cpp @@ -1,4 +1,5 @@ -#include +#include +#include #include #include #include @@ -17,8 +18,8 @@ namespace ErrorCodes namespace { - using Digest = Authentication::Digest; - using Util = Authentication::Util; + using Digest = AuthenticationData::Digest; + using Util = AuthenticationData::Util; bool checkPasswordPlainText(const String & password, const Digest & password_plaintext) { @@ -67,76 +68,76 @@ namespace } -bool Authentication::areCredentialsValid(const Credentials & credentials, const ExternalAuthenticators & external_authenticators) const +bool Authentication::areCredentialsValid(const Credentials & credentials, const AuthenticationData & auth_data, const ExternalAuthenticators & external_authenticators) { if (!credentials.isReady()) return false; if (const auto * gss_acceptor_context = typeid_cast(&credentials)) { - switch (type) + switch (auth_data.getType()) { - case NO_PASSWORD: - case PLAINTEXT_PASSWORD: - case SHA256_PASSWORD: - case DOUBLE_SHA1_PASSWORD: - case LDAP: - throw Require("ClickHouse Basic Authentication"); + case AuthenticationType::NO_PASSWORD: + case AuthenticationType::PLAINTEXT_PASSWORD: + case AuthenticationType::SHA256_PASSWORD: + case AuthenticationType::DOUBLE_SHA1_PASSWORD: + case AuthenticationType::LDAP: + throw Authentication::Require("ClickHouse Basic Authentication"); - case KERBEROS: - return external_authenticators.checkKerberosCredentials(kerberos_realm, *gss_acceptor_context); + case AuthenticationType::KERBEROS: + return external_authenticators.checkKerberosCredentials(auth_data.getKerberosRealm(), *gss_acceptor_context); - case MAX_TYPE: + case AuthenticationType::MAX_TYPE: break; } } if (const auto * mysql_credentials = typeid_cast(&credentials)) { - switch (type) + switch (auth_data.getType()) { - case NO_PASSWORD: + case AuthenticationType::NO_PASSWORD: return true; // N.B. even if the password is not empty! - case PLAINTEXT_PASSWORD: - return checkPasswordPlainTextMySQL(mysql_credentials->getScramble(), mysql_credentials->getScrambledPassword(), password_hash); + case AuthenticationType::PLAINTEXT_PASSWORD: + return checkPasswordPlainTextMySQL(mysql_credentials->getScramble(), mysql_credentials->getScrambledPassword(), auth_data.getPasswordHashBinary()); - case DOUBLE_SHA1_PASSWORD: - return checkPasswordDoubleSHA1MySQL(mysql_credentials->getScramble(), mysql_credentials->getScrambledPassword(), password_hash); + case AuthenticationType::DOUBLE_SHA1_PASSWORD: + return checkPasswordDoubleSHA1MySQL(mysql_credentials->getScramble(), mysql_credentials->getScrambledPassword(), auth_data.getPasswordHashBinary()); - case SHA256_PASSWORD: - case LDAP: - case KERBEROS: - throw Require("ClickHouse Basic Authentication"); + case AuthenticationType::SHA256_PASSWORD: + case AuthenticationType::LDAP: + case AuthenticationType::KERBEROS: + throw Authentication::Require("ClickHouse Basic Authentication"); - case MAX_TYPE: + case AuthenticationType::MAX_TYPE: break; } } if (const auto * basic_credentials = typeid_cast(&credentials)) { - switch (type) + switch (auth_data.getType()) { - case NO_PASSWORD: + case AuthenticationType::NO_PASSWORD: return true; // N.B. even if the password is not empty! - case PLAINTEXT_PASSWORD: - return checkPasswordPlainText(basic_credentials->getPassword(), password_hash); + case AuthenticationType::PLAINTEXT_PASSWORD: + return checkPasswordPlainText(basic_credentials->getPassword(), auth_data.getPasswordHashBinary()); - case SHA256_PASSWORD: - return checkPasswordSHA256(basic_credentials->getPassword(), password_hash); + case AuthenticationType::SHA256_PASSWORD: + return checkPasswordSHA256(basic_credentials->getPassword(), auth_data.getPasswordHashBinary()); - case DOUBLE_SHA1_PASSWORD: - return checkPasswordDoubleSHA1(basic_credentials->getPassword(), password_hash); + case AuthenticationType::DOUBLE_SHA1_PASSWORD: + return checkPasswordDoubleSHA1(basic_credentials->getPassword(), auth_data.getPasswordHashBinary()); - case LDAP: - return external_authenticators.checkLDAPCredentials(ldap_server_name, *basic_credentials); + case AuthenticationType::LDAP: + return external_authenticators.checkLDAPCredentials(auth_data.getLDAPServerName(), *basic_credentials); - case KERBEROS: - throw Require(kerberos_realm); + case AuthenticationType::KERBEROS: + throw Authentication::Require(auth_data.getKerberosRealm()); - case MAX_TYPE: + case AuthenticationType::MAX_TYPE: break; } } @@ -144,7 +145,7 @@ bool Authentication::areCredentialsValid(const Credentials & credentials, const if ([[maybe_unused]] const auto * always_allow_credentials = typeid_cast(&credentials)) return true; - throw Exception("areCredentialsValid(): authentication type " + toString(type) + " not supported", ErrorCodes::NOT_IMPLEMENTED); + throw Exception("areCredentialsValid(): authentication type " + toString(auth_data.getType()) + " not supported", ErrorCodes::NOT_IMPLEMENTED); } } diff --git a/src/Access/Authentication.h b/src/Access/Authentication.h new file mode 100644 index 00000000000..000ba8ca324 --- /dev/null +++ b/src/Access/Authentication.h @@ -0,0 +1,52 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +class Credentials; +class ExternalAuthenticators; + + +/// TODO: Try to move this checking to Credentials. +struct Authentication +{ + /// Checks the credentials (passwords, readiness, etc.) + static bool areCredentialsValid(const Credentials & credentials, const AuthenticationData & auth_data, const ExternalAuthenticators & external_authenticators); + + // A signaling class used to communicate requirements for credentials. + template + class Require : public Exception + { + public: + explicit Require(const String & realm_); + const String & getRealm() const; + + private: + const String realm; + }; +}; + + +template +Authentication::Require::Require(const String & realm_) + : Exception("Credentials required", ErrorCodes::BAD_ARGUMENTS) + , realm(realm_) +{ +} + +template +const String & Authentication::Require::getRealm() const +{ + return realm; +} + +} diff --git a/src/Access/Common/Authentication.h b/src/Access/Common/Authentication.h deleted file mode 100644 index e43d3793ee0..00000000000 --- a/src/Access/Common/Authentication.h +++ /dev/null @@ -1,331 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -namespace ErrorCodes -{ - extern const int SUPPORT_IS_DISABLED; - extern const int BAD_ARGUMENTS; - extern const int LOGICAL_ERROR; - extern const int NOT_IMPLEMENTED; -} - -class Credentials; -class ExternalAuthenticators; - -/// Authentication type and encrypted password for checking when a user logins. -class Authentication -{ -public: - enum Type - { - /// User doesn't have to enter password. - NO_PASSWORD, - - /// Password is stored as is. - PLAINTEXT_PASSWORD, - - /// Password is encrypted in SHA256 hash. - SHA256_PASSWORD, - - /// SHA1(SHA1(password)). - /// This kind of hash is used by the `mysql_native_password` authentication plugin. - DOUBLE_SHA1_PASSWORD, - - /// Password is checked by a [remote] LDAP server. Connection will be made at each authentication attempt. - LDAP, - - /// Kerberos authentication performed through GSS-API negotiation loop. - KERBEROS, - - MAX_TYPE, - }; - - struct TypeInfo - { - const char * const raw_name; - const String name; /// Lowercased with underscores, e.g. "sha256_password". - static const TypeInfo & get(Type type_); - }; - - // A signaling class used to communicate requirements for credentials. - template - class Require : public Exception - { - public: - explicit Require(const String & realm_); - const String & getRealm() const; - - private: - const String realm; - }; - - using Digest = std::vector; - - Authentication(Authentication::Type type_ = NO_PASSWORD) : type(type_) {} - Authentication(const Authentication & src) = default; - Authentication & operator =(const Authentication & src) = default; - Authentication(Authentication && src) = default; - Authentication & operator =(Authentication && src) = default; - - Type getType() const { return type; } - - /// Sets the password and encrypt it using the authentication type set in the constructor. - void setPassword(const String & password_); - - /// Returns the password. Allowed to use only for Type::PLAINTEXT_PASSWORD. - String getPassword() const; - - /// Sets the password as a string of hexadecimal digits. - void setPasswordHashHex(const String & hash); - String getPasswordHashHex() const; - - /// Sets the password in binary form. - void setPasswordHashBinary(const Digest & hash); - const Digest & getPasswordHashBinary() const { return password_hash; } - - /// Sets the server name for authentication type LDAP. - const String & getLDAPServerName() const; - void setLDAPServerName(const String & name); - - /// Sets the realm name for authentication type KERBEROS. - const String & getKerberosRealm() const; - void setKerberosRealm(const String & realm); - - /// Checks the credentials (passwords, readiness, etc.) - bool areCredentialsValid(const Credentials & credentials, const ExternalAuthenticators & external_authenticators) const; - - friend bool operator ==(const Authentication & lhs, const Authentication & rhs) { return (lhs.type == rhs.type) && (lhs.password_hash == rhs.password_hash); } - friend bool operator !=(const Authentication & lhs, const Authentication & rhs) { return !(lhs == rhs); } - - struct Util - { - static Digest encodePlainText(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 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(const Digest & text) { return encodeSHA1(encodeSHA1(text)); } - }; - -private: - Type type = Type::NO_PASSWORD; - Digest password_hash; - String ldap_server_name; - String kerberos_realm; -}; - - -inline const Authentication::TypeInfo & Authentication::TypeInfo::get(Type type_) -{ - static constexpr auto make_info = [](const char * raw_name_) - { - String init_name = raw_name_; - boost::to_lower(init_name); - return TypeInfo{raw_name_, std::move(init_name)}; - }; - - switch (type_) - { - case NO_PASSWORD: - { - static const auto info = make_info("NO_PASSWORD"); - return info; - } - case PLAINTEXT_PASSWORD: - { - static const auto info = make_info("PLAINTEXT_PASSWORD"); - return info; - } - case SHA256_PASSWORD: - { - static const auto info = make_info("SHA256_PASSWORD"); - return info; - } - case DOUBLE_SHA1_PASSWORD: - { - static const auto info = make_info("DOUBLE_SHA1_PASSWORD"); - return info; - } - case LDAP: - { - static const auto info = make_info("LDAP"); - return info; - } - case KERBEROS: - { - static const auto info = make_info("KERBEROS"); - return info; - } - case MAX_TYPE: - break; - } - throw Exception("Unknown authentication type: " + std::to_string(static_cast(type_)), ErrorCodes::LOGICAL_ERROR); -} - -template -Authentication::Require::Require(const String & realm_) - : Exception("Credentials required", ErrorCodes::BAD_ARGUMENTS) - , realm(realm_) -{ -} - -template -const String & Authentication::Require::getRealm() const -{ - return realm; -} - -inline String toString(Authentication::Type type_) -{ - return Authentication::TypeInfo::get(type_).raw_name; -} - - -inline Authentication::Digest Authentication::Util::encodeSHA256(const std::string_view & text [[maybe_unused]]) -{ -#if USE_SSL - Digest hash; - hash.resize(32); - ::DB::encodeSHA256(text, hash.data()); - return hash; -#else - throw DB::Exception( - "SHA256 passwords support is disabled, because ClickHouse was built without SSL library", - DB::ErrorCodes::SUPPORT_IS_DISABLED); -#endif -} - -inline Authentication::Digest Authentication::Util::encodeSHA1(const std::string_view & text) -{ - Poco::SHA1Engine engine; - engine.update(text.data(), text.size()); - return engine.digest(); -} - - -inline void Authentication::setPassword(const String & password_) -{ - switch (type) - { - case PLAINTEXT_PASSWORD: - return setPasswordHashBinary(Util::encodePlainText(password_)); - - case SHA256_PASSWORD: - return setPasswordHashBinary(Util::encodeSHA256(password_)); - - case DOUBLE_SHA1_PASSWORD: - return setPasswordHashBinary(Util::encodeDoubleSHA1(password_)); - - case NO_PASSWORD: - case LDAP: - case KERBEROS: - throw Exception("Cannot specify password for authentication type " + toString(type), ErrorCodes::LOGICAL_ERROR); - - case MAX_TYPE: - break; - } - throw Exception("setPassword(): authentication type " + toString(type) + " not supported", ErrorCodes::NOT_IMPLEMENTED); -} - - -inline String Authentication::getPassword() const -{ - if (type != PLAINTEXT_PASSWORD) - throw Exception("Cannot decode the password", ErrorCodes::LOGICAL_ERROR); - return String(password_hash.data(), password_hash.data() + password_hash.size()); -} - - -inline void Authentication::setPasswordHashHex(const String & hash) -{ - Digest digest; - digest.resize(hash.size() / 2); - boost::algorithm::unhex(hash.begin(), hash.end(), digest.data()); - setPasswordHashBinary(digest); -} - -inline String Authentication::getPasswordHashHex() const -{ - if (type == LDAP || type == KERBEROS) - throw Exception("Cannot get password hex hash for authentication type " + toString(type), ErrorCodes::LOGICAL_ERROR); - - String hex; - hex.resize(password_hash.size() * 2); - boost::algorithm::hex(password_hash.begin(), password_hash.end(), hex.data()); - return hex; -} - - -inline void Authentication::setPasswordHashBinary(const Digest & hash) -{ - switch (type) - { - case PLAINTEXT_PASSWORD: - { - password_hash = hash; - return; - } - - case SHA256_PASSWORD: - { - if (hash.size() != 32) - throw Exception( - "Password hash for the 'SHA256_PASSWORD' authentication type has length " + std::to_string(hash.size()) - + " but must be exactly 32 bytes.", - ErrorCodes::BAD_ARGUMENTS); - password_hash = hash; - return; - } - - case DOUBLE_SHA1_PASSWORD: - { - if (hash.size() != 20) - throw Exception( - "Password hash for the 'DOUBLE_SHA1_PASSWORD' authentication type has length " + std::to_string(hash.size()) - + " but must be exactly 20 bytes.", - ErrorCodes::BAD_ARGUMENTS); - password_hash = hash; - return; - } - - case NO_PASSWORD: - case LDAP: - case KERBEROS: - throw Exception("Cannot specify password binary hash for authentication type " + toString(type), ErrorCodes::LOGICAL_ERROR); - - case MAX_TYPE: - break; - } - throw Exception("setPasswordHashBinary(): authentication type " + toString(type) + " not supported", ErrorCodes::NOT_IMPLEMENTED); -} - -inline const String & Authentication::getLDAPServerName() const -{ - return ldap_server_name; -} - -inline void Authentication::setLDAPServerName(const String & name) -{ - ldap_server_name = name; -} - -inline const String & Authentication::getKerberosRealm() const -{ - return kerberos_realm; -} - -inline void Authentication::setKerberosRealm(const String & realm) -{ - kerberos_realm = realm; -} - -} diff --git a/src/Access/Common/AuthenticationData.cpp b/src/Access/Common/AuthenticationData.cpp new file mode 100644 index 00000000000..e460d1dcc4d --- /dev/null +++ b/src/Access/Common/AuthenticationData.cpp @@ -0,0 +1,196 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int SUPPORT_IS_DISABLED; + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; +} + + +const AuthenticationTypeInfo & AuthenticationTypeInfo::get(AuthenticationType type_) +{ + static constexpr auto make_info = [](const char * raw_name_) + { + String init_name = raw_name_; + boost::to_lower(init_name); + return AuthenticationTypeInfo{raw_name_, std::move(init_name)}; + }; + + switch (type_) + { + case AuthenticationType::NO_PASSWORD: + { + static const auto info = make_info("NO_PASSWORD"); + return info; + } + case AuthenticationType::PLAINTEXT_PASSWORD: + { + static const auto info = make_info("PLAINTEXT_PASSWORD"); + return info; + } + case AuthenticationType::SHA256_PASSWORD: + { + static const auto info = make_info("SHA256_PASSWORD"); + return info; + } + case AuthenticationType::DOUBLE_SHA1_PASSWORD: + { + static const auto info = make_info("DOUBLE_SHA1_PASSWORD"); + return info; + } + case AuthenticationType::LDAP: + { + static const auto info = make_info("LDAP"); + return info; + } + case AuthenticationType::KERBEROS: + { + static const auto info = make_info("KERBEROS"); + return info; + } + case AuthenticationType::MAX_TYPE: + break; + } + throw Exception("Unknown authentication type: " + std::to_string(static_cast(type_)), ErrorCodes::LOGICAL_ERROR); +} + + +AuthenticationData::Digest AuthenticationData::Util::encodeSHA256(const std::string_view & text [[maybe_unused]]) +{ +#if USE_SSL + Digest hash; + hash.resize(32); + ::DB::encodeSHA256(text, hash.data()); + return hash; +#else + throw DB::Exception( + "SHA256 passwords support is disabled, because ClickHouse was built without SSL library", + DB::ErrorCodes::SUPPORT_IS_DISABLED); +#endif +} + + +AuthenticationData::Digest AuthenticationData::Util::encodeSHA1(const std::string_view & text) +{ + Poco::SHA1Engine engine; + engine.update(text.data(), text.size()); + return engine.digest(); +} + + +bool operator ==(const AuthenticationData & lhs, const AuthenticationData & rhs) +{ + return (lhs.type == rhs.type) && (lhs.password_hash == rhs.password_hash) + && (lhs.ldap_server_name == rhs.ldap_server_name) && (lhs.kerberos_realm == rhs.kerberos_realm); +} + + +void AuthenticationData::setPassword(const String & password_) +{ + switch (type) + { + case AuthenticationType::PLAINTEXT_PASSWORD: + return setPasswordHashBinary(Util::encodePlainText(password_)); + + case AuthenticationType::SHA256_PASSWORD: + return setPasswordHashBinary(Util::encodeSHA256(password_)); + + case AuthenticationType::DOUBLE_SHA1_PASSWORD: + return setPasswordHashBinary(Util::encodeDoubleSHA1(password_)); + + case AuthenticationType::NO_PASSWORD: + case AuthenticationType::LDAP: + case AuthenticationType::KERBEROS: + throw Exception("Cannot specify password for authentication type " + toString(type), ErrorCodes::LOGICAL_ERROR); + + case AuthenticationType::MAX_TYPE: + break; + } + throw Exception("setPassword(): authentication type " + toString(type) + " not supported", ErrorCodes::NOT_IMPLEMENTED); +} + + +String AuthenticationData::getPassword() const +{ + if (type != AuthenticationType::PLAINTEXT_PASSWORD) + throw Exception("Cannot decode the password", ErrorCodes::LOGICAL_ERROR); + return String(password_hash.data(), password_hash.data() + password_hash.size()); +} + + +void AuthenticationData::setPasswordHashHex(const String & hash) +{ + Digest digest; + digest.resize(hash.size() / 2); + boost::algorithm::unhex(hash.begin(), hash.end(), digest.data()); + setPasswordHashBinary(digest); +} + + +String AuthenticationData::getPasswordHashHex() const +{ + if (type == AuthenticationType::LDAP || type == AuthenticationType::KERBEROS) + throw Exception("Cannot get password hex hash for authentication type " + toString(type), ErrorCodes::LOGICAL_ERROR); + + String hex; + hex.resize(password_hash.size() * 2); + boost::algorithm::hex(password_hash.begin(), password_hash.end(), hex.data()); + return hex; +} + + +void AuthenticationData::setPasswordHashBinary(const Digest & hash) +{ + switch (type) + { + case AuthenticationType::PLAINTEXT_PASSWORD: + { + password_hash = hash; + return; + } + + case AuthenticationType::SHA256_PASSWORD: + { + if (hash.size() != 32) + throw Exception( + "Password hash for the 'SHA256_PASSWORD' authentication type has length " + std::to_string(hash.size()) + + " but must be exactly 32 bytes.", + ErrorCodes::BAD_ARGUMENTS); + password_hash = hash; + return; + } + + case AuthenticationType::DOUBLE_SHA1_PASSWORD: + { + if (hash.size() != 20) + throw Exception( + "Password hash for the 'DOUBLE_SHA1_PASSWORD' authentication type has length " + std::to_string(hash.size()) + + " but must be exactly 20 bytes.", + ErrorCodes::BAD_ARGUMENTS); + password_hash = hash; + return; + } + + case AuthenticationType::NO_PASSWORD: + case AuthenticationType::LDAP: + case AuthenticationType::KERBEROS: + throw Exception("Cannot specify password binary hash for authentication type " + toString(type), ErrorCodes::LOGICAL_ERROR); + + case AuthenticationType::MAX_TYPE: + break; + } + throw Exception("setPasswordHashBinary(): authentication type " + toString(type) + " not supported", ErrorCodes::NOT_IMPLEMENTED); +} + +} diff --git a/src/Access/Common/AuthenticationData.h b/src/Access/Common/AuthenticationData.h new file mode 100644 index 00000000000..5e47fbb9e15 --- /dev/null +++ b/src/Access/Common/AuthenticationData.h @@ -0,0 +1,103 @@ +#pragma once + +#include +#include + +namespace DB +{ + +enum class AuthenticationType +{ + /// User doesn't have to enter password. + NO_PASSWORD, + + /// Password is stored as is. + PLAINTEXT_PASSWORD, + + /// Password is encrypted in SHA256 hash. + SHA256_PASSWORD, + + /// SHA1(SHA1(password)). + /// This kind of hash is used by the `mysql_native_password` authentication plugin. + DOUBLE_SHA1_PASSWORD, + + /// Password is checked by a [remote] LDAP server. Connection will be made at each authentication attempt. + LDAP, + + /// Kerberos authentication performed through GSS-API negotiation loop. + KERBEROS, + + MAX_TYPE, +}; + +struct AuthenticationTypeInfo +{ + const char * const raw_name; + const String name; /// Lowercased with underscores, e.g. "sha256_password". + static const AuthenticationTypeInfo & get(AuthenticationType type_); +}; + + +/// Stores data for checking password when a user logins. +class AuthenticationData +{ +public: + using Digest = std::vector; + + AuthenticationData(AuthenticationType type_ = AuthenticationType::NO_PASSWORD) : type(type_) {} + AuthenticationData(const AuthenticationData & src) = default; + AuthenticationData & operator =(const AuthenticationData & src) = default; + AuthenticationData(AuthenticationData && src) = default; + AuthenticationData & operator =(AuthenticationData && src) = default; + + AuthenticationType getType() const { return type; } + + /// Sets the password and encrypt it using the authentication type set in the constructor. + void setPassword(const String & password_); + + /// Returns the password. Allowed to use only for Type::PLAINTEXT_PASSWORD. + String getPassword() const; + + /// Sets the password as a string of hexadecimal digits. + void setPasswordHashHex(const String & hash); + String getPasswordHashHex() const; + + /// Sets the password in binary form. + void setPasswordHashBinary(const Digest & hash); + const Digest & getPasswordHashBinary() const { return password_hash; } + + /// Sets the server name for authentication type LDAP. + const String & getLDAPServerName() const { return ldap_server_name; } + void setLDAPServerName(const String & name) { ldap_server_name = name; } + + /// Sets the realm name for authentication type KERBEROS. + const String & getKerberosRealm() const { return kerberos_realm; } + void setKerberosRealm(const String & realm) { kerberos_realm = realm; } + + friend bool operator ==(const AuthenticationData & lhs, const AuthenticationData & rhs); + friend bool operator !=(const AuthenticationData & lhs, const AuthenticationData & rhs) { return !(lhs == rhs); } + + struct Util + { + static Digest encodePlainText(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 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(const Digest & text) { return encodeSHA1(encodeSHA1(text)); } + }; + +private: + AuthenticationType type = AuthenticationType::NO_PASSWORD; + Digest password_hash; + String ldap_server_name; + String kerberos_realm; +}; + + +inline String toString(AuthenticationType type_) +{ + return AuthenticationTypeInfo::get(type_).raw_name; +} + +} diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index aade1fdd6f1..a0ad5d4ec79 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -1,6 +1,7 @@ #include -#include +#include #include +#include #include #include #include @@ -495,7 +496,7 @@ bool IAccessStorage::areCredentialsValidImpl( if (credentials.getUserName() != user.getName()) return false; - return user.authentication.areCredentialsValid(credentials, external_authenticators); + return Authentication::areCredentialsValid(credentials, user.auth_data, external_authenticators); } diff --git a/src/Access/LDAPAccessStorage.cpp b/src/Access/LDAPAccessStorage.cpp index 8a612982c79..182e391cc77 100644 --- a/src/Access/LDAPAccessStorage.cpp +++ b/src/Access/LDAPAccessStorage.cpp @@ -527,8 +527,8 @@ UUID LDAPAccessStorage::loginImpl(const Credentials & credentials, const Poco::N // User does not exist, so we create one, and will add it if authentication is successful. auto user = std::make_shared(); user->setName(credentials.getUserName()); - user->authentication = Authentication(Authentication::Type::LDAP); - user->authentication.setLDAPServerName(ldap_server_name); + user->auth_data = AuthenticationData(AuthenticationType::LDAP); + user->auth_data.setLDAPServerName(ldap_server_name); if (!isAddressAllowedImpl(*user, address)) throwAddressNotAllowed(address); @@ -555,8 +555,8 @@ UUID LDAPAccessStorage::getIDOfLoggedUserImpl(const String & user_name) const // User does not exist, so we create one, and add it pretending that the authentication is successful. auto user = std::make_shared(); user->setName(user_name); - user->authentication = Authentication(Authentication::Type::LDAP); - user->authentication.setLDAPServerName(ldap_server_name); + user->auth_data = AuthenticationData(AuthenticationType::LDAP); + user->auth_data.setLDAPServerName(ldap_server_name); LDAPClient::SearchResultsList external_roles; diff --git a/src/Access/User.cpp b/src/Access/User.cpp index e21b48e11a0..d7c7f5c7ada 100644 --- a/src/Access/User.cpp +++ b/src/Access/User.cpp @@ -9,7 +9,7 @@ bool User::equal(const IAccessEntity & other) const if (!IAccessEntity::equal(other)) return false; const auto & other_user = typeid_cast(other); - return (authentication == other_user.authentication) && (allowed_client_hosts == other_user.allowed_client_hosts) + return (auth_data == other_user.auth_data) && (allowed_client_hosts == other_user.allowed_client_hosts) && (access == other_user.access) && (granted_roles == other_user.granted_roles) && (default_roles == other_user.default_roles) && (settings == other_user.settings) && (grantees == other_user.grantees) && (default_database == other_user.default_database); } diff --git a/src/Access/User.h b/src/Access/User.h index 4bde967ba2c..34badd5f847 100644 --- a/src/Access/User.h +++ b/src/Access/User.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include @@ -15,7 +15,7 @@ namespace DB */ struct User : public IAccessEntity { - Authentication authentication; + AuthenticationData auth_data; AllowedClientHosts allowed_client_hosts = AllowedClientHosts::AnyHostTag{}; AccessRights access; GrantedRoles granted_roles; diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index 7c5baa92b27..2d202c5094d 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -75,18 +75,18 @@ namespace if (has_password_plaintext) { - user->authentication = Authentication{Authentication::PLAINTEXT_PASSWORD}; - user->authentication.setPassword(config.getString(user_config + ".password")); + user->auth_data = AuthenticationData{AuthenticationType::PLAINTEXT_PASSWORD}; + user->auth_data.setPassword(config.getString(user_config + ".password")); } else if (has_password_sha256_hex) { - user->authentication = Authentication{Authentication::SHA256_PASSWORD}; - user->authentication.setPasswordHashHex(config.getString(user_config + ".password_sha256_hex")); + user->auth_data = AuthenticationData{AuthenticationType::SHA256_PASSWORD}; + user->auth_data.setPasswordHashHex(config.getString(user_config + ".password_sha256_hex")); } else if (has_password_double_sha1_hex) { - user->authentication = Authentication{Authentication::DOUBLE_SHA1_PASSWORD}; - user->authentication.setPasswordHashHex(config.getString(user_config + ".password_double_sha1_hex")); + user->auth_data = AuthenticationData{AuthenticationType::DOUBLE_SHA1_PASSWORD}; + user->auth_data.setPasswordHashHex(config.getString(user_config + ".password_double_sha1_hex")); } else if (has_ldap) { @@ -98,15 +98,15 @@ namespace if (ldap_server_name.empty()) throw Exception("LDAP server name cannot be empty for user " + user_name + ".", ErrorCodes::BAD_ARGUMENTS); - user->authentication = Authentication{Authentication::LDAP}; - user->authentication.setLDAPServerName(ldap_server_name); + user->auth_data = AuthenticationData{AuthenticationType::LDAP}; + user->auth_data.setLDAPServerName(ldap_server_name); } else if (has_kerberos) { const auto realm = config.getString(user_config + ".kerberos.realm", ""); - user->authentication = Authentication{Authentication::KERBEROS}; - user->authentication.setKerberosRealm(realm); + user->auth_data = AuthenticationData{AuthenticationType::KERBEROS}; + user->auth_data.setKerberosRealm(realm); } const auto profile_name_config = user_config + ".profile"; diff --git a/src/Core/PostgreSQLProtocol.h b/src/Core/PostgreSQLProtocol.h index 7d34e23eed2..dd26bf41b4a 100644 --- a/src/Core/PostgreSQLProtocol.h +++ b/src/Core/PostgreSQLProtocol.h @@ -825,7 +825,7 @@ public: Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) = 0; - virtual Authentication::Type getType() const = 0; + virtual AuthenticationType getType() const = 0; virtual ~AuthenticationMethod() = default; }; @@ -842,9 +842,9 @@ public: return setPassword(user_name, "", session, mt, address); } - Authentication::Type getType() const override + AuthenticationType getType() const override { - return Authentication::Type::NO_PASSWORD; + return AuthenticationType::NO_PASSWORD; } }; @@ -873,9 +873,9 @@ public: ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT); } - Authentication::Type getType() const override + AuthenticationType getType() const override { - return Authentication::Type::PLAINTEXT_PASSWORD; + return AuthenticationType::PLAINTEXT_PASSWORD; } }; @@ -883,7 +883,7 @@ class AuthenticationManager { private: Poco::Logger * log = &Poco::Logger::get("AuthenticationManager"); - std::unordered_map> type_to_method = {}; + std::unordered_map> type_to_method = {}; public: AuthenticationManager(const std::vector> & auth_methods) @@ -900,7 +900,7 @@ public: Messaging::MessageTransport & mt, const Poco::Net::SocketAddress & address) { - const Authentication::Type user_auth_type = session.getAuthenticationTypeOrLogInFailure(user_name); + const AuthenticationType user_auth_type = session.getAuthenticationTypeOrLogInFailure(user_name); if (type_to_method.find(user_auth_type) != type_to_method.end()) { type_to_method[user_auth_type]->authenticate(user_name, session, mt, address); diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index e213ee59256..721d31bec48 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -31,8 +31,8 @@ namespace else if (query.names->size() == 1) user.setName(query.names->front()->toString()); - if (query.authentication) - user.authentication = *query.authentication; + if (query.auth_data) + user.auth_data = *query.auth_data; if (override_name && !override_name->host_pattern.empty()) { diff --git a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp index c68109002db..b7952324819 100644 --- a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp @@ -59,9 +59,9 @@ namespace query->default_roles = user.default_roles.toASTWithNames(*manager); } - if (user.authentication.getType() != Authentication::NO_PASSWORD) + if (user.auth_data.getType() != AuthenticationType::NO_PASSWORD) { - query->authentication = user.authentication; + query->auth_data = user.auth_data; query->show_password = attach_mode; /// We don't show password unless it's an ATTACH statement. } diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 020d297a6b9..ac8187980c3 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -271,12 +271,12 @@ Session::~Session() } } -Authentication::Type Session::getAuthenticationType(const String & user_name) const +AuthenticationType Session::getAuthenticationType(const String & user_name) const { - return global_context->getAccessControlManager().read(user_name)->authentication.getType(); + return global_context->getAccessControlManager().read(user_name)->auth_data.getType(); } -Authentication::Type Session::getAuthenticationTypeOrLogInFailure(const String & user_name) const +AuthenticationType Session::getAuthenticationTypeOrLogInFailure(const String & user_name) const { try { diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 82ba6a4a1fc..273ed88b9b5 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include @@ -14,7 +14,7 @@ namespace Poco::Net { class SocketAddress; } namespace DB { class Credentials; -class Authentication; +class AuthenticationData; struct NamedSessionData; class NamedSessionsStorage; struct User; @@ -41,10 +41,10 @@ public: Session& operator=(const Session &) = delete; /// Provides information about the authentication type of a specified user. - Authentication::Type getAuthenticationType(const String & user_name) const; + AuthenticationType getAuthenticationType(const String & user_name) const; /// Same as getAuthenticationType, but adds LoginFailure event in case of error. - Authentication::Type getAuthenticationTypeOrLogInFailure(const String & user_name) const; + AuthenticationType getAuthenticationTypeOrLogInFailure(const String & user_name) const; /// Sets the current user, checks the credentials and that the specified address is allowed to connect from. /// The function throws an exception if there is no such user or password is wrong. diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index a4847d4c492..6dac7ace662 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -45,7 +45,7 @@ auto eventTime() return std::make_pair(time_in_seconds(finish_time), time_in_microseconds(finish_time)); } -using AuthType = Authentication::Type; +using AuthType = AuthenticationType; using Interface = ClientInfo::Interface; void fillColumnArray(const Strings & data, IColumn & column) @@ -84,7 +84,7 @@ NamesAndTypesList SessionLogElement::getNamesAndTypes() {"Logout", static_cast(SESSION_LOGOUT)} }); -#define AUTH_TYPE_NAME_AND_VALUE(v) std::make_pair(Authentication::TypeInfo::get(v).raw_name, static_cast(v)) +#define AUTH_TYPE_NAME_AND_VALUE(v) std::make_pair(AuthenticationTypeInfo::get(v).raw_name, static_cast(v)) const auto identified_with_column = std::make_shared( DataTypeEnum8::Values { @@ -152,7 +152,7 @@ NamesAndTypesList SessionLogElement::getNamesAndTypes() void SessionLogElement::appendToBlock(MutableColumns & columns) const { assert(type >= SESSION_LOGIN_FAILURE && type <= SESSION_LOGOUT); - assert(user_identified_with >= Authentication::Type::NO_PASSWORD && user_identified_with <= Authentication::Type::MAX_TYPE); + assert(user_identified_with >= AuthenticationType::NO_PASSWORD && user_identified_with <= AuthenticationType::MAX_TYPE); size_t i = 0; @@ -214,8 +214,8 @@ void SessionLog::addLoginSuccess(const UUID & auth_id, std::optional ses { const auto user = access->getUser(); log_entry.user = user->getName(); - log_entry.user_identified_with = user->authentication.getType(); - log_entry.external_auth_server = user->authentication.getLDAPServerName(); + log_entry.user_identified_with = user->auth_data.getType(); + log_entry.external_auth_server = user->auth_data.getLDAPServerName(); } if (session_id) @@ -244,7 +244,7 @@ void SessionLog::addLoginFailure( log_entry.user = user; log_entry.auth_failure_reason = reason.message(); log_entry.client_info = info; - log_entry.user_identified_with = Authentication::Type::NO_PASSWORD; + log_entry.user_identified_with = AuthenticationType::NO_PASSWORD; add(log_entry); } diff --git a/src/Interpreters/SessionLog.h b/src/Interpreters/SessionLog.h index 794a8d3897f..93766d685e0 100644 --- a/src/Interpreters/SessionLog.h +++ b/src/Interpreters/SessionLog.h @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB { @@ -42,7 +42,7 @@ struct SessionLogElement Decimal64 event_time_microseconds{}; String user; - Authentication::Type user_identified_with = Authentication::Type::NO_PASSWORD; + AuthenticationType user_identified_with = AuthenticationType::NO_PASSWORD; String external_auth_server; Strings roles; Strings profiles; diff --git a/src/Parsers/Access/ASTCreateUserQuery.cpp b/src/Parsers/Access/ASTCreateUserQuery.cpp index 5fcec4c8b07..116deef43d9 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.cpp +++ b/src/Parsers/Access/ASTCreateUserQuery.cpp @@ -23,67 +23,67 @@ namespace } - void formatAuthentication(const Authentication & authentication, bool show_password, const IAST::FormatSettings & settings) + void formatAuthenticationData(const AuthenticationData & auth_data, bool show_password, const IAST::FormatSettings & settings) { - auto authentication_type = authentication.getType(); - if (authentication_type == Authentication::NO_PASSWORD) + auto auth_type = auth_data.getType(); + if (auth_type == AuthenticationType::NO_PASSWORD) { settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " NOT IDENTIFIED" << (settings.hilite ? IAST::hilite_none : ""); return; } - String authentication_type_name = Authentication::TypeInfo::get(authentication_type).name; + String auth_type_name = AuthenticationTypeInfo::get(auth_type).name; String by_keyword = "BY"; std::optional by_value; if ( show_password || - authentication_type == Authentication::LDAP || - authentication_type == Authentication::KERBEROS + auth_type == AuthenticationType::LDAP || + auth_type == AuthenticationType::KERBEROS ) { - switch (authentication_type) + switch (auth_type) { - case Authentication::PLAINTEXT_PASSWORD: + case AuthenticationType::PLAINTEXT_PASSWORD: { - by_value = authentication.getPassword(); + by_value = auth_data.getPassword(); break; } - case Authentication::SHA256_PASSWORD: + case AuthenticationType::SHA256_PASSWORD: { - authentication_type_name = "sha256_hash"; - by_value = authentication.getPasswordHashHex(); + auth_type_name = "sha256_hash"; + by_value = auth_data.getPasswordHashHex(); break; } - case Authentication::DOUBLE_SHA1_PASSWORD: + case AuthenticationType::DOUBLE_SHA1_PASSWORD: { - authentication_type_name = "double_sha1_hash"; - by_value = authentication.getPasswordHashHex(); + auth_type_name = "double_sha1_hash"; + by_value = auth_data.getPasswordHashHex(); break; } - case Authentication::LDAP: + case AuthenticationType::LDAP: { by_keyword = "SERVER"; - by_value = authentication.getLDAPServerName(); + by_value = auth_data.getLDAPServerName(); break; } - case Authentication::KERBEROS: + case AuthenticationType::KERBEROS: { by_keyword = "REALM"; - const auto & realm = authentication.getKerberosRealm(); + const auto & realm = auth_data.getKerberosRealm(); if (!realm.empty()) by_value = realm; break; } - case Authentication::NO_PASSWORD: [[fallthrough]]; - case Authentication::MAX_TYPE: - throw Exception("AST: Unexpected authentication type " + toString(authentication_type), ErrorCodes::LOGICAL_ERROR); + case AuthenticationType::NO_PASSWORD: [[fallthrough]]; + case AuthenticationType::MAX_TYPE: + throw Exception("AST: Unexpected authentication type " + toString(auth_type), ErrorCodes::LOGICAL_ERROR); } } - settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " IDENTIFIED WITH " << authentication_type_name + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " IDENTIFIED WITH " << auth_type_name << (settings.hilite ? IAST::hilite_none : ""); if (by_value) @@ -258,8 +258,8 @@ void ASTCreateUserQuery::formatImpl(const FormatSettings & format, FormatState & if (!new_name.empty()) formatRenameTo(new_name, format); - if (authentication) - formatAuthentication(*authentication, show_password, format); + if (auth_data) + formatAuthenticationData(*auth_data, show_password, format); if (hosts) formatHosts(nullptr, *hosts, format); diff --git a/src/Parsers/Access/ASTCreateUserQuery.h b/src/Parsers/Access/ASTCreateUserQuery.h index 183acb6d3fe..92db71e8581 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.h +++ b/src/Parsers/Access/ASTCreateUserQuery.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include @@ -44,7 +44,7 @@ public: std::shared_ptr names; String new_name; - std::optional authentication; + std::optional auth_data; bool show_password = true; /// formatImpl() will show the password or hash. std::optional hosts; diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index 421730b9ec5..8dcbfb46692 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -34,20 +34,20 @@ namespace } - bool parseAuthentication(IParserBase::Pos & pos, Expected & expected, Authentication & authentication) + bool parseAuthenticationData(IParserBase::Pos & pos, Expected & expected, AuthenticationData & auth_data) { return IParserBase::wrapParseImpl(pos, [&] { if (ParserKeyword{"NOT IDENTIFIED"}.ignore(pos, expected)) { - authentication = Authentication{Authentication::NO_PASSWORD}; + auth_data = AuthenticationData{AuthenticationType::NO_PASSWORD}; return true; } if (!ParserKeyword{"IDENTIFIED"}.ignore(pos, expected)) return false; - std::optional type; + std::optional type; bool expect_password = false; bool expect_hash = false; bool expect_ldap_server_name = false; @@ -55,17 +55,17 @@ namespace if (ParserKeyword{"WITH"}.ignore(pos, expected)) { - for (auto check_type : collections::range(Authentication::MAX_TYPE)) + for (auto check_type : collections::range(AuthenticationType::MAX_TYPE)) { - if (ParserKeyword{Authentication::TypeInfo::get(check_type).raw_name}.ignore(pos, expected)) + if (ParserKeyword{AuthenticationTypeInfo::get(check_type).raw_name}.ignore(pos, expected)) { type = check_type; - if (check_type == Authentication::LDAP) + if (check_type == AuthenticationType::LDAP) expect_ldap_server_name = true; - else if (check_type == Authentication::KERBEROS) + else if (check_type == AuthenticationType::KERBEROS) expect_kerberos_realm = true; - else if (check_type != Authentication::NO_PASSWORD) + else if (check_type != AuthenticationType::NO_PASSWORD) expect_password = true; break; @@ -76,12 +76,12 @@ namespace { if (ParserKeyword{"SHA256_HASH"}.ignore(pos, expected)) { - type = Authentication::SHA256_PASSWORD; + type = AuthenticationType::SHA256_PASSWORD; expect_hash = true; } else if (ParserKeyword{"DOUBLE_SHA1_HASH"}.ignore(pos, expected)) { - type = Authentication::DOUBLE_SHA1_PASSWORD; + type = AuthenticationType::DOUBLE_SHA1_PASSWORD; expect_hash = true; } else @@ -91,7 +91,7 @@ namespace if (!type) { - type = Authentication::SHA256_PASSWORD; + type = AuthenticationType::SHA256_PASSWORD; expect_password = true; } @@ -124,15 +124,15 @@ namespace } } - authentication = Authentication{*type}; + auth_data = AuthenticationData{*type}; if (expect_password) - authentication.setPassword(value); + auth_data.setPassword(value); else if (expect_hash) - authentication.setPasswordHashHex(value); + auth_data.setPasswordHashHex(value); else if (expect_ldap_server_name) - authentication.setLDAPServerName(value); + auth_data.setLDAPServerName(value); else if (expect_kerberos_realm) - authentication.setKerberosRealm(value); + auth_data.setKerberosRealm(value); return true; }); @@ -360,7 +360,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec auto names_ref = names->names; String new_name; - std::optional authentication; + std::optional auth_data; std::optional hosts; std::optional add_hosts; std::optional remove_hosts; @@ -372,12 +372,12 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec while (true) { - if (!authentication) + if (!auth_data) { - Authentication new_authentication; - if (parseAuthentication(pos, expected, new_authentication)) + AuthenticationData new_auth_data; + if (parseAuthenticationData(pos, expected, new_auth_data)) { - authentication = std::move(new_authentication); + auth_data = std::move(new_auth_data); continue; } } @@ -460,7 +460,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query->cluster = std::move(cluster); query->names = std::move(names); query->new_name = std::move(new_name); - query->authentication = std::move(authentication); + query->auth_data = std::move(auth_data); query->hosts = std::move(hosts); query->add_hosts = std::move(add_hosts); query->remove_hosts = std::move(remove_hosts); diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 32ff2d854cd..b2a00055532 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include #include diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 2ba5359ca31..2f28095f976 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -243,7 +243,7 @@ void MySQLHandler::authenticate(const String & user_name, const String & auth_pl try { // For compatibility with JavaScript MySQL client, Native41 authentication plugin is used when possible (if password is specified using double SHA1). Otherwise SHA256 plugin is used. - if (session->getAuthenticationTypeOrLogInFailure(user_name) == DB::Authentication::SHA256_PASSWORD) + if (session->getAuthenticationTypeOrLogInFailure(user_name) == DB::AuthenticationType::SHA256_PASSWORD) { authPluginSSL(); } diff --git a/src/Server/PostgreSQLHandler.h b/src/Server/PostgreSQLHandler.h index ca74b78a109..ded9616296a 100644 --- a/src/Server/PostgreSQLHandler.h +++ b/src/Server/PostgreSQLHandler.h @@ -6,6 +6,10 @@ #include #include "IServer.h" +#if !defined(ARCADIA_BUILD) +# include +#endif + #if USE_SSL # include #endif diff --git a/src/Server/PostgreSQLHandlerFactory.h b/src/Server/PostgreSQLHandlerFactory.h index 4550e9ee8e9..9103cbaad90 100644 --- a/src/Server/PostgreSQLHandlerFactory.h +++ b/src/Server/PostgreSQLHandlerFactory.h @@ -6,6 +6,10 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include +#endif + namespace DB { diff --git a/src/Storages/System/StorageSystemUsers.cpp b/src/Storages/System/StorageSystemUsers.cpp index bfce1d50160..e72c1a95385 100644 --- a/src/Storages/System/StorageSystemUsers.cpp +++ b/src/Storages/System/StorageSystemUsers.cpp @@ -25,8 +25,8 @@ namespace DataTypeEnum8::Values getAuthenticationTypeEnumValues() { DataTypeEnum8::Values enum_values; - for (auto type : collections::range(Authentication::MAX_TYPE)) - enum_values.emplace_back(Authentication::TypeInfo::get(type).name, static_cast(type)); + for (auto type : collections::range(AuthenticationType::MAX_TYPE)) + enum_values.emplace_back(AuthenticationTypeInfo::get(type).name, static_cast(type)); return enum_values; } } @@ -91,7 +91,7 @@ void StorageSystemUsers::fillData(MutableColumns & res_columns, ContextPtr conte auto add_row = [&](const String & name, const UUID & id, const String & storage_name, - const Authentication & authentication, + const AuthenticationData & auth_data, const AllowedClientHosts & allowed_hosts, const RolesOrUsersSet & default_roles, const RolesOrUsersSet & grantees, @@ -100,19 +100,19 @@ void StorageSystemUsers::fillData(MutableColumns & res_columns, ContextPtr conte column_name.insertData(name.data(), name.length()); column_id.push_back(id.toUnderType()); column_storage.insertData(storage_name.data(), storage_name.length()); - column_auth_type.push_back(static_cast(authentication.getType())); + column_auth_type.push_back(static_cast(auth_data.getType())); if ( - authentication.getType() == Authentication::Type::LDAP || - authentication.getType() == Authentication::Type::KERBEROS + auth_data.getType() == AuthenticationType::LDAP || + auth_data.getType() == AuthenticationType::KERBEROS ) { Poco::JSON::Object auth_params_json; - if (authentication.getType() == Authentication::Type::LDAP) - auth_params_json.set("server", authentication.getLDAPServerName()); - else if (authentication.getType() == Authentication::Type::KERBEROS) - auth_params_json.set("realm", authentication.getKerberosRealm()); + if (auth_data.getType() == AuthenticationType::LDAP) + auth_params_json.set("server", auth_data.getLDAPServerName()); + else if (auth_data.getType() == AuthenticationType::KERBEROS) + auth_params_json.set("realm", auth_data.getKerberosRealm()); std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); @@ -197,7 +197,7 @@ void StorageSystemUsers::fillData(MutableColumns & res_columns, ContextPtr conte if (!storage) continue; - add_row(user->getName(), id, storage->getStorageName(), user->authentication, user->allowed_client_hosts, + add_row(user->getName(), id, storage->getStorageName(), user->auth_data, user->allowed_client_hosts, user->default_roles, user->grantees, user->default_database); } } From 4d2be6d8bb3f701033ab1add9510360519e621f9 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Mon, 1 Nov 2021 19:39:35 +0300 Subject: [PATCH 916/919] Add database engines MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Добавил движки баз данных. --- docs/en/sql-reference/statements/create/database.md | 2 +- docs/ru/sql-reference/statements/create/database.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/create/database.md b/docs/en/sql-reference/statements/create/database.md index 44c9153175c..787bbc02346 100644 --- a/docs/en/sql-reference/statements/create/database.md +++ b/docs/en/sql-reference/statements/create/database.md @@ -26,7 +26,7 @@ ClickHouse creates the `db_name` database on all the servers of a specified clus ### ENGINE {#engine} -By default, ClickHouse uses its own [Atomic](../../../engines/database-engines/atomic.md) database engine. There are also [Lazy](../../../engines/database-engines/lazy.md), MySQL, PostgresSQL, MaterializedMySQL, MaterializedPostgreSQL, Dictionary, Replicated . +By default, ClickHouse uses its own [Atomic](../../../engines/database-engines/atomic.md) database engine. There are also [Lazy](../../../engines/database-engines/lazy.md), [MySQL](../../../engines/database-engines/mysql.md), [PostgresSQL](../../../engines/database-engines/postgresql.md), [MaterializedMySQL](../../../engines/database-engines/materialized-mysql.md), [MaterializedPostgreSQL](../../../engines/database-engines/materialized-postgresql.md), [Replicated](../../../engines/database-engines/replicated.md), [SQLite](../../../engines/database-engines/sqlite.md). ### COMMENT {#comment} diff --git a/docs/ru/sql-reference/statements/create/database.md b/docs/ru/sql-reference/statements/create/database.md index 6ba3db32419..b697f4caada 100644 --- a/docs/ru/sql-reference/statements/create/database.md +++ b/docs/ru/sql-reference/statements/create/database.md @@ -26,7 +26,7 @@ ClickHouse создаёт базу данных с именем `db_name` на ### ENGINE {#engine} -[MySQL](../../../engines/database-engines/mysql.md) позволяет получать данные с удаленного сервера MySQL. По умолчанию ClickHouse использует собственный [движок баз данных](../../../engines/database-engines/index.md). Есть также движок баз данных [lazy](../../../engines/database-engines/lazy.md). +По умолчанию ClickHouse использует собственный движок баз данных [Atomic](../../../engines/database-engines/atomic.md). Есть также движки баз данных [Lazy](../../../engines/database-engines/lazy.md), [MySQL](../../../engines/database-engines/mysql.md), [PostgresSQL](../../../engines/database-engines/postgresql.md), [MaterializedMySQL](../../../engines/database-engines/materialized-mysql.md), [MaterializedPostgreSQL](../../../engines/database-engines/materialized-postgresql.md), [Replicated](../../../engines/database-engines/replicated.md), [SQLite](../../../engines/database-engines/sqlite.md). ### COMMENT {#comment} From 38f7d1ebc27b6ca0b6a10882b6d3c9df9e310abe Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 1 Nov 2021 22:59:57 +0300 Subject: [PATCH 917/919] Update KeyCondition.cpp --- src/Storages/MergeTree/KeyCondition.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 07052a69f07..dda7f235d97 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -334,7 +334,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map { out.function = RPNElement::FUNCTION_IS_NULL; // isNull means +Inf (NULLS_LAST) or -Inf (NULLS_FIRST), - // which is eqivalent to not in Range (-Inf, +Inf) + // which is equivalent to not in Range (-Inf, +Inf) out.range = Range(); return true; } From f880c8c2e2f94c0ac9e97f00b583284504c75246 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 2 Nov 2021 00:52:35 +0300 Subject: [PATCH 918/919] Auto version update to [21.11.1.8636] [54456] --- cmake/autogenerated_versions.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index dd6ead7d97f..273d6977a5d 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -6,7 +6,7 @@ SET(VERSION_REVISION 54456) SET(VERSION_MAJOR 21) SET(VERSION_MINOR 11) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH 7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7) -SET(VERSION_DESCRIBE v21.11.1.1-prestable) -SET(VERSION_STRING 21.11.1.1) +SET(VERSION_GITHASH 503a418dedf0011e9040c3a1b6913e0b5488be4c) +SET(VERSION_DESCRIBE v21.11.1.8636-prestable) +SET(VERSION_STRING 21.11.1.8636) # end of autochange From 4289336c27d24ea68a36b623562c8ce775fd7ac8 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 2 Nov 2021 00:56:45 +0300 Subject: [PATCH 919/919] Auto version update to [21.12.1.1] [54457] --- cmake/autogenerated_versions.txt | 8 +-- debian/changelog | 4 +- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- .../StorageSystemContributors.generated.cpp | 54 +++++++++++++++++++ 6 files changed, 63 insertions(+), 9 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 273d6977a5d..f13110d7179 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -2,11 +2,11 @@ # NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. -SET(VERSION_REVISION 54456) +SET(VERSION_REVISION 54457) SET(VERSION_MAJOR 21) -SET(VERSION_MINOR 11) +SET(VERSION_MINOR 12) SET(VERSION_PATCH 1) SET(VERSION_GITHASH 503a418dedf0011e9040c3a1b6913e0b5488be4c) -SET(VERSION_DESCRIBE v21.11.1.8636-prestable) -SET(VERSION_STRING 21.11.1.8636) +SET(VERSION_DESCRIBE v21.12.1.1-prestable) +SET(VERSION_STRING 21.12.1.1) # end of autochange diff --git a/debian/changelog b/debian/changelog index 460424bdb36..a2709485e44 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (21.11.1.1) unstable; urgency=low +clickhouse (21.12.1.1) unstable; urgency=low * Modified source code - -- clickhouse-release Thu, 09 Sep 2021 12:03:26 +0300 + -- clickhouse-release Tue, 02 Nov 2021 00:56:42 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index 926014da9e6..6f9a957852e 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb https://repo.clickhouse.com/deb/stable/ main/" -ARG version=21.11.1.* +ARG version=21.12.1.* RUN sed -i 's|http://archive|http://ru.archive|g' /etc/apt/sources.list diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index 48b59d1e754..04842e7a3de 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:20.04 ARG repository="deb https://repo.clickhouse.com/deb/stable/ main/" -ARG version=21.11.1.* +ARG version=21.12.1.* ARG gosu_ver=1.10 # set non-empty deb_location_url url to create a docker image diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index fc8c2ebbe6e..76967da9f9a 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb https://repo.clickhouse.com/deb/stable/ main/" -ARG version=21.11.1.* +ARG version=21.12.1.* RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index a097e08871d..c33fa6cad44 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -26,6 +26,7 @@ const char * auto_contributors[] { "Aleksandra (Ася)", "Aleksandr Karo", "Aleksandrov Vladimir", + "Aleksandr Shalimov", "alekseik1", "Aleksei Levushkin", "Aleksei Semiglazov", @@ -36,6 +37,7 @@ const char * auto_contributors[] { "Alexander Avdonkin", "Alexander Bezpiatov", "Alexander Burmak", + "Alexander Chashnikov", "Alexander Ermolaev", "Alexander Gololobov", "Alexander GQ Gerasiov", @@ -101,9 +103,11 @@ const char * auto_contributors[] { "ana-uvarova", "AnaUvarova", "Andr0901", + "andrc1901", "Andreas Hunkeler", "AndreevDm", "Andrei Bodrov", + "Andrei Ch", "Andrei Chulkov", "andrei-karpliuk", "Andrei Nekrashevich", @@ -127,6 +131,7 @@ const char * auto_contributors[] { "Anmol Arora", "Anna", "Anna Shakhova", + "anneji", "anneji-dev", "annvsh", "anrodigina", @@ -181,6 +186,7 @@ const char * auto_contributors[] { "BanyRule", "Baudouin Giard", "BayoNet", + "bbkas", "benamazing", "benbiti", "Benjamin Naecker", @@ -190,6 +196,7 @@ const char * auto_contributors[] { "bharatnc", "Big Elephant", "Bill", + "BiteTheDDDDt", "BlahGeek", "blazerer", "bluebirddm", @@ -220,6 +227,7 @@ const char * auto_contributors[] { "Chao Wang", "chasingegg", "chengy8934", + "chenjian", "chenqi", "chenxing-xc", "chenxing.xc", @@ -232,7 +240,9 @@ const char * auto_contributors[] { "Ciprian Hacman", "Clement Rodriguez", "Clément Rodriguez", + "ClickHouse Admin", "cn-ds", + "Cody Baker", "Colum", "comunodi", "Constantin S. Pan", @@ -343,6 +353,7 @@ const char * auto_contributors[] { "fastio", "favstovol", "FawnD2", + "Federico Ceratto", "FeehanG", "feihengye", "felixoid", @@ -372,6 +383,7 @@ const char * auto_contributors[] { "fuwhu", "Fu Zhe", "fuzhe1989", + "fuzzERot", "Gagan Arneja", "Gao Qiang", "g-arslan", @@ -407,6 +419,7 @@ const char * auto_contributors[] { "heng zhao", "hermano", "hexiaoting", + "hhell", "Hiroaki Nakamura", "hotid", "huangzhaowei", @@ -456,6 +469,7 @@ const char * auto_contributors[] { "Ivan Kush", "Ivan Kushnarenko", "Ivan Lezhankin", + "Ivan Milov", "Ivan Remen", "Ivan Starkov", "ivanzhukov", @@ -470,6 +484,7 @@ const char * auto_contributors[] { "jasine", "Jason", "Jason Keirstead", + "jasperzhu", "javartisan", "javi", "javi santana", @@ -483,6 +498,7 @@ const char * auto_contributors[] { "Jiang Tao", "jianmei zhang", "jkuklis", + "João Figueiredo", "Jochen Schalanda", "John", "John Hummel", @@ -492,9 +508,11 @@ const char * auto_contributors[] { "jyz0309", "Kang Liu", "Karl Pietrzak", + "karnevil13", "keenwolf", "Keiji Yoshida", "Ken Chen", + "Ken MacInnis", "Kevin Chiang", "Kevin Michel", "kevin wan", @@ -509,6 +527,7 @@ const char * auto_contributors[] { "KochetovNicolai", "kolsys", "Konstantin Grabar", + "Konstantin Ilchenko", "Konstantin Lebedev", "Konstantin Malanchev", "Konstantin Podshumok", @@ -529,6 +548,7 @@ const char * auto_contributors[] { "l1tsolaiki", "lalex", "Latysheva Alexandra", + "laurieliyang", "lehasm", "Léo Ercolanelli", "Leonardo Cecchi", @@ -539,6 +559,9 @@ const char * auto_contributors[] { "levushkin aleksej", "levysh", "Lewinma", + "lhuang0928", + "lhuang09287750", + "liang.huang", "liangqian", "libenwang", "lichengxiang", @@ -593,6 +616,7 @@ const char * auto_contributors[] { "mastertheknife", "Matthew Peveler", "Matwey V. Kornilov", + "Mátyás Jani", "Max", "Max Akhmedov", "Max Bruce", @@ -613,11 +637,13 @@ const char * auto_contributors[] { "maxkuzn", "maxulan", "Max Vetrov", + "MaxWk", "Mc.Spring", "mehanizm", "MeiK", "melin", "memo", + "Memo", "meo", "meoww-bot", "mergify[bot]", @@ -636,6 +662,7 @@ const char * auto_contributors[] { "Miguel Fernández", "miha-g", "Mihail Fandyushin", + "mikael", "Mikahil Nacharov", "Mike", "Mike F", @@ -658,6 +685,8 @@ const char * auto_contributors[] { "millb", "Misko Lee", "mnkonkova", + "mo-avatar", + "Mohamad Fadhil", "Mohammad Hossein Sekhavat", "morty", "moscas", @@ -698,6 +727,7 @@ const char * auto_contributors[] { "nikitamikhaylov", "Nikita Mikhaylov", "Nikita Orlov", + "Nikita Tikhomirov", "Nikita Vasilev", "Nikolai Kochetov", "Nikolai Sorokin", @@ -739,15 +769,18 @@ const char * auto_contributors[] { "Paramtamtam", "Patrick Zippenfenig", "Pavel", + "Pavel Cheremushkin", "Pavel Kartaviy", "Pavel Kartavyy", "Pavel Kovalenko", "Pavel Kruglov", "Pavel Litvinenko", + "Pavel Medvedev", "Pavel Patrin", "Pavel Yakunin", "Pavlo Bashynskiy", "Pawel Rog", + "pawelsz-rb", "pdv-ru", "Peng Jian", "Persiyanov Dmitriy Andreevich", @@ -770,6 +803,7 @@ const char * auto_contributors[] { "qianmoQ", "quid", "Quid37", + "quoctan132", "r1j1k", "Rafael David Tinoco", "rainbowsysu", @@ -789,6 +823,7 @@ const char * auto_contributors[] { "robot-clickhouse", "robot-metrika-test", "rodrigargar", + "Rohit Agarwal", "Romain Neutron", "roman", "Roman Bug", @@ -816,6 +851,7 @@ const char * auto_contributors[] { "Saulius Valatka", "sdk2", "Sean Haynes", + "Sébastien", "Sébastien Launay", "serebrserg", "Sergei Bocharov", @@ -840,10 +876,12 @@ const char * auto_contributors[] { "sev7e0", "SevaCode", "sevirov", + "Seyed Mehrshad Hosseini", "sfod", "shangshujie", "shedx", "Sherry Wang", + "Shoh Jahon", "Silviu Caragea", "Simeon Emanuilov", "Simon Liu", @@ -887,16 +925,23 @@ const char * auto_contributors[] { "Taleh Zaliyev", "Tangaev", "tao jiang", + "Tatiana", "Tatiana Kirillova", "tavplubix", "TCeason", + "Teja", + "Teja Srivastasa", "Tema Novikov", "templarzq", + "Tentoshka", "terrylin", "The-Alchemist", "Thomas Berdy", + "Thom O'Connor", + "tianzhou", "Tiaonmmn", "tiger.yan", + "Tigran Khudaverdyan", "tison", "TiunovNN", "Tobias Adamson", @@ -934,7 +979,9 @@ const char * auto_contributors[] { "Veloman Yunkan", "Veniamin Gvozdikov", "Veselkov Konstantin", + "vesslanjin", "vgocoder", + "Viachaslau Boben", "vic", "vicdashkov", "vicgao", @@ -950,6 +997,7 @@ const char * auto_contributors[] { "Vitaliy Zakaznikov", "Vitaly", "Vitaly Baranov", + "Vitaly Orlov", "Vitaly Samigullin", "Vitaly Stoyan", "vitstn", @@ -980,6 +1028,7 @@ const char * auto_contributors[] { "vzakaznikov", "wangchao", "Wang Fenjin", + "WangZengrui", "weeds085490", "Weiqing Xu", "William Shallum", @@ -994,6 +1043,7 @@ const char * auto_contributors[] { "Yangkuan Liu", "yangshuai", "Yatsishin Ilya", + "yeer", "Yegor Andreenko", "Yegor Levankov", "ygrek", @@ -1023,6 +1073,7 @@ const char * auto_contributors[] { "ywill3", "zamulla", "zhang2014", + "zhanglistar", "zhangshengyu", "zhangxiao018", "zhangxiao871", @@ -1051,6 +1102,7 @@ const char * auto_contributors[] { "Смитюх Вячеслав", "Сундуков Алексей", "万康", + "凌涛", "吴健", "小路", "张中南", @@ -1058,8 +1110,10 @@ const char * auto_contributors[] { "张风啸", "徐炘", "曲正鹏", + "木木夕120", "未来星___费", "极客青年", + "枢木", "董海镔", "谢磊", "贾顺名(Jarvis)",