From 030e5e625ea4143d98c3fd56d522d862241c8474 Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Wed, 5 May 2021 17:11:56 +0200 Subject: [PATCH 001/396] add Map combinator --- .../AggregateFunctionMap.cpp | 125 +++++++++++ src/AggregateFunctions/AggregateFunctionMap.h | 206 ++++++++++++++++++ .../AggregateFunctionSumMap.cpp | 19 +- .../AggregateFunctionSumMap.h | 15 +- .../registerAggregateFunctions.cpp | 2 + 5 files changed, 357 insertions(+), 10 deletions(-) create mode 100644 src/AggregateFunctions/AggregateFunctionMap.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionMap.h diff --git a/src/AggregateFunctions/AggregateFunctionMap.cpp b/src/AggregateFunctions/AggregateFunctionMap.cpp new file mode 100644 index 00000000000..581587e91d3 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionMap.cpp @@ -0,0 +1,125 @@ +#include "AggregateFunctionMap.h" +#include "AggregateFunctions/AggregateFunctionCombinatorFactory.h" +#include "Functions/FunctionHelpers.h" + +namespace DB +{ +class AggregateFunctionCombinatorMap final : public IAggregateFunctionCombinator +{ +public: + String getName() const override { return "Map"; } + + DataTypes transformArguments(const DataTypes & arguments) const override + { + if (arguments.empty()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Incorrect number of arguments for aggregate function with " + getName() + " suffix"); + + const auto * map_type = checkAndGetDataType(arguments[0].get()); + if (map_type) + return DataTypes({map_type->getValueType()}); + + // we need this part just to pass to redirection for mapped arrays + const auto * tup_type = checkAndGetDataType(arguments[0].get()); + if (tup_type) + { + const auto * val_array_type = checkAndGetDataType(tup_type->getElements()[1].get()); + if (val_array_type) + return DataTypes({val_array_type->getNestedType()}); + } + + if (arguments.size() >= 2) + { + const auto * val_array_type = checkAndGetDataType(arguments[1].get()); + if (val_array_type) + return DataTypes({val_array_type->getNestedType()}); + } + + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function " + getName() + " requires map as argument"); + } + + AggregateFunctionPtr transformAggregateFunction( + const AggregateFunctionPtr & nested_function, + const AggregateFunctionProperties &, + const DataTypes & arguments, + const Array & params) const override + { + const auto * map_type = checkAndGetDataType(arguments[0].get()); + if (map_type) + { + auto key_type_id = map_type->getKeyType()->getTypeId(); + + switch (key_type_id) + { + case TypeIndex::Enum8: + case TypeIndex::Int8: + return std::make_shared>(nested_function, arguments); + case TypeIndex::Enum16: + case TypeIndex::Int16: + return std::make_shared>(nested_function, arguments); + case TypeIndex::Int32: + return std::make_shared>(nested_function, arguments); + case TypeIndex::Int64: + return std::make_shared>(nested_function, arguments); + case TypeIndex::UInt8: + return std::make_shared>(nested_function, arguments); + case TypeIndex::Date: + case TypeIndex::UInt16: + return std::make_shared>(nested_function, arguments); + case TypeIndex::DateTime: + case TypeIndex::UInt32: + return std::make_shared>(nested_function, arguments); + case TypeIndex::UInt64: + return std::make_shared>(nested_function, arguments); + case TypeIndex::UUID: + return std::make_shared>(nested_function, arguments); + case TypeIndex::FixedString: + case TypeIndex::String: + return std::make_shared>(nested_function, arguments); + default: + throw Exception{"Illegal columns in arguments for combinator " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + } + } + else if (!arguments.empty()) + { + // check if we got tuple of arrays or just arrays and if so, try to redirect to sum/min/max-MappedArrays to implement old behavior + auto nested_func_name = nested_function->getName(); + if (nested_func_name == "sum" || nested_func_name == "min" || nested_func_name == "max") + { + bool match; + const auto * tup_type = checkAndGetDataType(arguments[0].get()); + auto check_func = [](DataTypePtr t) { + return t->getTypeId() == TypeIndex::Array; + }; + + if (tup_type) + { + const auto & types = tup_type->getElements(); + match = arguments.size() == 1 && types.size() >= 2 && std::all_of(types.begin(), types.end(), check_func); + } + else + { + // sumMappedArrays and others support more than 2 mapped arrays + match = arguments.size() >= 2 && std::all_of(arguments.begin(), arguments.end(), check_func); + } + + if (match) + { + AggregateFunctionProperties out_properties; + auto & aggr_func_factory = AggregateFunctionFactory::instance(); + return aggr_func_factory.get(nested_func_name + "MappedArrays", arguments, params, out_properties); + } + } + } + + throw Exception{"Illegal columns in arguments for combinator " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + } +}; + +void registerAggregateFunctionCombinatorMap(AggregateFunctionCombinatorFactory & factory) +{ + factory.registerCombinator(std::make_shared()); +} + +} diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h new file mode 100644 index 00000000000..02715b40c90 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -0,0 +1,206 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "AggregateFunctions/AggregateFunctionFactory.h" + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +template +struct AggregateFunctionMapCombinatorData +{ + std::unordered_map merged_maps; + + static void writeKey(KeyType key, WriteBuffer & buf) { writeBinary(key, buf); } + static void readKey(KeyType & key, ReadBuffer & buf) { readBinary(key, buf); } +}; + +template <> +struct AggregateFunctionMapCombinatorData +{ + std::unordered_map merged_maps; + + static void writeKey(String key, WriteBuffer & buf) { writeString(key, buf); } + static void readKey(String & key, ReadBuffer & buf) { readString(key, buf); } +}; + +template +class AggregateFunctionMap final + : public IAggregateFunctionDataHelper, AggregateFunctionMap> +{ +private: + DataTypePtr key_type; + AggregateFunctionPtr nested_func; + using Base = IAggregateFunctionDataHelper, AggregateFunctionMap>; + +public: + AggregateFunctionMap(AggregateFunctionPtr nested, const DataTypes & types) : Base(types, nested->getParameters()), nested_func(nested) + { + if (types.empty()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function " + getName() + " require at least one argument"); + + const auto * map_type = checkAndGetDataType(types[0].get()); + if (!map_type) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function " + getName() + " requires map as argument"); + + key_type = map_type->getKeyType(); + } + + String getName() const override { return nested_func->getName() + "Map"; } + + DataTypePtr getReturnType() const override { return std::make_shared(DataTypes{key_type, nested_func->getReturnType()}); } + + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override + { + const auto & map_column = assert_cast(*columns[0]); + const auto & map_nested_tuple = map_column.getNestedData(); + const IColumn::Offsets & map_array_offsets = map_column.getNestedColumn().getOffsets(); + + const size_t offset = map_array_offsets[row_num - 1]; + const size_t size = (map_array_offsets[row_num] - offset); + + const auto & key_column = map_nested_tuple.getColumn(0); + const auto & val_column = map_nested_tuple.getColumn(1); + + auto & merged_maps = this->data(place).merged_maps; + + for (size_t i = 0; i < size; ++i) + { + KeyType key; + if constexpr (std::is_same::value) + { + key = key_column.operator[](offset + i).get(); + } + else + { + key = assert_cast &>(key_column).getData()[offset + i]; + } + + AggregateDataPtr nested_place; + auto it = merged_maps.find(key); + + if (it == merged_maps.end()) + { + // create a new place for each key + nested_place = arena->alloc(sizeof(AggregateDataPtr)); + nested_func->create(nested_place); + merged_maps.emplace(key, nested_place); + } + else + nested_place = it->second; + + const IColumn * nested_columns[1] = {&val_column}; + nested_func->add(nested_place, nested_columns, offset + i, arena); + } + } + + void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override + { + auto & merged_maps = this->data(place).merged_maps; + const auto & rhs_maps = this->data(rhs).merged_maps; + + for (const auto & elem : rhs_maps) + { + const auto & it = merged_maps.find(elem.first); + + if (it != merged_maps.end()) + { + nested_func->merge(it->second, elem.second, arena); + } + else + merged_maps[elem.first] = elem.second; + } + } + + void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override + { + auto & merged_maps = this->data(place).merged_maps; + writeVarUInt(merged_maps.size(), buf); + + for (const auto & elem : merged_maps) + { + this->data(place).writeKey(elem.first, buf); + nested_func->serialize(elem.second, buf); + } + } + + void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override + { + auto & merged_maps = this->data(place).merged_maps; + size_t size; + + readVarUInt(size, buf); + for (size_t i = 0; i < size; ++i) + { + KeyType key; + AggregateDataPtr nested_place; + + this->data(place).readKey(key, buf); + nested_place = arena->alloc(sizeof(AggregateDataPtr)); + nested_func->create(nested_place); + merged_maps.emplace(key, nested_place); + nested_func->deserialize(nested_place, buf, arena); + } + } + + void insertResultInto(AggregateDataPtr place, IColumn & to, Arena * arena) const override + { + auto & map_column = assert_cast(to); + auto & nested_column = map_column.getNestedColumn(); + auto & nested_data_column = map_column.getNestedData(); + + auto & key_column = nested_data_column.getColumn(0); + auto & val_column = nested_data_column.getColumn(1); + + auto & merged_maps = this->data(place).merged_maps; + + size_t res_offset = 0; + + // sort the keys + std::vector keys; + keys.reserve(merged_maps.size()); + for (auto & it : merged_maps) + { + keys.push_back(it.first); + } + std::sort(keys.begin(), keys.end()); + + // insert using sorted keys to result column + for (auto & key : keys) + { + res_offset++; + key_column.insert(key); + nested_func->insertResultInto(merged_maps[key], val_column, arena); + } + + IColumn::Offsets & res_offsets = nested_column.getOffsets(); + res_offsets.push_back(res_offset); + } + + bool allocatesMemoryInArena() const override { return true; } + + AggregateFunctionPtr getNestedFunction() const override { return nested_func; } +}; + +} diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 245b089108b..32c51d5f746 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -145,9 +145,20 @@ struct MaxMapDispatchOnTupleArgument void registerAggregateFunctionSumMap(AggregateFunctionFactory & factory) { - factory.registerFunction("sumMap", createAggregateFunctionMap< + // these functions used to be called *Map, with now these names occupied by + // Map combinator, which redirects calls here if was called with + // array or tuple arguments. + factory.registerFunction("sumMappedArrays", createAggregateFunctionMap< SumMapVariants::DispatchOnTupleArgument>); + factory.registerFunction("minMappedArrays", + createAggregateFunctionMap); + + factory.registerFunction("maxMappedArrays", + createAggregateFunctionMap); + + // these functions could be renamed to *MappedArrays too, but it would + // break backward compatibility factory.registerFunction("sumMapWithOverflow", createAggregateFunctionMap< SumMapVariants::DispatchOnTupleArgument>); @@ -157,12 +168,6 @@ void registerAggregateFunctionSumMap(AggregateFunctionFactory & factory) factory.registerFunction("sumMapFilteredWithOverflow", createAggregateFunctionMap< SumMapVariants::DispatchOnTupleArgument>); - - factory.registerFunction("minMap", - createAggregateFunctionMap); - - factory.registerFunction("maxMap", - createAggregateFunctionMap); } } diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index b103f42fcc5..9bc507f5b5c 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -377,7 +377,16 @@ public: assertNoParameters(getName(), params_); } - String getName() const override { return "sumMap"; } + String getName() const override { + if constexpr (overflow) + { + return "sumMapWithOverflow"; + } + else + { + return "sumMappedArrays"; + } + } bool keepKey(const T &) const { return true; } }; @@ -528,7 +537,7 @@ public: assertNoParameters(getName(), params_); } - String getName() const override { return "minMap"; } + String getName() const override { return "minMappedArrays"; } bool keepKey(const T &) const { return true; } }; @@ -552,7 +561,7 @@ public: assertNoParameters(getName(), params_); } - String getName() const override { return "maxMap"; } + String getName() const override { return "maxMappedArrays"; } bool keepKey(const T &) const { return true; } }; diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 383f10ac24b..35efb41cf21 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -61,6 +61,7 @@ void registerAggregateFunctionCombinatorNull(AggregateFunctionCombinatorFactory void registerAggregateFunctionCombinatorOrFill(AggregateFunctionCombinatorFactory &); void registerAggregateFunctionCombinatorResample(AggregateFunctionCombinatorFactory &); void registerAggregateFunctionCombinatorDistinct(AggregateFunctionCombinatorFactory &); +void registerAggregateFunctionCombinatorMap(AggregateFunctionCombinatorFactory & factory); void registerWindowFunctions(AggregateFunctionFactory & factory); @@ -132,6 +133,7 @@ void registerAggregateFunctions() registerAggregateFunctionCombinatorOrFill(factory); registerAggregateFunctionCombinatorResample(factory); registerAggregateFunctionCombinatorDistinct(factory); + registerAggregateFunctionCombinatorMap(factory); } } From 03be2c31330b0f0ce674793857a37f1af2686709 Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Mon, 10 May 2021 16:06:59 +0200 Subject: [PATCH 002/396] *MappedArrays will return old name for backwards compability --- src/AggregateFunctions/AggregateFunctionMap.cpp | 10 +++++++++- src/AggregateFunctions/AggregateFunctionSumMap.h | 9 +++++---- .../DataTypeCustomSimpleAggregateFunction.cpp | 3 ++- 3 files changed, 16 insertions(+), 6 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMap.cpp b/src/AggregateFunctions/AggregateFunctionMap.cpp index 581587e91d3..c9c9072ff6c 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionMap.cpp @@ -4,6 +4,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + class AggregateFunctionCombinatorMap final : public IAggregateFunctionCombinator { public: @@ -89,7 +95,9 @@ public: { bool match; const auto * tup_type = checkAndGetDataType(arguments[0].get()); - auto check_func = [](DataTypePtr t) { + + auto check_func = [](DataTypePtr t) + { return t->getTypeId() == TypeIndex::Array; }; diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index 9bc507f5b5c..2c97ad387b8 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -377,14 +377,15 @@ public: assertNoParameters(getName(), params_); } - String getName() const override { + String getName() const override + { if constexpr (overflow) { return "sumMapWithOverflow"; } else { - return "sumMappedArrays"; + return "sumMap"; } } @@ -537,7 +538,7 @@ public: assertNoParameters(getName(), params_); } - String getName() const override { return "minMappedArrays"; } + String getName() const override { return "minMap"; } bool keepKey(const T &) const { return true; } }; @@ -561,7 +562,7 @@ public: assertNoParameters(getName(), params_); } - String getName() const override { return "maxMappedArrays"; } + String getName() const override { return "maxMap"; } bool keepKey(const T &) const { return true; } }; diff --git a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp index 023629fc699..dc0411331e6 100644 --- a/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp +++ b/src/DataTypes/DataTypeCustomSimpleAggregateFunction.cpp @@ -31,7 +31,8 @@ void DataTypeCustomSimpleAggregateFunction::checkSupportedFunctions(const Aggreg /// TODO Make it sane. static const std::vector supported_functions{"any", "anyLast", "min", "max", "sum", "sumWithOverflow", "groupBitAnd", "groupBitOr", "groupBitXor", - "sumMap", "minMap", "maxMap", "groupArrayArray", "groupUniqArrayArray"}; + "sumMap", "minMap", "maxMap", "groupArrayArray", "groupUniqArrayArray", + "sumMappedArrays", "minMappedArrays", "maxMappedArrays"}; // check function if (std::find(std::begin(supported_functions), std::end(supported_functions), function->getName()) == std::end(supported_functions)) From 0570c9db3f59e6169d5a4eb0b096cfd452f40ca7 Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Tue, 18 May 2021 10:42:27 +0200 Subject: [PATCH 003/396] Fix allocation in Map combinator --- src/AggregateFunctions/AggregateFunctionMap.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index 02715b40c90..c2b2784881a 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -103,7 +103,7 @@ public: if (it == merged_maps.end()) { // create a new place for each key - nested_place = arena->alloc(sizeof(AggregateDataPtr)); + nested_place = arena->alloc(nested_func->sizeOfData()); nested_func->create(nested_place); merged_maps.emplace(key, nested_place); } @@ -157,7 +157,7 @@ public: AggregateDataPtr nested_place; this->data(place).readKey(key, buf); - nested_place = arena->alloc(sizeof(AggregateDataPtr)); + nested_place = arena->alloc(nested_func->sizeOfData()); nested_func->create(nested_place); merged_maps.emplace(key, nested_place); nested_func->deserialize(nested_place, buf, arena); From c85aa0a4d670a413e994f311c23428cbd76a923c Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Tue, 18 May 2021 11:48:34 +0200 Subject: [PATCH 004/396] Fix offsets in insertResultInto for map combinator --- src/AggregateFunctions/AggregateFunctionMap.h | 3 +- .../01852_map_combinator.reference | 30 +++++++++++++ .../0_stateless/01852_map_combinator.sql | 42 +++++++++++++++++++ 3 files changed, 74 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01852_map_combinator.reference create mode 100644 tests/queries/0_stateless/01852_map_combinator.sql diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index c2b2784881a..1ae836a13d8 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -195,7 +195,8 @@ public: } IColumn::Offsets & res_offsets = nested_column.getOffsets(); - res_offsets.push_back(res_offset); + auto last_offset = res_offsets[res_offsets.size() - 1]; + res_offsets.push_back(last_offset + res_offset); } bool allocatesMemoryInArena() const override { return true; } diff --git a/tests/queries/0_stateless/01852_map_combinator.reference b/tests/queries/0_stateless/01852_map_combinator.reference new file mode 100644 index 00000000000..59a2d22933b --- /dev/null +++ b/tests/queries/0_stateless/01852_map_combinator.reference @@ -0,0 +1,30 @@ +1 {1:10,2:10,3:10} +1 {3:10,4:10,5:10} +2 {4:10,5:10,6:10} +2 {6:10,7:10,8:10} +3 {1:10,2:10,3:10} +4 {3:10,4:10,5:10} +5 {4:10,5:10,6:10} +5 {6:10,7:10,8:10} +Map(UInt16,UInt64) {1:20,2:20,3:40,4:40,5:40,6:40,7:20,8:20} +Map(UInt16,UInt32) {1:20,2:20,3:40,4:40,5:40,6:40,7:20,8:20} +Map(UInt16,UInt64) {1:20,2:20,3:40,4:40,5:40,6:40,7:20,8:20} +{1:10,2:10,3:10,4:10,5:10,6:10,7:10,8:10} +{1:10,2:10,3:10,4:10,5:10,6:10,7:10,8:10} +Map(UInt16,Float64) {1:10,2:10,3:10,4:10,5:10,6:10,7:10,8:10} +{1:2,2:2,3:4,4:4,5:4,6:4,7:2,8:2} +1 {1:10,2:10,3:20,4:10,5:10} +2 {4:10,5:10,6:20,7:10,8:10} +3 {1:10,2:10,3:10} +4 {3:10,4:10,5:10} +5 {4:10,5:10,6:20,7:10,8:10} +{'01234567-89ab-cdef-0123-456789abcdef':1} +{'1':'1'} +{'1':'1'} +{1:1} +{'1970-01-02':1} +{'1970-01-01 03:00:01':1} +{'a':1} +{'1':'2'} +{1:1.00000,2:2.00000,3:6.00000,4:8.00000,5:10.00000,6:12.00000,7:7.00000,8:8.00000} +{1:1.00000,2:2.00000,3:6.00000,4:8.00000,5:10.00000,6:12.00000,7:7.00000,8:8.00000} diff --git a/tests/queries/0_stateless/01852_map_combinator.sql b/tests/queries/0_stateless/01852_map_combinator.sql new file mode 100644 index 00000000000..26911b983ae --- /dev/null +++ b/tests/queries/0_stateless/01852_map_combinator.sql @@ -0,0 +1,42 @@ +SET send_logs_level = 'fatal'; +SET allow_experimental_map_type = 1; + +DROP TABLE IF EXISTS map_comb; +CREATE TABLE map_comb(a int, statusMap Map(UInt16, UInt32)) ENGINE = Log; + +INSERT INTO map_comb VALUES (1, map(1, 10, 2, 10, 3, 10)),(1, map(3, 10, 4, 10, 5, 10)),(2, map(4, 10, 5, 10, 6, 10)),(2, map(6, 10, 7, 10, 8, 10)),(3, map(1, 10, 2, 10, 3, 10)),(4, map(3, 10, 4, 10, 5, 10)),(5, map(4, 10, 5, 10, 6, 10)),(5, map(6, 10, 7, 10, 8, 10)); + +SELECT * FROM map_comb ORDER BY a; +SELECT toTypeName(res), sumMap(statusMap) as res FROM map_comb; +SELECT toTypeName(res), sumWithOverflowMap(statusMap) as res FROM map_comb; +SELECT toTypeName(res), sumMapMerge(s) as res FROM (SELECT sumMapState(statusMap) AS s FROM map_comb); +SELECT minMap(statusMap) FROM map_comb; +SELECT maxMap(statusMap) FROM map_comb; +SELECT toTypeName(res), avgMap(statusMap) as res FROM map_comb; +SELECT countMap(statusMap) FROM map_comb; +SELECT a, sumMap(statusMap) FROM map_comb GROUP BY a ORDER BY a; + +DROP TABLE map_comb; + +-- check different types +select minMap(val) from values ('val Map(UUID, Int32)', + (map('01234567-89ab-cdef-0123-456789abcdef', 1)), + (map('01234567-89ab-cdef-0123-456789abcdef', 2))); +select minMap(val) from values ('val Map(String, String)', (map('1', '1')), (map('1', '2'))); +select minMap(val) from values ('val Map(FixedString(1), FixedString(1))', (map('1', '1')), (map('1', '2'))); +select minMap(val) from values ('val Map(UInt64, UInt64)', (map(1, 1)), (map(1, 2))); +select minMap(val) from values ('val Map(Date, Int16)', (map(1, 1)), (map(1, 2))); +select minMap(val) from values ('val Map(DateTime(\'Europe/Moscow\'), Int32)', (map(1, 1)), (map(1, 2))); +select minMap(val) from values ('val Map(Enum16(\'a\'=1), Int16)', (map('a', 1)), (map('a', 2))); +select maxMap(val) from values ('val Map(String, String)', (map('1', '1')), (map('1', '2'))); + +DROP TABLE IF EXISTS sum_map_decimal; + +CREATE TABLE sum_map_decimal(statusMap Map(UInt16,Decimal32(5))) ENGINE = Log; + +INSERT INTO sum_map_decimal VALUES (map(1,'1.0',2,'2.0',3,'3.0')), (map(3,'3.0',4,'4.0',5,'5.0')), (map(4,'4.0',5,'5.0',6,'6.0')), (map(6,'6.0',7,'7.0',8,'8.0')); + +SELECT sumMap(statusMap) FROM sum_map_decimal; +SELECT sumWithOverflowMap(statusMap) FROM sum_map_decimal; + +DROP TABLE sum_map_decimal; From 715766d7ec651c7b1ef23e2bb7993206249660ab Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Tue, 25 May 2021 17:00:27 +0200 Subject: [PATCH 005/396] Move validations to transformArguments --- .../AggregateFunctionMap.cpp | 64 +++++++++---------- 1 file changed, 30 insertions(+), 34 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMap.cpp b/src/AggregateFunctions/AggregateFunctionMap.cpp index c9c9072ff6c..31505b89fe2 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionMap.cpp @@ -24,22 +24,40 @@ public: const auto * map_type = checkAndGetDataType(arguments[0].get()); if (map_type) + { + if (arguments->size() > 1) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + getName() + " combinator takes only one map argument"); + return DataTypes({map_type->getValueType()}); + } // we need this part just to pass to redirection for mapped arrays + auto check_func = [](DataTypePtr t) + { + return t->getTypeId() == TypeIndex::Array; + }; + const auto * tup_type = checkAndGetDataType(arguments[0].get()); if (tup_type) { - const auto * val_array_type = checkAndGetDataType(tup_type->getElements()[1].get()); - if (val_array_type) - return DataTypes({val_array_type->getNestedType()}); + const auto & types = tup_type->getElements(); + bool arrays_match = arguments.size() == 1 && types.size() >= 2 && std::all_of(types.begin(), types.end(), check_func); + if (arrays_match) + { + const auto & val_array_type = assert_cast(types[1]); + return DataTypes({val_array_type.getNestedType()}); + } } - - if (arguments.size() >= 2) + else { - const auto * val_array_type = checkAndGetDataType(arguments[1].get()); - if (val_array_type) + bool arrays_match = arguments.size() >= 2 && std::all_of(arguments.begin(), arguments.end(), check_func); + if (arrays_match) + { + const auto & val_array_type = assert_cast(arguments[1]); return DataTypes({val_array_type->getNestedType()}); + } } throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregate function " + getName() + " requires map as argument"); @@ -87,37 +105,15 @@ public: throw Exception{"Illegal columns in arguments for combinator " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } } - else if (!arguments.empty()) + else { - // check if we got tuple of arrays or just arrays and if so, try to redirect to sum/min/max-MappedArrays to implement old behavior + // in case of tuple of arrays or just arrays (checked in transformArguments), try to redirect to sum/min/max-MappedArrays to implement old behavior auto nested_func_name = nested_function->getName(); if (nested_func_name == "sum" || nested_func_name == "min" || nested_func_name == "max") { - bool match; - const auto * tup_type = checkAndGetDataType(arguments[0].get()); - - auto check_func = [](DataTypePtr t) - { - return t->getTypeId() == TypeIndex::Array; - }; - - if (tup_type) - { - const auto & types = tup_type->getElements(); - match = arguments.size() == 1 && types.size() >= 2 && std::all_of(types.begin(), types.end(), check_func); - } - else - { - // sumMappedArrays and others support more than 2 mapped arrays - match = arguments.size() >= 2 && std::all_of(arguments.begin(), arguments.end(), check_func); - } - - if (match) - { - AggregateFunctionProperties out_properties; - auto & aggr_func_factory = AggregateFunctionFactory::instance(); - return aggr_func_factory.get(nested_func_name + "MappedArrays", arguments, params, out_properties); - } + AggregateFunctionProperties out_properties; + auto & aggr_func_factory = AggregateFunctionFactory::instance(); + return aggr_func_factory.get(nested_func_name + "MappedArrays", arguments, params, out_properties); } } From 5a5cb238d8b29b8de37baceb290547a2790231fc Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Wed, 26 May 2021 16:47:58 +0200 Subject: [PATCH 006/396] Map combinator: add more arguments checks, fix memory align errors, support more key types, add some docs and tests --- .../aggregate-functions/combinators.md | 6 +++ .../AggregateFunctionMap.cpp | 40 +++++++++++-------- src/AggregateFunctions/AggregateFunctionMap.h | 16 ++++---- .../01852_map_combinator.reference | 4 ++ .../0_stateless/01852_map_combinator.sql | 15 +++++++ 5 files changed, 56 insertions(+), 25 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/combinators.md b/docs/en/sql-reference/aggregate-functions/combinators.md index 3fc5121ebcc..44615628eef 100644 --- a/docs/en/sql-reference/aggregate-functions/combinators.md +++ b/docs/en/sql-reference/aggregate-functions/combinators.md @@ -25,6 +25,12 @@ Example 2: `uniqArray(arr)` – Counts the number of unique elements in all ‘a -If and -Array can be combined. However, ‘Array’ must come first, then ‘If’. Examples: `uniqArrayIf(arr, cond)`, `quantilesTimingArrayIf(level1, level2)(arr, cond)`. Due to this order, the ‘cond’ argument won’t be an array. +## -Map {#agg-functions-combinator-map} + +The -Map suffix can be appended to any aggregate function. This will create an aggregate function which gets Map type as an argument, and aggregates values of each key of the map separately using the specified aggregate function. The result is also of a Map type. + +Examples: `sumMap(map(1,1))`, `avgMap(map('a', 1))`. + ## -SimpleState {#agg-functions-combinator-simplestate} If you apply this combinator, the aggregate function returns the same value but with a different type. This is a [SimpleAggregateFunction(...)](../../sql-reference/data-types/simpleaggregatefunction.md) that can be stored in a table to work with [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md) tables. diff --git a/src/AggregateFunctions/AggregateFunctionMap.cpp b/src/AggregateFunctions/AggregateFunctionMap.cpp index 31505b89fe2..09214427ad6 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionMap.cpp @@ -25,19 +25,14 @@ public: const auto * map_type = checkAndGetDataType(arguments[0].get()); if (map_type) { - if (arguments->size() > 1) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - getName() + " combinator takes only one map argument"); + if (arguments.size() > 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, getName() + " combinator takes only one map argument"); return DataTypes({map_type->getValueType()}); } // we need this part just to pass to redirection for mapped arrays - auto check_func = [](DataTypePtr t) - { - return t->getTypeId() == TypeIndex::Array; - }; + auto check_func = [](DataTypePtr t) { return t->getTypeId() == TypeIndex::Array; }; const auto * tup_type = checkAndGetDataType(arguments[0].get()); if (tup_type) @@ -46,8 +41,8 @@ public: bool arrays_match = arguments.size() == 1 && types.size() >= 2 && std::all_of(types.begin(), types.end(), check_func); if (arrays_match) { - const auto & val_array_type = assert_cast(types[1]); - return DataTypes({val_array_type.getNestedType()}); + const auto * val_array_type = assert_cast(types[1].get()); + return DataTypes({val_array_type->getNestedType()}); } } else @@ -55,7 +50,7 @@ public: bool arrays_match = arguments.size() >= 2 && std::all_of(arguments.begin(), arguments.end(), check_func); if (arrays_match) { - const auto & val_array_type = assert_cast(arguments[1]); + const auto * val_array_type = assert_cast(arguments[1].get()); return DataTypes({val_array_type->getNestedType()}); } } @@ -72,9 +67,9 @@ public: const auto * map_type = checkAndGetDataType(arguments[0].get()); if (map_type) { - auto key_type_id = map_type->getKeyType()->getTypeId(); + const auto & key_type = map_type->getKeyType(); - switch (key_type_id) + switch (key_type->getTypeId()) { case TypeIndex::Enum8: case TypeIndex::Int8: @@ -86,6 +81,10 @@ public: return std::make_shared>(nested_function, arguments); case TypeIndex::Int64: return std::make_shared>(nested_function, arguments); + case TypeIndex::Int128: + return std::make_shared>(nested_function, arguments); + case TypeIndex::Int256: + return std::make_shared>(nested_function, arguments); case TypeIndex::UInt8: return std::make_shared>(nested_function, arguments); case TypeIndex::Date: @@ -96,13 +95,19 @@ public: return std::make_shared>(nested_function, arguments); case TypeIndex::UInt64: return std::make_shared>(nested_function, arguments); - case TypeIndex::UUID: + case TypeIndex::UInt128: return std::make_shared>(nested_function, arguments); + case TypeIndex::UInt256: + return std::make_shared>(nested_function, arguments); + case TypeIndex::UUID: + return std::make_shared>(nested_function, arguments); case TypeIndex::FixedString: case TypeIndex::String: return std::make_shared>(nested_function, arguments); default: - throw Exception{"Illegal columns in arguments for combinator " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception{ + "Map key type " + key_type->getName() + " is not is not supported by combinator " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } } else @@ -115,9 +120,10 @@ public: auto & aggr_func_factory = AggregateFunctionFactory::instance(); return aggr_func_factory.get(nested_func_name + "MappedArrays", arguments, params, out_properties); } + else + throw Exception{ + "Aggregation '" + nested_func_name + "Map' is not implemented for mapped arrays", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } - - throw Exception{"Illegal columns in arguments for combinator " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } }; diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index 1ae836a13d8..75bb2e75840 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -58,7 +58,11 @@ public: { if (types.empty()) throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function " + getName() + " require at least one argument"); + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function " + getName() + " requires at least one argument"); + + if (types.size() > 1) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function " + getName() + " requires only one map argument"); const auto * map_type = checkAndGetDataType(types[0].get()); if (!map_type) @@ -103,7 +107,7 @@ public: if (it == merged_maps.end()) { // create a new place for each key - nested_place = arena->alloc(nested_func->sizeOfData()); + nested_place = arena->alignedAlloc(nested_func->sizeOfData(), nested_func->alignOfData()); nested_func->create(nested_place); merged_maps.emplace(key, nested_place); } @@ -157,7 +161,7 @@ public: AggregateDataPtr nested_place; this->data(place).readKey(key, buf); - nested_place = arena->alloc(nested_func->sizeOfData()); + nested_place = arena->alignedAlloc(nested_func->sizeOfData(), nested_func->alignOfData()); nested_func->create(nested_place); merged_maps.emplace(key, nested_place); nested_func->deserialize(nested_place, buf, arena); @@ -175,8 +179,6 @@ public: auto & merged_maps = this->data(place).merged_maps; - size_t res_offset = 0; - // sort the keys std::vector keys; keys.reserve(merged_maps.size()); @@ -189,14 +191,12 @@ public: // insert using sorted keys to result column for (auto & key : keys) { - res_offset++; key_column.insert(key); nested_func->insertResultInto(merged_maps[key], val_column, arena); } IColumn::Offsets & res_offsets = nested_column.getOffsets(); - auto last_offset = res_offsets[res_offsets.size() - 1]; - res_offsets.push_back(last_offset + res_offset); + res_offsets.push_back(val_column.size()); } bool allocatesMemoryInArena() const override { return true; } diff --git a/tests/queries/0_stateless/01852_map_combinator.reference b/tests/queries/0_stateless/01852_map_combinator.reference index 59a2d22933b..7c0648ccb65 100644 --- a/tests/queries/0_stateless/01852_map_combinator.reference +++ b/tests/queries/0_stateless/01852_map_combinator.reference @@ -26,5 +26,9 @@ Map(UInt16,Float64) {1:10,2:10,3:10,4:10,5:10,6:10,7:10,8:10} {'1970-01-01 03:00:01':1} {'a':1} {'1':'2'} +{1:1} +{1:1} +{1:1} +{1:1} {1:1.00000,2:2.00000,3:6.00000,4:8.00000,5:10.00000,6:12.00000,7:7.00000,8:8.00000} {1:1.00000,2:2.00000,3:6.00000,4:8.00000,5:10.00000,6:12.00000,7:7.00000,8:8.00000} diff --git a/tests/queries/0_stateless/01852_map_combinator.sql b/tests/queries/0_stateless/01852_map_combinator.sql index 26911b983ae..20923460eb6 100644 --- a/tests/queries/0_stateless/01852_map_combinator.sql +++ b/tests/queries/0_stateless/01852_map_combinator.sql @@ -29,6 +29,21 @@ select minMap(val) from values ('val Map(Date, Int16)', (map(1, 1)), (map(1, 2) select minMap(val) from values ('val Map(DateTime(\'Europe/Moscow\'), Int32)', (map(1, 1)), (map(1, 2))); select minMap(val) from values ('val Map(Enum16(\'a\'=1), Int16)', (map('a', 1)), (map('a', 2))); select maxMap(val) from values ('val Map(String, String)', (map('1', '1')), (map('1', '2'))); +select minMap(val) from values ('val Map(Int128, Int128)', (map(1, 1)), (map(1, 2))); +select minMap(val) from values ('val Map(Int256, Int256)', (map(1, 1)), (map(1, 2))); +select minMap(val) from values ('val Map(UInt128, UInt128)', (map(1, 1)), (map(1, 2))); +select minMap(val) from values ('val Map(UInt256, UInt256)', (map(1, 1)), (map(1, 2))); + +select sumMap(map(1,2), 1, 2); -- { serverError 42 } +select sumMap(map(1,2), map(1,3)); -- { serverError 42 } + +-- array and tuple arguments +select avgMap([1,1,1], [2,2,2]); -- { serverError 43 } +select minMap((1,1)); -- { serverError 43 } +select minMap(([1,1,1],1)); -- { serverError 43 } +select minMap([1,1,1],1); -- { serverError 43 } +select minMap([1,1,1]); -- { serverError 43 } +select minMap(([1,1,1])); -- { serverError 43 } DROP TABLE IF EXISTS sum_map_decimal; From 95f5a4d0fae94cc8bc601d03006eb37864972b0e Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Mon, 14 Jun 2021 22:11:34 +0500 Subject: [PATCH 007/396] Optimize string keys and fix string serialization in map combinator --- src/AggregateFunctions/AggregateFunctionMap.h | 48 +++++++++++++++---- 1 file changed, 40 insertions(+), 8 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index 75bb2e75840..8ac6a5d6eb6 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -6,6 +6,8 @@ #include #include #include +#include +#include #include #include #include @@ -15,6 +17,7 @@ #include #include #include +#include "common/types.h" #include #include "AggregateFunctions/AggregateFunctionFactory.h" @@ -29,6 +32,7 @@ namespace ErrorCodes template struct AggregateFunctionMapCombinatorData { + using SearchType = KeyType; std::unordered_map merged_maps; static void writeKey(KeyType key, WriteBuffer & buf) { writeBinary(key, buf); } @@ -38,10 +42,29 @@ struct AggregateFunctionMapCombinatorData template <> struct AggregateFunctionMapCombinatorData { - std::unordered_map merged_maps; + struct StringHash + { + using hash_type = std::hash; + using is_transparent = void; - static void writeKey(String key, WriteBuffer & buf) { writeString(key, buf); } - static void readKey(String & key, ReadBuffer & buf) { readString(key, buf); } + size_t operator()(std::string_view str) const { return hash_type{}(str); } + }; + + using SearchType = std::string_view; + std::unordered_map> merged_maps; + + static void writeKey(String key, WriteBuffer & buf) + { + writeVarUInt(key.size(), buf); + writeString(key, buf); + } + static void readKey(String & key, ReadBuffer & buf) + { + UInt64 size; + readVarUInt(size, buf); + key.resize(size); + buf.readStrict(key.data(), size); + } }; template @@ -51,7 +74,9 @@ class AggregateFunctionMap final private: DataTypePtr key_type; AggregateFunctionPtr nested_func; - using Base = IAggregateFunctionDataHelper, AggregateFunctionMap>; + + using Data = AggregateFunctionMapCombinatorData; + using Base = IAggregateFunctionDataHelper>; public: AggregateFunctionMap(AggregateFunctionPtr nested, const DataTypes & types) : Base(types, nested->getParameters()), nested_func(nested) @@ -91,10 +116,17 @@ public: for (size_t i = 0; i < size; ++i) { - KeyType key; + typename Data::SearchType key; + if constexpr (std::is_same::value) { - key = key_column.operator[](offset + i).get(); + StringRef key_ref; + if (key_type->getTypeId() == TypeIndex::FixedString) + key_ref = assert_cast(key_column).getDataAt(offset + i); + else + key_ref = assert_cast(key_column).getDataAt(offset + i); + + key = static_cast(key_ref); } else { @@ -152,10 +184,10 @@ public: void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override { auto & merged_maps = this->data(place).merged_maps; - size_t size; + UInt64 size; readVarUInt(size, buf); - for (size_t i = 0; i < size; ++i) + for (UInt64 i = 0; i < size; ++i) { KeyType key; AggregateDataPtr nested_place; From fd15fc5e4ab6101996bed18ff07a95300177c3e1 Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Tue, 15 Jun 2021 16:20:42 +0500 Subject: [PATCH 008/396] Fix style errors, disable heterogeneous comparison lookup where it's not supported --- src/AggregateFunctions/AggregateFunctionMap.cpp | 10 +++++----- src/AggregateFunctions/AggregateFunctionMap.h | 14 +++++++++++--- 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMap.cpp b/src/AggregateFunctions/AggregateFunctionMap.cpp index 09214427ad6..1808e1c2ffa 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionMap.cpp @@ -105,9 +105,9 @@ public: case TypeIndex::String: return std::make_shared>(nested_function, arguments); default: - throw Exception{ - "Map key type " + key_type->getName() + " is not is not supported by combinator " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Map key type " + key_type->getName() + " is not is not supported by combinator " + getName()); } } else @@ -121,8 +121,8 @@ public: return aggr_func_factory.get(nested_func_name + "MappedArrays", arguments, params, out_properties); } else - throw Exception{ - "Aggregation '" + nested_func_name + "Map' is not implemented for mapped arrays", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Aggregation '" + nested_func_name + "Map' is not implemented for mapped arrays"); } } }; diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index 8ac6a5d6eb6..63974091a01 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -3,11 +3,11 @@ #include #include #include +#include #include +#include #include #include -#include -#include #include #include #include @@ -47,10 +47,14 @@ struct AggregateFunctionMapCombinatorData using hash_type = std::hash; using is_transparent = void; - size_t operator()(std::string_view str) const { return hash_type{}(str); } + size_t operator()(std::string_view str) const { return hash_type{}(str); } }; +#ifdef __cpp_lib_generic_unordered_lookup using SearchType = std::string_view; +#else + using SearchType = std::string; +#endif std::unordered_map> merged_maps; static void writeKey(String key, WriteBuffer & buf) @@ -126,7 +130,11 @@ public: else key_ref = assert_cast(key_column).getDataAt(offset + i); +#ifdef __cpp_lib_generic_unordered_lookup key = static_cast(key_ref); +#else + key = key_ref.toString(); +#endif } else { From c8fa36e3d67e95241e8cf3c5b5a9af2b038c89e6 Mon Sep 17 00:00:00 2001 From: adevyatova Date: Wed, 18 Aug 2021 15:16:37 +0000 Subject: [PATCH 009/396] 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/396] 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 e3ae32f7635c82a441ef5af55cd787c2302765f0 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov Date: Thu, 23 Sep 2021 19:39:15 +0300 Subject: [PATCH 011/396] 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 9cbf4b842295a09196ae096425925f98912cc5a2 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Wed, 13 Oct 2021 22:20:24 +0300 Subject: [PATCH 012/396] 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 324dfd4f814e246b544e8d79fe1b79c9cf152446 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 14 Oct 2021 13:32:49 +0300 Subject: [PATCH 013/396] 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 2979c0dee787318ac7f8c49639414a815dd1b30d Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 14 Oct 2021 15:05:49 +0300 Subject: [PATCH 014/396] 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 252634c6e9f4048f70b68e22d2131edd004b1b2d Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 14 Oct 2021 22:01:06 +0300 Subject: [PATCH 015/396] 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 c67e04b3a5d3d7a5e6a626ac8b35f8ee5e6b011e Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 15 Oct 2021 16:09:34 +0300 Subject: [PATCH 016/396] 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 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 017/396] 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 018/396] 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 019/396] 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 020/396] 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 021/396] 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 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 022/396] 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 023/396] 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 d90302aa3b818f727d4c548af38e0dfd9b207844 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 13 Oct 2021 11:19:37 +0300 Subject: [PATCH 024/396] output_format_avro_rows_in_file --- src/Core/Settings.h | 4 +- src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../Formats/Impl/AvroRowOutputFormat.cpp | 72 ++++++++++++++++--- .../Formats/Impl/AvroRowOutputFormat.h | 8 ++- .../Kafka/WriteBufferToKafkaProducer.cpp | 11 ++- tests/integration/test_storage_kafka/test.py | 47 ++++++++++++ 7 files changed, 129 insertions(+), 15 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f55f10c0267..2b1aa5bf837 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -572,8 +572,6 @@ class IColumn; M(Bool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \ M(Bool, input_format_avro_allow_missing_fields, false, "For Avro/AvroConfluent format: when field is not found in schema use default value instead of error", 0) \ M(URI, format_avro_schema_registry_url, "", "For AvroConfluent format: Confluent Schema Registry URL.", 0) \ - M(String, output_format_avro_string_column_pattern, "", "For Avro format: regexp of String columns to select as AVRO string.", 0) \ - \ M(Bool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.", 0) \ \ M(Bool, output_format_json_quote_denormals, false, "Enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format.", 0) \ @@ -590,6 +588,8 @@ class IColumn; M(UInt64, output_format_parquet_row_group_size, 1000000, "Row group size in rows.", 0) \ M(String, output_format_avro_codec, "", "Compression codec used for output. Possible values: 'null', 'deflate', 'snappy'.", 0) \ M(UInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \ + M(String, output_format_avro_string_column_pattern, "", "For Avro format: regexp of String columns to select as AVRO string.", 0) \ + M(UInt64, output_format_avro_rows_in_file, 1000000, "Max rows in a file (if permitted by storage)", 0) \ M(Bool, output_format_tsv_crlf_end_of_line, false, "If it is set true, end of line in TSV format will be \\r\\n instead of \\n.", 0) \ M(String, output_format_csv_null_representation, "\\N", "Custom NULL representation in CSV format", 0) \ M(String, output_format_tsv_null_representation, "\\N", "Custom NULL representation in TSV format", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 9901081d7dd..1115933c1ac 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -53,6 +53,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.avro.output_sync_interval = settings.output_format_avro_sync_interval; format_settings.avro.schema_registry_url = settings.format_avro_schema_registry_url.toString(); format_settings.avro.string_column_pattern = settings.output_format_avro_string_column_pattern.toString(); + format_settings.avro.output_rows_in_file = settings.output_format_avro_rows_in_file; format_settings.csv.allow_double_quotes = settings.format_csv_allow_double_quotes; 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; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 8c894c77e82..fadb95efd50 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -64,6 +64,7 @@ struct FormatSettings UInt64 output_sync_interval = 16 * 1024; bool allow_missing_fields = false; String string_column_pattern; + UInt64 output_rows_in_file = 1000000; } avro; struct CSV diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp index 24b231e9ea8..9035c9f8627 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp @@ -43,6 +43,8 @@ #include +#include + namespace DB { namespace ErrorCodes @@ -93,7 +95,7 @@ public: virtual void backup(size_t len) override { out.position() -= len; } virtual uint64_t byteCount() const override { return out.count(); } - virtual void flush() override { out.next(); } + virtual void flush() override { /* out.next(); */} private: WriteBuffer & out; @@ -385,12 +387,8 @@ AvroRowOutputFormat::AvroRowOutputFormat( WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & settings_) : IRowOutputFormat(header_, out_, params_) , settings(settings_) + , params(params_) , serializer(header_.getColumnsWithTypeAndName(), std::make_unique(settings)) - , file_writer( - std::make_unique(out_), - serializer.getSchema(), - settings.avro.output_sync_interval, - getCodec(settings.avro.output_codec)) { } @@ -398,19 +396,71 @@ AvroRowOutputFormat::~AvroRowOutputFormat() = default; void AvroRowOutputFormat::writePrefix() { - file_writer.syncIfNeeded(); + file_writer_ptr = std::make_unique( + std::make_unique(out), + serializer.getSchema(), + settings.avro.output_sync_interval, + getCodec(settings.avro.output_codec)); + + file_writer_ptr->syncIfNeeded(); } void AvroRowOutputFormat::write(const Columns & columns, size_t row_num) { - file_writer.syncIfNeeded(); - serializer.serializeRow(columns, row_num, file_writer.encoder()); - file_writer.incr(); + file_writer_ptr->syncIfNeeded(); + serializer.serializeRow(columns, row_num, file_writer_ptr->encoder()); + file_writer_ptr->incr(); } void AvroRowOutputFormat::writeSuffix() { - file_writer.close(); + file_writer_ptr.reset(); +} + +void AvroRowOutputFormat::consume(DB::Chunk chunk) +{ + LOG_TRACE(&Poco::Logger::get("AvroBlockOutputFormat"), "top of consume"); + + if (params.callback) + consumeImplCallback(std::move(chunk)); + else + consumeImpl(std::move(chunk)); +} + +void AvroRowOutputFormat::consumeImpl(DB::Chunk chunk) +{ + auto num_rows = chunk.getNumRows(); + const auto & columns = chunk.getColumns(); + + writePrefix(); + for (size_t row = 0; row < num_rows; ++row) + write(columns, row); + + first_row = false; +} + +void AvroRowOutputFormat::consumeImplCallback(DB::Chunk chunk) +{ + auto num_rows = chunk.getNumRows(); + const auto & columns = chunk.getColumns(); + + for (size_t row = 0; row < num_rows;) + { + writePrefix(); + for (size_t row_in_file = 0; + row_in_file < settings.avro.output_rows_in_file && row < num_rows; + ++row, ++row_in_file) + { + write(columns, row); + } + + + file_writer_ptr->flush(); + writeSuffix(); + + params.callback(columns, num_rows); + first_row = false; + } } void registerOutputFormatProcessorAvro(FormatFactory & factory) diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.h b/src/Processors/Formats/Impl/AvroRowOutputFormat.h index c807736071e..af4125d831a 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.h +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.h @@ -49,6 +49,7 @@ public: AvroRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & settings_); virtual ~AvroRowOutputFormat() override; + void consume(Chunk) override; String getName() const override { return "AvroRowOutputFormat"; } void write(const Columns & columns, size_t row_num) override; void writeField(const IColumn &, const ISerialization &, size_t) override {} @@ -57,8 +58,13 @@ public: private: FormatSettings settings; + Params params; AvroSerializer serializer; - avro::DataFileWriterBase file_writer; + std::unique_ptr file_writer_ptr; + + void consumeImpl(Chunk); + void consumeImplCallback(Chunk); + }; } diff --git a/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp b/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp index 34ab48e501d..d574c32b6e0 100644 --- a/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp +++ b/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp @@ -3,6 +3,8 @@ #include "Columns/ColumnString.h" #include "Columns/ColumnsNumber.h" +#include + namespace DB { WriteBufferToKafkaProducer::WriteBufferToKafkaProducer( @@ -54,13 +56,14 @@ WriteBufferToKafkaProducer::~WriteBufferToKafkaProducer() void WriteBufferToKafkaProducer::countRow(const Columns & columns, size_t current_row) { + LOG_TRACE(&Poco::Logger::get("WriteBufferToKafkaProducer"), "top of countRow"); if (++rows % max_rows == 0) { const std::string & last_chunk = chunks.back(); size_t last_chunk_size = offset(); // if last character of last chunk is delimiter - we don't need it - if (delim && last_chunk[last_chunk_size - 1] == delim) + if (last_chunk_size && delim && last_chunk[last_chunk_size - 1] == delim) --last_chunk_size; std::string payload; @@ -76,6 +79,8 @@ void WriteBufferToKafkaProducer::countRow(const Columns & columns, size_t curren cppkafka::MessageBuilder builder(topic); builder.payload(payload); + LOG_TRACE(&Poco::Logger::get("WriteBufferToKafkaProducer"), "payload size {}", payload.size()); + // Note: if it will be few rows per message - it will take the value from last row of block if (key_column_index) { @@ -116,6 +121,7 @@ void WriteBufferToKafkaProducer::countRow(const Columns & columns, size_t curren void WriteBufferToKafkaProducer::flush() { + LOG_TRACE(&Poco::Logger::get("WriteBufferToKafkaProducer"), "flush"); // For unknown reason we may hit some internal timeout when inserting for the first time. while (true) { @@ -136,11 +142,13 @@ void WriteBufferToKafkaProducer::flush() void WriteBufferToKafkaProducer::nextImpl() { + LOG_TRACE(&Poco::Logger::get("WriteBufferToKafkaProducer"), "nextImpl"); addChunk(); } void WriteBufferToKafkaProducer::addChunk() { + LOG_TRACE(&Poco::Logger::get("WriteBufferToKafkaProducer"), "addChunk"); chunks.push_back(std::string()); chunks.back().resize(chunk_size); set(chunks.back().data(), chunk_size); @@ -148,6 +156,7 @@ void WriteBufferToKafkaProducer::addChunk() void WriteBufferToKafkaProducer::reinitializeChunks() { + LOG_TRACE(&Poco::Logger::get("WriteBufferToKafkaProducer"), "reinitializeChunks"); rows = 0; chunks.clear(); /// We cannot leave the buffer in the undefined state (i.e. without any diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 6106966e5b7..e01f4bf146e 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -8,6 +8,7 @@ import logging import io import string import ast +import math import avro.schema import avro.io @@ -1829,6 +1830,52 @@ def test_kafka_produce_key_timestamp(kafka_cluster): kafka_delete_topic(admin_client, topic_name) +def test_kafka_produce_consume_avro(kafka_cluster): + + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) + + topic_name = "insert_avro" + kafka_create_topic(admin_client, topic_name) + + num_rows = 75 + + instance.query(''' + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.kafka; + DROP TABLE IF EXISTS test.kafka_writer; + + CREATE TABLE test.kafka_writer (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'avro', + kafka_group_name = 'avro', + kafka_format = 'Avro'; + + + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'avro', + kafka_group_name = 'avro', + kafka_format = 'Avro'; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.kafka; + ''') + + instance.query("INSERT INTO test.kafka_writer select number*10 as key, number*100 as value from numbers({num_rows}) SETTINGS output_format_avro_rows_in_file = 7".format(num_rows=num_rows)) + + instance.wait_for_log_line("Committed offset {offset}".format(offset=math.ceil(num_rows/7))) + + expected_num_rows = instance.query("SELECT COUNT(1) FROM test.view", ignore_error=True) + assert (int(expected_num_rows) == num_rows) + + expected_max_key = instance.query("SELECT max(key) FROM test.view", ignore_error=True) + assert (int(expected_max_key) == (num_rows - 1) * 10) + + kafka_delete_topic(admin_client, topic_name) + + def test_kafka_flush_by_time(kafka_cluster): admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_name = "flush_by_time" From deb696fd27e8a9f7e3f931bf7c0e76abacd410f9 Mon Sep 17 00:00:00 2001 From: Alexey Date: Mon, 18 Oct 2021 16:07:19 +0000 Subject: [PATCH 025/396] 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 6dad340f109261e8e796c40428c4474ae5960170 Mon Sep 17 00:00:00 2001 From: lehasm Date: Tue, 19 Oct 2021 08:21:45 +0300 Subject: [PATCH 026/396] 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 027/396] 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 a16eda68dc54ff0c3be7362dfec02061074f6e04 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 19 Oct 2021 12:39:14 +0300 Subject: [PATCH 028/396] 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 74fd33b62eb56f88e6b372382ef4dd3d9e92f5d6 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 19 Oct 2021 13:25:22 +0300 Subject: [PATCH 029/396] 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 32dd422cc707a00ec470435e29b0812220eefba2 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Tue, 19 Oct 2021 16:42:16 +0300 Subject: [PATCH 030/396] 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 facfa16978a2ef625761e1c128dae41fa3f2a737 Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Tue, 19 Oct 2021 21:40:04 +0300 Subject: [PATCH 031/396] 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 032/396] 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 033/396] 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 1b763284a23cbe30adb3b687fe71ed3cef962258 Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Tue, 19 Oct 2021 21:56:11 +0300 Subject: [PATCH 034/396] 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 035/396] 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 9f9f4a561cc7ee20f419ba7b390ddac77b98f499 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 19 Oct 2021 23:26:55 +0300 Subject: [PATCH 036/396] output_format_avro_rows_in_file - fixes --- src/Core/Settings.h | 2 +- src/Processors/Formats/IRowOutputFormat.h | 38 +++++++++---------- .../Formats/Impl/AvroRowOutputFormat.cpp | 17 ++++----- .../Formats/Impl/AvroRowOutputFormat.h | 2 +- 4 files changed, 29 insertions(+), 30 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2b1aa5bf837..5785769fed9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -572,8 +572,8 @@ class IColumn; M(Bool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \ M(Bool, input_format_avro_allow_missing_fields, false, "For Avro/AvroConfluent format: when field is not found in schema use default value instead of error", 0) \ M(URI, format_avro_schema_registry_url, "", "For AvroConfluent format: Confluent Schema Registry URL.", 0) \ - M(Bool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.", 0) \ \ + M(Bool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.", 0) \ M(Bool, output_format_json_quote_denormals, false, "Enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format.", 0) \ \ M(Bool, output_format_json_escape_forward_slashes, true, "Controls escaping forward slashes for string outputs in JSON output format. This is intended for compatibility with JavaScript. Don't confuse with backslashes that are always escaped.", 0) \ diff --git a/src/Processors/Formats/IRowOutputFormat.h b/src/Processors/Formats/IRowOutputFormat.h index c35d93b6133..cb9021d9e95 100644 --- a/src/Processors/Formats/IRowOutputFormat.h +++ b/src/Processors/Formats/IRowOutputFormat.h @@ -33,6 +33,25 @@ protected: void consumeExtremes(Chunk chunk) override; void finalize() override; + bool prefix_written = false; + bool suffix_written = false; + + void writePrefixIfNot() + { + if (!prefix_written) + writePrefix(); + + prefix_written = true; + } + + void writeSuffixIfNot() + { + if (!suffix_written) + writeSuffix(); + + suffix_written = true; + } + public: using Params = RowOutputFormatParams; @@ -64,27 +83,8 @@ public: virtual void writeLastSuffix() {} /// Write something after resultset, totals end extremes. private: - bool prefix_written = false; - bool suffix_written = false; - Params params; - void writePrefixIfNot() - { - if (!prefix_written) - writePrefix(); - - prefix_written = true; - } - - void writeSuffixIfNot() - { - if (!suffix_written) - writeSuffix(); - - suffix_written = true; - } - }; } diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp index 9035c9f8627..73b8b561a00 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp @@ -43,8 +43,6 @@ #include -#include - namespace DB { namespace ErrorCodes @@ -95,7 +93,7 @@ public: virtual void backup(size_t len) override { out.position() -= len; } virtual uint64_t byteCount() const override { return out.count(); } - virtual void flush() override { /* out.next(); */} + virtual void flush() override { } private: WriteBuffer & out; @@ -396,6 +394,7 @@ AvroRowOutputFormat::~AvroRowOutputFormat() = default; void AvroRowOutputFormat::writePrefix() { + // we have to recreate avro::DataFileWriterBase object due to its interface limitations file_writer_ptr = std::make_unique( std::make_unique(out), serializer.getSchema(), @@ -419,10 +418,8 @@ void AvroRowOutputFormat::writeSuffix() void AvroRowOutputFormat::consume(DB::Chunk chunk) { - LOG_TRACE(&Poco::Logger::get("AvroBlockOutputFormat"), "top of consume"); - if (params.callback) - consumeImplCallback(std::move(chunk)); + consumeImplWithCallback(std::move(chunk)); else consumeImpl(std::move(chunk)); } @@ -432,14 +429,16 @@ void AvroRowOutputFormat::consumeImpl(DB::Chunk chunk) auto num_rows = chunk.getNumRows(); const auto & columns = chunk.getColumns(); - writePrefix(); + writePrefixIfNot(); for (size_t row = 0; row < num_rows; ++row) + { write(columns, row); + first_row = false; + } - first_row = false; } -void AvroRowOutputFormat::consumeImplCallback(DB::Chunk chunk) +void AvroRowOutputFormat::consumeImplWithCallback(DB::Chunk chunk) { auto num_rows = chunk.getNumRows(); const auto & columns = chunk.getColumns(); diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.h b/src/Processors/Formats/Impl/AvroRowOutputFormat.h index af4125d831a..7515946df5d 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.h +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.h @@ -63,7 +63,7 @@ private: std::unique_ptr file_writer_ptr; void consumeImpl(Chunk); - void consumeImplCallback(Chunk); + void consumeImplWithCallback(Chunk); }; From 78e5727bfdd97439b99b033366eb711a2a66f681 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Tue, 19 Oct 2021 23:47:29 +0300 Subject: [PATCH 037/396] 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 038/396] 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 039/396] 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 040/396] 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 041/396] 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 042/396] 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 043/396] 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 044/396] 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 c8e5a67c0dcbd5c95b9ff33488e25482dd29a279 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 20 Oct 2021 00:40:14 +0300 Subject: [PATCH 045/396] output_format_avro_rows_in_file tested against RabbitMQ --- .../integration/test_storage_rabbitmq/test.py | 44 ++++++++++++++++++- 1 file changed, 43 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 36d63588386..1c27c95d2ab 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -6,6 +6,7 @@ import threading import logging import time from random import randrange +import math import pika import pytest @@ -250,7 +251,7 @@ def test_rabbitmq_macros(rabbitmq_cluster): for i in range(50): message += json.dumps({'key': i, 'value': i}) + '\n' channel.basic_publish(exchange='macro', routing_key='', body=message) - + connection.close() time.sleep(1) @@ -2027,6 +2028,47 @@ def test_rabbitmq_queue_consume(rabbitmq_cluster): instance.query('DROP TABLE test.rabbitmq_queue') +def test_rabbitmq_produce_consume_avro(rabbitmq_cluster): + num_rows = 75 + + instance.query(''' + DROP TABLE IF EXISTS test.view; + DROP TABLE IF EXISTS test.rabbit; + DROP TABLE IF EXISTS test.rabbit_writer; + + CREATE TABLE test.rabbit_writer (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_format = 'Avro', + rabbitmq_exchange_name = 'avro', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'avro'; + + CREATE TABLE test.rabbit (key UInt64, value UInt64) + ENGINE = RabbitMQ + SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + rabbitmq_format = 'Avro', + rabbitmq_exchange_name = 'avro', + rabbitmq_exchange_type = 'direct', + rabbitmq_routing_key_list = 'avro'; + + CREATE MATERIALIZED VIEW test.view Engine=Log AS + SELECT key, value FROM test.rabbit; + ''') + + instance.query("INSERT INTO test.rabbit_writer select number*10 as key, number*100 as value from numbers({num_rows}) SETTINGS output_format_avro_rows_in_file = 7".format(num_rows=num_rows)) + + + # Ideally we should wait for an event + time.sleep(3) + + expected_num_rows = instance.query("SELECT COUNT(1) FROM test.view", ignore_error=True) + assert (int(expected_num_rows) == num_rows) + + expected_max_key = instance.query("SELECT max(key) FROM test.view", ignore_error=True) + assert (int(expected_max_key) == (num_rows - 1) * 10) + + def test_rabbitmq_bad_args(rabbitmq_cluster): credentials = pika.PlainCredentials('root', 'clickhouse') parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials) From fa6856ab5efed7f352dfc9ff9f03123facab5eea Mon Sep 17 00:00:00 2001 From: karnevil13 Date: Wed, 20 Oct 2021 01:38:34 +0300 Subject: [PATCH 046/396] 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 70072860883b4abba36c9f68b7217f394a4933c1 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 20 Oct 2021 14:48:54 +0300 Subject: [PATCH 047/396] 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 048/396] 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 049/396] 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 872cca550a9d21e4f5c5b4e996a68f45c254b82a Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 20 Oct 2021 15:47:20 +0300 Subject: [PATCH 050/396] 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 9d74ae16df157178ec7f1698d652de9cf4fa6f23 Mon Sep 17 00:00:00 2001 From: Alexey Date: Wed, 20 Oct 2021 20:01:57 +0000 Subject: [PATCH 051/396] 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 82f33151e764a91f970c1deec777f783bc3f07d2 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 20 Oct 2021 23:47:50 +0300 Subject: [PATCH 052/396] output_format_avro_rows_in_file fixes per code review --- src/Formats/FormatSettings.h | 2 +- src/Processors/Formats/IRowOutputFormat.h | 17 +++++++++-------- .../Formats/Impl/AvroRowOutputFormat.cpp | 3 --- .../Formats/Impl/AvroRowOutputFormat.h | 1 - .../Kafka/WriteBufferToKafkaProducer.cpp | 10 ---------- .../RabbitMQ/WriteBufferToRabbitMQProducer.cpp | 2 +- 6 files changed, 11 insertions(+), 24 deletions(-) diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index fadb95efd50..d39c1ba05fa 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -64,7 +64,7 @@ struct FormatSettings UInt64 output_sync_interval = 16 * 1024; bool allow_missing_fields = false; String string_column_pattern; - UInt64 output_rows_in_file = 1000000; + UInt64 output_rows_in_file = 1; } avro; struct CSV diff --git a/src/Processors/Formats/IRowOutputFormat.h b/src/Processors/Formats/IRowOutputFormat.h index cb9021d9e95..18575419cd0 100644 --- a/src/Processors/Formats/IRowOutputFormat.h +++ b/src/Processors/Formats/IRowOutputFormat.h @@ -23,9 +23,18 @@ class WriteBuffer; */ class IRowOutputFormat : public IOutputFormat { +public: + using Params = RowOutputFormatParams; + +private: + bool prefix_written = false; + bool suffix_written = false; + protected: DataTypes types; Serializations serializations; + Params params; + bool first_row = true; void consume(Chunk chunk) override; @@ -33,9 +42,6 @@ protected: void consumeExtremes(Chunk chunk) override; void finalize() override; - bool prefix_written = false; - bool suffix_written = false; - void writePrefixIfNot() { if (!prefix_written) @@ -53,8 +59,6 @@ protected: } public: - using Params = RowOutputFormatParams; - IRowOutputFormat(const Block & header, WriteBuffer & out_, const Params & params_); /** Write a row. @@ -82,9 +86,6 @@ public: virtual void writeAfterExtremes() {} virtual void writeLastSuffix() {} /// Write something after resultset, totals end extremes. -private: - Params params; - }; } diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp index 73b8b561a00..e5845003ca2 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp @@ -385,7 +385,6 @@ AvroRowOutputFormat::AvroRowOutputFormat( WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & settings_) : IRowOutputFormat(header_, out_, params_) , settings(settings_) - , params(params_) , serializer(header_.getColumnsWithTypeAndName(), std::make_unique(settings)) { } @@ -433,7 +432,6 @@ void AvroRowOutputFormat::consumeImpl(DB::Chunk chunk) for (size_t row = 0; row < num_rows; ++row) { write(columns, row); - first_row = false; } } @@ -458,7 +456,6 @@ void AvroRowOutputFormat::consumeImplWithCallback(DB::Chunk chunk) writeSuffix(); params.callback(columns, num_rows); - first_row = false; } } diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.h b/src/Processors/Formats/Impl/AvroRowOutputFormat.h index 7515946df5d..a3e8493f757 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.h +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.h @@ -58,7 +58,6 @@ public: private: FormatSettings settings; - Params params; AvroSerializer serializer; std::unique_ptr file_writer_ptr; diff --git a/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp b/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp index d574c32b6e0..7b736e95d25 100644 --- a/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp +++ b/src/Storages/Kafka/WriteBufferToKafkaProducer.cpp @@ -3,8 +3,6 @@ #include "Columns/ColumnString.h" #include "Columns/ColumnsNumber.h" -#include - namespace DB { WriteBufferToKafkaProducer::WriteBufferToKafkaProducer( @@ -55,8 +53,6 @@ WriteBufferToKafkaProducer::~WriteBufferToKafkaProducer() void WriteBufferToKafkaProducer::countRow(const Columns & columns, size_t current_row) { - - LOG_TRACE(&Poco::Logger::get("WriteBufferToKafkaProducer"), "top of countRow"); if (++rows % max_rows == 0) { const std::string & last_chunk = chunks.back(); @@ -79,8 +75,6 @@ void WriteBufferToKafkaProducer::countRow(const Columns & columns, size_t curren cppkafka::MessageBuilder builder(topic); builder.payload(payload); - LOG_TRACE(&Poco::Logger::get("WriteBufferToKafkaProducer"), "payload size {}", payload.size()); - // Note: if it will be few rows per message - it will take the value from last row of block if (key_column_index) { @@ -121,7 +115,6 @@ void WriteBufferToKafkaProducer::countRow(const Columns & columns, size_t curren void WriteBufferToKafkaProducer::flush() { - LOG_TRACE(&Poco::Logger::get("WriteBufferToKafkaProducer"), "flush"); // For unknown reason we may hit some internal timeout when inserting for the first time. while (true) { @@ -142,13 +135,11 @@ void WriteBufferToKafkaProducer::flush() void WriteBufferToKafkaProducer::nextImpl() { - LOG_TRACE(&Poco::Logger::get("WriteBufferToKafkaProducer"), "nextImpl"); addChunk(); } void WriteBufferToKafkaProducer::addChunk() { - LOG_TRACE(&Poco::Logger::get("WriteBufferToKafkaProducer"), "addChunk"); chunks.push_back(std::string()); chunks.back().resize(chunk_size); set(chunks.back().data(), chunk_size); @@ -156,7 +147,6 @@ void WriteBufferToKafkaProducer::addChunk() void WriteBufferToKafkaProducer::reinitializeChunks() { - LOG_TRACE(&Poco::Logger::get("WriteBufferToKafkaProducer"), "reinitializeChunks"); rows = 0; chunks.clear(); /// We cannot leave the buffer in the undefined state (i.e. without any diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index 1929a103414..5dce82e3a2e 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -88,7 +88,7 @@ void WriteBufferToRabbitMQProducer::countRow() const std::string & last_chunk = chunks.back(); size_t last_chunk_size = offset(); - if (delim && last_chunk[last_chunk_size - 1] == delim) + if (last_chunk_size && delim && last_chunk[last_chunk_size - 1] == delim) --last_chunk_size; std::string payload; From cd17643eb87ec7b49e351d063088d395cfd25a67 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 21 Oct 2021 09:22:42 +0300 Subject: [PATCH 053/396] 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 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 054/396] 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 551a1065c195acd18f319ce4ce5b3c52c2e763aa Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 21 Oct 2021 14:19:25 +0300 Subject: [PATCH 055/396] output_format_avro_rows_in_file default is 1000000 --- src/Formats/FormatSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index d39c1ba05fa..fadb95efd50 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -64,7 +64,7 @@ struct FormatSettings UInt64 output_sync_interval = 16 * 1024; bool allow_missing_fields = false; String string_column_pattern; - UInt64 output_rows_in_file = 1; + UInt64 output_rows_in_file = 1000000; } avro; struct CSV 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 056/396] 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 d5c5a3213bdc8342b225537b003c3e07678fdaae Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 21 Oct 2021 16:52:27 +0300 Subject: [PATCH 057/396] Add custom null representation support for TSV/CSV input formats, fix bugs in deserializing NULLs in some cases --- src/Core/Settings.h | 4 +- .../Serializations/SerializationNullable.cpp | 200 +++++++++--------- src/Formats/FormatFactory.cpp | 4 +- 3 files changed, 102 insertions(+), 106 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f91bf684c85..402e2b2f6a4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -593,8 +593,8 @@ class IColumn; M(String, output_format_avro_codec, "", "Compression codec used for output. Possible values: 'null', 'deflate', 'snappy'.", 0) \ M(UInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \ M(Bool, output_format_tsv_crlf_end_of_line, false, "If it is set true, end of line in TSV format will be \\r\\n instead of \\n.", 0) \ - M(String, output_format_csv_null_representation, "\\N", "Custom NULL representation in CSV format", 0) \ - M(String, output_format_tsv_null_representation, "\\N", "Custom NULL representation in TSV format", 0) \ + M(String, format_csv_null_representation, "\\N", "Custom NULL representation in CSV format", 0) \ + M(String, format_tsv_null_representation, "\\N", "Custom NULL representation in TSV format", 0) \ M(Bool, output_format_decimal_trailing_zeros, false, "Output trailing zeros when printing Decimal values. E.g. 1.230000 instead of 1.23.", 0) \ \ M(UInt64, input_format_allow_errors_num, 0, "Maximum absolute amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \ diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index 560a4812123..c3a13cc3a52 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -7,12 +7,10 @@ #include #include #include -#include #include #include #include -#include -#include +#include #include namespace DB @@ -21,6 +19,7 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_READ_ALL_DATA; + extern const int INVALID_SETTING_VALUE; } DataTypePtr SerializationNullable::SubcolumnCreator::create(const DataTypePtr & prev) const @@ -260,56 +259,54 @@ template ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested) { - /// Little tricky, because we cannot discriminate null from first character. + const String & null_representation = settings.tsv.null_representation; - if (istr.eof() || *istr.position() != '\\') /// Some data types can deserialize absence of data (e.g. empty string), so eof is ok. + if (istr.eof() || (!null_representation.empty() && *istr.position() != null_representation[0])) { /// This is not null, surely. return safeDeserialize(column, *nested, [] { return false; }, [&nested, &istr, &settings] (IColumn & nested_column) { nested->deserializeTextEscaped(nested_column, istr, settings); }); } - else + + PeekableReadBuffer buf(istr); + auto check_for_null = [&buf, &null_representation]() { - /// Now we know, that data in buffer starts with backslash. - ++istr.position(); + buf.setCheckpoint(); + SCOPE_EXIT(buf.dropCheckpoint()); + if (checkString(null_representation, buf) && (buf.eof() || *buf.position() == '\t' || *buf.position() == '\n')) + return true; - if (istr.eof()) - throw ParsingException("Unexpected end of stream, while parsing value of Nullable type, after backslash", ErrorCodes::CANNOT_READ_ALL_DATA); + buf.rollbackToCheckpoint(); + return false; + }; - return safeDeserialize(column, *nested, - [&istr] - { - if (*istr.position() == 'N') - { - ++istr.position(); - return true; - } - return false; - }, - [&nested, &istr, &settings] (IColumn & nested_column) - { - if (istr.position() != istr.buffer().begin()) - { - /// We could step back to consume backslash again. - --istr.position(); - nested->deserializeTextEscaped(nested_column, istr, settings); - } - else - { - /// Otherwise, we need to place backslash back in front of istr. - ReadBufferFromMemory prefix("\\", 1); - ConcatReadBuffer prepended_istr(prefix, istr); + auto deserialize_nested = [&nested, &settings, &buf, &null_representation] (IColumn & nested_column) + { + auto * pos = buf.position(); + nested->deserializeTextEscaped(nested_column, buf, settings); + /// Check that we don't have any unread data in PeekableReadBuffer own memory. + if (likely(!buf.hasUnreadData())) + return; - nested->deserializeTextEscaped(nested_column, prepended_istr, settings); + /// We have some unread data in PeekableReadBuffer own memory. + /// It can happen only if there is a string instead of a number + /// or if someone uses tab or LF in TSV null_representation. + /// In the first case we cannot continue reading anyway. The second case seems to be unlikely. + if (null_representation.find('\t') != std::string::npos || null_representation.find('\n') != std::string::npos) + throw DB::ParsingException("TSV custom null representation containing '\\t' or '\\n' may not work correctly " + "for large input.", ErrorCodes::CANNOT_READ_ALL_DATA); - /// Synchronise cursor position in original buffer. + WriteBufferFromOwnString parsed_value; + nested->serializeTextEscaped(nested_column, nested_column.size() - 1, parsed_value, settings); + throw DB::ParsingException("Error while parsing \"" + std::string(pos, std::min(size_t{10}, buf.available())) + "\" as Nullable" + + " at position " + std::to_string(buf.count()) + ": got \"" + std::string(pos, buf.position() - pos) + + "\", which was deserialized as \"" + + parsed_value.str() + "\". It seems that input data is ill-formatted.", + ErrorCodes::CANNOT_READ_ALL_DATA); + }; - if (prepended_istr.count() > 1) - istr.position() = prepended_istr.position(); - } - }); - } + return safeDeserialize(column, *nested, check_for_null, deserialize_nested); } void SerializationNullable::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -350,13 +347,30 @@ template ReturnType SerializationNullable::deserializeWholeTextImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested) { - return safeDeserialize(column, *nested, - [&istr] - { - return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr) - || checkStringByFirstCharacterAndAssertTheRest("ᴺᵁᴸᴸ", istr); - }, - [&nested, &istr, &settings] (IColumn & nested_column) { nested->deserializeWholeText(nested_column, istr, settings); }); + PeekableReadBuffer buf(istr); + auto check_for_null = [&buf]() + { + buf.setCheckpoint(); + SCOPE_EXIT(buf.dropCheckpoint()); + + if (checkStringCaseInsensitive("NULL", buf)) + return true; + + buf.rollbackToCheckpoint(); + if (checkStringCaseInsensitive("ᴺᵁᴸᴸ", buf)) + return true; + + buf.rollbackToCheckpoint(); + return false; + }; + + auto deserialize_nested = [&nested, &settings, &buf] (IColumn & nested_column) + { + nested->deserializeWholeText(nested_column, buf, settings); + assert(!buf.hasUnreadData()); + }; + + return safeDeserialize(column, *nested, check_for_null, deserialize_nested); } @@ -377,71 +391,53 @@ void SerializationNullable::deserializeTextCSV(IColumn & column, ReadBuffer & is template ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, - const SerializationPtr & nested) + const SerializationPtr & nested) { - constexpr char const * null_literal = "NULL"; - constexpr size_t len = 4; - size_t null_prefix_len = 0; - - auto check_for_null = [&istr, &settings, &null_prefix_len] + const String & null_representation = settings.csv.null_representation; + if (istr.eof() || (!null_representation.empty() && *istr.position() != null_representation[0])) { - if (checkStringByFirstCharacterAndAssertTheRest("\\N", istr)) - return true; - if (!settings.csv.unquoted_null_literal_as_null) - return false; + /// This is not null, surely. + return safeDeserialize(column, *nested, + [] { return false; }, + [&nested, &istr, &settings] (IColumn & nested_column) { nested->deserializeTextCSV(nested_column, istr, settings); }); + } - /// Check for unquoted NULL - while (!istr.eof() && null_prefix_len < len && null_literal[null_prefix_len] == *istr.position()) - { - ++null_prefix_len; - ++istr.position(); - } - if (null_prefix_len == len) + PeekableReadBuffer buf(istr); + auto check_for_null = [&buf, &null_representation, &settings]() + { + buf.setCheckpoint(); + SCOPE_EXIT(buf.dropCheckpoint()); + if (checkString(null_representation, buf) && (buf.eof() || *buf.position() == settings.csv.delimiter || *buf.position() == '\r' || *buf.position() == '\n')) return true; - /// Value and "NULL" have common prefix, but value is not "NULL". - /// Restore previous buffer position if possible. - if (null_prefix_len <= istr.offset()) - { - istr.position() -= null_prefix_len; - null_prefix_len = 0; - } + buf.rollbackToCheckpoint(); return false; }; - auto deserialize_nested = [&nested, &settings, &istr, &null_prefix_len] (IColumn & nested_column) + auto deserialize_nested = [&nested, &settings, &buf, &null_representation] (IColumn & nested_column) { - if (likely(!null_prefix_len)) - nested->deserializeTextCSV(nested_column, istr, settings); - else - { - /// Previous buffer position was not restored, - /// so we need to prepend extracted characters (rare case) - ReadBufferFromMemory prepend(null_literal, null_prefix_len); - ConcatReadBuffer buf(prepend, istr); - nested->deserializeTextCSV(nested_column, buf, settings); + auto * pos = buf.position(); + nested->deserializeTextCSV(nested_column, buf, settings); + /// Check that we don't have any unread data in PeekableReadBuffer own memory. + if (likely(!buf.hasUnreadData())) + return; - /// Check if all extracted characters were read by nested parser and update buffer position - if (null_prefix_len < buf.count()) - istr.position() = buf.position(); - else if (null_prefix_len > buf.count()) - { - /// It can happen only if there is an unquoted string instead of a number - /// or if someone uses 'U' or 'L' as delimiter in CSV. - /// In the first case we cannot continue reading anyway. The second case seems to be unlikely. - if (settings.csv.delimiter == 'U' || settings.csv.delimiter == 'L') - throw DB::ParsingException("Enabled setting input_format_csv_unquoted_null_literal_as_null may not work correctly " - "with format_csv_delimiter = 'U' or 'L' for large input.", ErrorCodes::CANNOT_READ_ALL_DATA); - WriteBufferFromOwnString parsed_value; - nested->serializeTextCSV(nested_column, nested_column.size() - 1, parsed_value, settings); - throw DB::ParsingException("Error while parsing \"" + std::string(null_literal, null_prefix_len) - + std::string(istr.position(), std::min(size_t{10}, istr.available())) + "\" as Nullable" - + " at position " + std::to_string(istr.count()) + ": got \"" + std::string(null_literal, buf.count()) - + "\", which was deserialized as \"" - + parsed_value.str() + "\". It seems that input data is ill-formatted.", - ErrorCodes::CANNOT_READ_ALL_DATA); - } - } + /// We have some unread data in PeekableReadBuffer own memory. + /// It can happen only if there is an unquoted string instead of a number + /// or if someone uses csv delimiter, LF or CR in CSV null representation. + /// In the first case we cannot continue reading anyway. The second case seems to be unlikely. + if (null_representation.find(settings.csv.delimiter) != std::string::npos || null_representation.find('\r') != std::string::npos + || null_representation.find('\n') != std::string::npos) + throw DB::ParsingException("CSV custom null representation containing format_csv_delimiter, '\\r' or '\\n' may not work correctly " + "for large input.", ErrorCodes::CANNOT_READ_ALL_DATA); + + WriteBufferFromOwnString parsed_value; + nested->serializeTextCSV(nested_column, nested_column.size() - 1, parsed_value, settings); + throw DB::ParsingException("Error while parsing \"" + std::string(pos, std::min(size_t{10}, buf.available())) + "\" as Nullable" + + " at position " + std::to_string(buf.count()) + ": got \"" + std::string(pos, buf.position() - pos) + + "\", which was deserialized as \"" + + parsed_value.str() + "\". It seems that input data is ill-formatted.", + ErrorCodes::CANNOT_READ_ALL_DATA); }; return safeDeserialize(column, *nested, check_for_null, deserialize_nested); diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index d2dc18a03fd..152b58f9fa7 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -58,7 +58,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.delimiter = settings.format_csv_delimiter; format_settings.csv.empty_as_default = settings.input_format_defaults_for_omitted_fields; 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.null_representation = settings.format_csv_null_representation; format_settings.csv.unquoted_null_literal_as_null = settings.input_format_csv_unquoted_null_literal_as_null; format_settings.csv.input_format_arrays_as_nested_csv = settings.input_format_csv_arrays_as_nested_csv; format_settings.custom.escaping_rule = settings.format_custom_escaping_rule; @@ -102,7 +102,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.tsv.crlf_end_of_line = settings.output_format_tsv_crlf_end_of_line; format_settings.tsv.empty_as_default = settings.input_format_tsv_empty_as_default; format_settings.tsv.input_format_enum_as_number = settings.input_format_tsv_enum_as_number; - format_settings.tsv.null_representation = settings.output_format_tsv_null_representation; + format_settings.tsv.null_representation = settings.format_tsv_null_representation; format_settings.values.accurate_types_of_literals = settings.input_format_values_accurate_types_of_literals; 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; From 7a37e24b52f415e2590e73b1e2d0614c82720c55 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 21 Oct 2021 16:58:25 +0300 Subject: [PATCH 058/396] Small fix in deserializeWholeText --- src/DataTypes/Serializations/SerializationNullable.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index c3a13cc3a52..cc73bac9a97 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -353,11 +353,11 @@ ReturnType SerializationNullable::deserializeWholeTextImpl(IColumn & column, Rea buf.setCheckpoint(); SCOPE_EXIT(buf.dropCheckpoint()); - if (checkStringCaseInsensitive("NULL", buf)) + if (checkStringCaseInsensitive("NULL", buf) && buf.eof()) return true; buf.rollbackToCheckpoint(); - if (checkStringCaseInsensitive("ᴺᵁᴸᴸ", buf)) + if (checkStringCaseInsensitive("ᴺᵁᴸᴸ", buf) && buf.eof()) return true; buf.rollbackToCheckpoint(); 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 059/396] 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 b50fff4e2bbd5d1777327001e531d22d3ce4ce09 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 21 Oct 2021 17:33:26 +0300 Subject: [PATCH 060/396] Fix style --- src/DataTypes/Serializations/SerializationNullable.cpp | 1 - src/DataTypes/Serializations/SerializationString.cpp | 3 --- 2 files changed, 4 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index cc73bac9a97..a6618128275 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -19,7 +19,6 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_READ_ALL_DATA; - extern const int INVALID_SETTING_VALUE; } DataTypePtr SerializationNullable::SubcolumnCreator::create(const DataTypePtr & prev) const diff --git a/src/DataTypes/Serializations/SerializationString.cpp b/src/DataTypes/Serializations/SerializationString.cpp index c3c24ed6749..f655a6ca1f9 100644 --- a/src/DataTypes/Serializations/SerializationString.cpp +++ b/src/DataTypes/Serializations/SerializationString.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include @@ -9,8 +8,6 @@ #include #include -#include -#include #include #include From 8aad00818a5d17e85153faf3e3e0c4849c8d1b35 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 21 Oct 2021 18:01:23 +0300 Subject: [PATCH 061/396] Fix build --- src/DataTypes/Serializations/SerializationNullable.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index a6618128275..75f9d40740b 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB { From a0aa30a4e982367d35a2f8734d919cd8b5f6ef49 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 21 Oct 2021 19:41:43 +0300 Subject: [PATCH 062/396] Fix tests --- tests/queries/0_stateless/00301_csv.sh | 2 +- tests/queries/0_stateless/01474_custom_null_tsv.sh | 2 +- .../02029_output_csv_null_representation.reference | 4 ++-- .../0_stateless/02029_output_csv_null_representation.sql | 8 ++++---- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/00301_csv.sh b/tests/queries/0_stateless/00301_csv.sh index 0aee9abe25c..333af9dcfe0 100755 --- a/tests/queries/0_stateless/00301_csv.sh +++ b/tests/queries/0_stateless/00301_csv.sh @@ -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 --format_csv_null_representation='NULL' --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/01474_custom_null_tsv.sh b/tests/queries/0_stateless/01474_custom_null_tsv.sh index 9dc1c4b7777..fb5939faf5e 100755 --- a/tests/queries/0_stateless/01474_custom_null_tsv.sh +++ b/tests/queries/0_stateless/01474_custom_null_tsv.sh @@ -9,7 +9,7 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE tsv_custom_null (id Nullable(UInt32)) E $CLICKHOUSE_CLIENT --query="INSERT INTO tsv_custom_null VALUES (NULL)"; -$CLICKHOUSE_CLIENT --output_format_tsv_null_representation='MyNull' --query="SELECT * FROM tsv_custom_null FORMAT TSV"; +$CLICKHOUSE_CLIENT --format_tsv_null_representation='MyNull' --query="SELECT * FROM tsv_custom_null FORMAT TSV"; $CLICKHOUSE_CLIENT --query="DROP TABLE tsv_custom_null"; diff --git a/tests/queries/0_stateless/02029_output_csv_null_representation.reference b/tests/queries/0_stateless/02029_output_csv_null_representation.reference index a5174f4424f..eda4b09e312 100644 --- a/tests/queries/0_stateless/02029_output_csv_null_representation.reference +++ b/tests/queries/0_stateless/02029_output_csv_null_representation.reference @@ -1,4 +1,4 @@ -# output_format_csv_null_representation should initially be \\N +# format_csv_null_representation should initially be \\N "val1",\N,"val3" -# Changing output_format_csv_null_representation +# Changing format_csv_null_representation "val1",∅,"val3" diff --git a/tests/queries/0_stateless/02029_output_csv_null_representation.sql b/tests/queries/0_stateless/02029_output_csv_null_representation.sql index 772c6c89144..a27c552ee60 100644 --- a/tests/queries/0_stateless/02029_output_csv_null_representation.sql +++ b/tests/queries/0_stateless/02029_output_csv_null_representation.sql @@ -7,10 +7,10 @@ CREATE TABLE test_data ( INSERT INTO test_data VALUES ('val1', NULL, 'val3'); -SELECT '# output_format_csv_null_representation should initially be \\N'; +SELECT '# format_csv_null_representation should initially be \\N'; SELECT * FROM test_data FORMAT CSV; -SELECT '# Changing output_format_csv_null_representation'; -SET output_format_csv_null_representation = '∅'; +SELECT '# Changing format_csv_null_representation'; +SET format_csv_null_representation = '∅'; SELECT * FROM test_data FORMAT CSV; -SET output_format_csv_null_representation = '\\N'; +SET format_csv_null_representation = '\\N'; From cf088eebe5802c39f371f2f236df34de922e2e8b Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Thu, 21 Oct 2021 21:29:03 +0300 Subject: [PATCH 063/396] Delete function bayesAB --- .../functions/machine-learning-functions.md | 80 +------------------ .../functions/machine-learning-functions.md | 79 +----------------- 2 files changed, 2 insertions(+), 157 deletions(-) diff --git a/docs/en/sql-reference/functions/machine-learning-functions.md b/docs/en/sql-reference/functions/machine-learning-functions.md index 4d9322526df..b823340058e 100644 --- a/docs/en/sql-reference/functions/machine-learning-functions.md +++ b/docs/en/sql-reference/functions/machine-learning-functions.md @@ -15,82 +15,4 @@ The [stochasticLinearRegression](../../sql-reference/aggregate-functions/referen ## stochasticLogisticRegression {#stochastic-logistic-regression} -The [stochasticLogisticRegression](../../sql-reference/aggregate-functions/reference/stochasticlogisticregression.md#agg_functions-stochasticlogisticregression) aggregate function implements stochastic gradient descent method for binary classification problem. Uses `evalMLMethod` to predict on new data. - -## bayesAB {#bayesab} - -Compares test groups (variants) and calculates for each group the probability to be the best one. The first group is used as a control group. - -**Syntax** - -``` sql -bayesAB(distribution_name, higher_is_better, variant_names, x, y) -``` - -**Arguments** - -- `distribution_name` — Name of the probability distribution. [String](../../sql-reference/data-types/string.md). Possible values: - - - `beta` for [Beta distribution](https://en.wikipedia.org/wiki/Beta_distribution) - - `gamma` for [Gamma distribution](https://en.wikipedia.org/wiki/Gamma_distribution) - -- `higher_is_better` — Boolean flag. [Boolean](../../sql-reference/data-types/boolean.md). Possible values: - - - `0` — lower values are considered to be better than higher - - `1` — higher values are considered to be better than lower - -- `variant_names` — Variant names. [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). - -- `x` — Numbers of tests for the corresponding variants. [Array](../../sql-reference/data-types/array.md)([Float64](../../sql-reference/data-types/float.md)). - -- `y` — Numbers of successful tests for the corresponding variants. [Array](../../sql-reference/data-types/array.md)([Float64](../../sql-reference/data-types/float.md)). - -!!! note "Note" - All three arrays must have the same size. All `x` and `y` values must be non-negative constant numbers. `y` cannot be larger than `x`. - -**Returned values** - -For each variant the function calculates: -- `beats_control` — long-term probability to out-perform the first (control) variant -- `to_be_best` — long-term probability to out-perform all other variants - -Type: JSON. - -**Example** - -Query: - -``` sql -SELECT bayesAB('beta', 1, ['Control', 'A', 'B'], [3000., 3000., 3000.], [100., 90., 110.]) FORMAT PrettySpace; -``` - -Result: - -``` text -{ - "data":[ - { - "variant_name":"Control", - "x":3000, - "y":100, - "beats_control":0, - "to_be_best":0.22619 - }, - { - "variant_name":"A", - "x":3000, - "y":90, - "beats_control":0.23469, - "to_be_best":0.04671 - }, - { - "variant_name":"B", - "x":3000, - "y":110, - "beats_control":0.7580899999999999, - "to_be_best":0.7271 - } - ] -} -``` - +The [stochasticLogisticRegression](../../sql-reference/aggregate-functions/reference/stochasticlogisticregression.md#agg_functions-stochasticlogisticregression) aggregate function implements stochastic gradient descent method for binary classification problem. Uses `evalMLMethod` to predict on new data. \ No newline at end of file diff --git a/docs/ru/sql-reference/functions/machine-learning-functions.md b/docs/ru/sql-reference/functions/machine-learning-functions.md index ce7d3cfd09e..fb8b872e0d0 100644 --- a/docs/ru/sql-reference/functions/machine-learning-functions.md +++ b/docs/ru/sql-reference/functions/machine-learning-functions.md @@ -15,81 +15,4 @@ toc_title: "Функции машинного обучения" ### Stochastic Logistic Regression {#stochastic-logistic-regression} -Агрегатная функция [stochasticLogisticRegression](../../sql-reference/functions/machine-learning-functions.md#agg_functions-stochasticlogisticregression) реализует стохастический градиентный спуск для задачи бинарной классификации. - -## bayesAB {#bayesab} - -Сравнивает тестовые группы (варианты) и для каждой группы рассчитывает вероятность того, что эта группа окажется лучшей. Первая из перечисленных групп считается контрольной. - -**Синтаксис** - -``` sql -bayesAB(distribution_name, higher_is_better, variant_names, x, y) -``` - -**Аргументы** - -- `distribution_name` — вероятностное распределение. [String](../../sql-reference/data-types/string.md). Возможные значения: - - - `beta` для [Бета-распределения](https://ru.wikipedia.org/wiki/Бета-распределение) - - `gamma` для [Гамма-распределения](https://ru.wikipedia.org/wiki/Гамма-распределение) - -- `higher_is_better` — способ определения предпочтений. [Boolean](../../sql-reference/data-types/boolean.md). Возможные значения: - - - `0` — чем меньше значение, тем лучше - - `1` — чем больше значение, тем лучше - -- `variant_names` — массив, содержащий названия вариантов. [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). - -- `x` — массив, содержащий число проведенных тестов (испытаний) для каждого варианта. [Array](../../sql-reference/data-types/array.md)([Float64](../../sql-reference/data-types/float.md)). - -- `y` — массив, содержащий число успешных тестов (испытаний) для каждого варианта. [Array](../../sql-reference/data-types/array.md)([Float64](../../sql-reference/data-types/float.md)). - -!!! note "Замечание" - Все три массива должны иметь одинаковый размер. Все значения `x` и `y` должны быть неотрицательными числами (константами). Значение `y` не может превышать соответствующее значение `x`. - -**Возвращаемые значения** - -Для каждого варианта рассчитываются: -- `beats_control` — вероятность, что данный вариант превосходит контрольный в долгосрочной перспективе -- `to_be_best` — вероятность, что данный вариант является лучшим в долгосрочной перспективе - -Тип: JSON. - -**Пример** - -Запрос: - -``` sql -SELECT bayesAB('beta', 1, ['Control', 'A', 'B'], [3000., 3000., 3000.], [100., 90., 110.]) FORMAT PrettySpace; -``` - -Результат: - -``` text -{ - "data":[ - { - "variant_name":"Control", - "x":3000, - "y":100, - "beats_control":0, - "to_be_best":0.22619 - }, - { - "variant_name":"A", - "x":3000, - "y":90, - "beats_control":0.23469, - "to_be_best":0.04671 - }, - { - "variant_name":"B", - "x":3000, - "y":110, - "beats_control":0.7580899999999999, - "to_be_best":0.7271 - } - ] -} -``` +Агрегатная функция [stochasticLogisticRegression](../../sql-reference/functions/machine-learning-functions.md#agg_functions-stochasticlogisticregression) реализует стохастический градиентный спуск для задачи бинарной классификации. \ No newline at end of file From 6725dc2a16de0f2609a435a218f72f3f43b29b0a Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Thu, 21 Oct 2021 22:41:11 +0300 Subject: [PATCH 064/396] Create information_schema MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Задокументировал information_schema. --- docs/en/operations/system-tables/columns.md | 31 +++-- docs/en/operations/system-tables/databases.md | 19 ++- .../system-tables/information_schema.md | 117 ++++++++++++++++++ docs/en/operations/system-tables/tables.md | 9 +- .../functions/rounding-functions.md | 2 +- 5 files changed, 158 insertions(+), 20 deletions(-) create mode 100644 docs/en/operations/system-tables/information_schema.md diff --git a/docs/en/operations/system-tables/columns.md b/docs/en/operations/system-tables/columns.md index da4bcec48ed..6720d219579 100644 --- a/docs/en/operations/system-tables/columns.md +++ b/docs/en/operations/system-tables/columns.md @@ -24,6 +24,11 @@ Columns: - `is_in_primary_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the primary key expression. - `is_in_sampling_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the sampling key expression. - `compression_codec` ([String](../../sql-reference/data-types/string.md)) — Compression codec name. +- `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. +- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. Otherwise, the NULL value is returned. +- `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. Otherwise, the NULL value is returned. +- `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. Otherwise, the NULL value is returned. +- `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Subtype code for `DateTime` and `ISO` interval data types. For other data types, the NULL value is returned. **Example** @@ -34,10 +39,11 @@ SELECT * FROM system.columns LIMIT 2 FORMAT Vertical; ```text Row 1: ────── -database: system -table: aggregate_function_combinators -name: name +database: INFORMATION_SCHEMA +table: COLUMNS +name: table_catalog type: String +position: 1 default_kind: default_expression: data_compressed_bytes: 0 @@ -49,13 +55,19 @@ is_in_sorting_key: 0 is_in_primary_key: 0 is_in_sampling_key: 0 compression_codec: +character_octet_length: ᴺᵁᴸᴸ +numeric_precision: ᴺᵁᴸᴸ +numeric_precision_radix: ᴺᵁᴸᴸ +numeric_scale: ᴺᵁᴸᴸ +datetime_precision: ᴺᵁᴸᴸ Row 2: ────── -database: system -table: aggregate_function_combinators -name: is_internal -type: UInt8 +database: INFORMATION_SCHEMA +table: COLUMNS +name: table_schema +type: String +position: 2 default_kind: default_expression: data_compressed_bytes: 0 @@ -67,6 +79,11 @@ is_in_sorting_key: 0 is_in_primary_key: 0 is_in_sampling_key: 0 compression_codec: +character_octet_length: ᴺᵁᴸᴸ +numeric_precision: ᴺᵁᴸᴸ +numeric_precision_radix: ᴺᵁᴸᴸ +numeric_scale: ᴺᵁᴸᴸ +datetime_precision: ᴺᵁᴸᴸ ``` The `system.columns` table contains the following columns (the column type is shown in brackets): diff --git a/docs/en/operations/system-tables/databases.md b/docs/en/operations/system-tables/databases.md index 2c78fd25c2b..45eebf2ae85 100644 --- a/docs/en/operations/system-tables/databases.md +++ b/docs/en/operations/system-tables/databases.md @@ -9,6 +9,7 @@ Columns: - `data_path` ([String](../../sql-reference/data-types/string.md)) — Data path. - `metadata_path` ([String](../../sql-reference/data-types/enum.md)) — Metadata path. - `uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — Database UUID. +- `comment` ([String](../../sql-reference/data-types/enum.md)) — Database comment. The `name` column from this system table is used for implementing the `SHOW DATABASES` query. @@ -17,22 +18,20 @@ The `name` column from this system table is used for implementing the `SHOW DATA Create a database. ``` sql -CREATE DATABASE test +CREATE DATABASE test; ``` Check all of the available databases to the user. ``` sql -SELECT * FROM system.databases +SELECT * FROM system.databases; ``` ``` text -┌─name───────────────────────────┬─engine─┬─data_path──────────────────┬─metadata_path───────────────────────────────────────────────────────┬─────────────────────────────────uuid─┐ -│ _temporary_and_external_tables │ Memory │ /var/lib/clickhouse/ │ │ 00000000-0000-0000-0000-000000000000 │ -│ default │ Atomic │ /var/lib/clickhouse/store/ │ /var/lib/clickhouse/store/d31/d317b4bd-3595-4386-81ee-c2334694128a/ │ d317b4bd-3595-4386-81ee-c2334694128a │ -│ test │ Atomic │ /var/lib/clickhouse/store/ │ /var/lib/clickhouse/store/39b/39bf0cc5-4c06-4717-87fe-c75ff3bd8ebb/ │ 39bf0cc5-4c06-4717-87fe-c75ff3bd8ebb │ -│ system │ Atomic │ /var/lib/clickhouse/store/ │ /var/lib/clickhouse/store/1d1/1d1c869d-e465-4b1b-a51f-be033436ebf9/ │ 1d1c869d-e465-4b1b-a51f-be033436ebf9 │ -└────────────────────────────────┴────────┴────────────────────────────┴─────────────────────────────────────────────────────────────────────┴──────────────────────────────────────┘ +┌─name───────────────┬─engine─┬─data_path──────────────────┬─metadata_path───────────────────────────────────────────────────────┬─uuid─────────────────────────────────┬─comment─┐ +│ INFORMATION_SCHEMA │ Memory │ /var/lib/clickhouse/ │ │ 00000000-0000-0000-0000-000000000000 │ │ +│ default │ Atomic │ /var/lib/clickhouse/store/ │ /var/lib/clickhouse/store/d31/d317b4bd-3595-4386-81ee-c2334694128a/ │ 24363899-31d7-42a0-a436-389931d752a0 │ │ +│ information_schema │ Memory │ /var/lib/clickhouse/ │ │ 00000000-0000-0000-0000-000000000000 │ │ +│ system │ Atomic │ /var/lib/clickhouse/store/ │ /var/lib/clickhouse/store/1d1/1d1c869d-e465-4b1b-a51f-be033436ebf9/ │ 03e9f3d1-cc88-4a49-83e9-f3d1cc881a49 │ │ +└────────────────────┴────────┴────────────────────────────┴─────────────────────────────────────────────────────────────────────┴──────────────────────────────────────┴─────────┘ ``` - -[Original article](https://clickhouse.tech/docs/en/operations/system-tables/databases) diff --git a/docs/en/operations/system-tables/information_schema.md b/docs/en/operations/system-tables/information_schema.md new file mode 100644 index 00000000000..e5c1af7d43a --- /dev/null +++ b/docs/en/operations/system-tables/information_schema.md @@ -0,0 +1,117 @@ +# information_schema {#information-schema} + +`INFORMATION_SCHEMA` (`information_schema`) is a system database that contains views. Using these views, you can get information about the metadata of database objects. + +The structure and composition of system tables may change in different versions of the product, but the support of the `information_schema` makes it possible to change the structure of system tables without changing the method of access to metadata. Metadata requests do not depend on the DBMS used. + +``` sql +SHOW TABLES FROM INFORMATION_SCHEMA; +``` + +``` text +┌─name─────┐ +│ COLUMNS │ +│ SCHEMATA │ +│ TABLES │ +│ VIEWS │ +└──────────┘ +``` + +`INFORMATION_SCHEMA` contains the following views: + +- `SCHEMATA` — The view that can be used to get all the current database schemas. + +``` sql +SELECT * FROM information_schema.schemata WHERE schema_name ILIKE 'information_schema' LIMIT 1 FORMAT Vertical; +``` + +``` text +Row 1: +────── +catalog_name: INFORMATION_SCHEMA +schema_name: INFORMATION_SCHEMA +schema_owner: default +default_character_set_catalog: ᴺᵁᴸᴸ +default_character_set_schema: ᴺᵁᴸᴸ +default_character_set_name: ᴺᵁᴸᴸ +sql_path: ᴺᵁᴸᴸ +``` + +- `TABLES` — The view that can be used to get all tables in the current database. + +``` sql +SELECT * FROM INFORMATION_SCHEMA.TABLES WHERE (table_schema = currentDatabase() OR table_schema = '') AND table_name NOT LIKE '%inner%' LIMIT 1 FORMAT Vertical; +``` + +``` text +Row 1: +────── +table_catalog: default +table_schema: default +table_name: describe_example +table_type: BASE TABLE +``` + +- `COLUMNS` — The view that can be used to get a list of table columns in the current database. + +``` sql +SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE (table_schema=currentDatabase() OR table_schema='') AND table_name NOT LIKE '%inner%' LIMIT 1 FORMAT Vertical; +``` + +``` text +Row 1: +────── +table_catalog: default +table_schema: default +table_name: describe_example +column_name: id +ordinal_position: 1 +column_default: +is_nullable: 0 +data_type: UInt64 +character_maximum_length: ᴺᵁᴸᴸ +character_octet_length: ᴺᵁᴸᴸ +numeric_precision: 64 +numeric_precision_radix: 2 +numeric_scale: 0 +datetime_precision: ᴺᵁᴸᴸ +character_set_catalog: ᴺᵁᴸᴸ +character_set_schema: ᴺᵁᴸᴸ +character_set_name: ᴺᵁᴸᴸ +collation_catalog: ᴺᵁᴸᴸ +collation_schema: ᴺᵁᴸᴸ +collation_name: ᴺᵁᴸᴸ +domain_catalog: ᴺᵁᴸᴸ +domain_schema: ᴺᵁᴸᴸ +domain_name: ᴺᵁᴸᴸ +``` + +- `VIEWS` — The view that can be used to get a list of all views in the current database. + +``` sql +CREATE VIEW v (n Nullable(Int32), f Float64) AS SELECT n, f FROM t; +CREATE MATERIALIZED VIEW mv ENGINE = Null AS SELECT * FROM system.one; +SELECT * FROM information_schema.views WHERE table_schema = currentDatabase(); +``` + +``` text +Row 1: +────── +table_catalog: default +table_schema: default +table_name: mv +view_definition: SELECT * FROM system.one +check_option: NONE +is_updatable: NO +is_insertable_into: YES +is_trigger_updatable: NO +is_trigger_deletable: NO +is_trigger_insertable_into: NO +``` + +**See Also** + +- [View](../../sql-reference/statements/create/view.md). +- [system.tables](../../operations/system-tables/tables.md). +- [system.columns](../../operations/system-tables/columns.md). +- [system.views](../../operations/system-tables/views.md). diff --git a/docs/en/operations/system-tables/tables.md b/docs/en/operations/system-tables/tables.md index f37da02cf5b..9bacd468b87 100644 --- a/docs/en/operations/system-tables/tables.md +++ b/docs/en/operations/system-tables/tables.md @@ -30,6 +30,8 @@ Columns: - `engine_full` ([String](../../sql-reference/data-types/string.md)) - Parameters of the table engine. +- `as_select` ([String](../../sql-reference/data-types/string.md)) - `SELECT` query for view. + - `partition_key` ([String](../../sql-reference/data-types/string.md)) - The partition key expression specified in the table. - `sorting_key` ([String](../../sql-reference/data-types/string.md)) - The sorting key expression specified in the table. @@ -56,6 +58,7 @@ Columns: - `comment` ([String](../../sql-reference/data-types/string.md)) - The comment for the table. +- `has_own_data` ([UInt8](../../sql-reference/data-types/int-uint.md)) - Flag that indicates whether the table itself stores some data on disk or only accesses some other source. The `system.tables` table is used in `SHOW TABLES` query implementation. @@ -80,6 +83,7 @@ dependencies_database: [] dependencies_table: [] create_table_query: CREATE TABLE base.t1 (`n` UInt64) ENGINE = MergeTree ORDER BY n SETTINGS index_granularity = 8192 engine_full: MergeTree ORDER BY n SETTINGS index_granularity = 8192 +as_select: SELECT database AS table_catalog partition_key: sorting_key: n primary_key: n @@ -90,6 +94,7 @@ total_bytes: 99 lifetime_rows: ᴺᵁᴸᴸ lifetime_bytes: ᴺᵁᴸᴸ comment: +has_own_data: 0 Row 2: ────── @@ -105,6 +110,7 @@ dependencies_database: [] dependencies_table: [] create_table_query: CREATE TABLE default.`53r93yleapyears` (`id` Int8, `febdays` Int8) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192 engine_full: MergeTree ORDER BY id SETTINGS index_granularity = 8192 +as_select: SELECT name AS catalog_name partition_key: sorting_key: id primary_key: id @@ -115,6 +121,5 @@ total_bytes: 155 lifetime_rows: ᴺᵁᴸᴸ lifetime_bytes: ᴺᵁᴸᴸ comment: +has_own_data: 0 ``` - -[Original article](https://clickhouse.tech/docs/en/operations/system-tables/tables) diff --git a/docs/ru/sql-reference/functions/rounding-functions.md b/docs/ru/sql-reference/functions/rounding-functions.md index 1eede1ea57c..bcd47973a21 100644 --- a/docs/ru/sql-reference/functions/rounding-functions.md +++ b/docs/ru/sql-reference/functions/rounding-functions.md @@ -173,7 +173,7 @@ roundBankers(4.5) = 4 roundBankers(3.55, 1) = 3.6 roundBankers(3.65, 1) = 3.6 roundBankers(10.35, 1) = 10.4 -roundBankers(10.755, 2) = 11,76 +roundBankers(10.755, 2) = 10,76 ``` **Смотрите также** From 910378b64d8169e417c1c0f2f08c3d6737c37f1e Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Thu, 21 Oct 2021 22:45:18 +0300 Subject: [PATCH 065/396] Update docs/ru/operations/server-configuration-parameters/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index cb53e245dd9..8c0d3f3f90b 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -474,7 +474,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part Значение по умолчанию: `4096` (как в linux [5.4+](https://git.kernel.org/pub/scm/linux/kernel/git/torvalds/linux.git/commit/?id=19f92a030ca6d772ab44b22ee6a01378a8cb32d4)). Обычно это значение незачем менять по следующим причинам: -- значение по умолчанию достаточно велико, +- значение по умолчанию достаточно велико, - для принятия соединения клиента у сервера есть отдельный поток. Так что даже если у вас `TcpExtListenOverflows` (из `nstat`) не на нуле From 4756c1a3b5d16ca839103b77ed6d44818e176564 Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Thu, 21 Oct 2021 22:45:25 +0300 Subject: [PATCH 066/396] Update docs/ru/operations/server-configuration-parameters/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 8c0d3f3f90b..3e0b06c3714 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -475,7 +475,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part Обычно это значение незачем менять по следующим причинам: - значение по умолчанию достаточно велико, -- для принятия соединения клиента у сервера есть отдельный поток. +- для принятия соединений клиента у сервера есть отдельный поток. Так что даже если у вас `TcpExtListenOverflows` (из `nstat`) не на нуле и это число растет, для сервера ClickHouse это не повод увеличивать значение From 1f00496ffdd7fc56fb1cb2e20ca1dd2aaf14f722 Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Thu, 21 Oct 2021 22:47:33 +0300 Subject: [PATCH 067/396] Update docs/ru/operations/server-configuration-parameters/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 3e0b06c3714..c9cd68c515a 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -480,7 +480,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part Так что даже если у вас `TcpExtListenOverflows` (из `nstat`) не на нуле и это число растет, для сервера ClickHouse это не повод увеличивать значение по умолчанию, поскольку: -- обычно если 4096 недостаточно, это говорит о внутренних проблемах ClickHouse с масштабированием, +- обычно если 4096 недостаточно, это говорит о внутренних проблемах ClickHouse с масштабированием, так что лучше сообщить о проблеме, так что лучше сообщить о проблеме, - и это не значит, что сервер сможет принять еще больше подключений в дальнейшем (а если и сможет, клиенты, вероятно, отсоединятся). From f3d1507b2176be36fb99210c8d51311e6aa8bbd9 Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Thu, 21 Oct 2021 22:47:40 +0300 Subject: [PATCH 068/396] Update docs/ru/operations/server-configuration-parameters/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index c9cd68c515a..3c6a0d8436e 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -482,7 +482,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part по умолчанию, поскольку: - обычно если 4096 недостаточно, это говорит о внутренних проблемах ClickHouse с масштабированием, так что лучше сообщить о проблеме, так что лучше сообщить о проблеме, -- и это не значит, что сервер сможет принять еще больше подключений в дальнейшем +- и это не значит, что сервер сможет принять еще больше подключений в дальнейшем (а если и сможет, клиенты, вероятно, уже отсоединятся). (а если и сможет, клиенты, вероятно, отсоединятся). Примеры: From 723e8e2287d3b283c97ee80bfa445ad23c0b0e4e Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Thu, 21 Oct 2021 22:47:47 +0300 Subject: [PATCH 069/396] Update docs/ru/operations/server-configuration-parameters/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 3c6a0d8436e..70b92b54bb5 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -477,7 +477,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part - значение по умолчанию достаточно велико, - для принятия соединений клиента у сервера есть отдельный поток. -Так что даже если у вас `TcpExtListenOverflows` (из `nstat`) не на нуле +Так что даже если у вас `TcpExtListenOverflows` (из `nstat`) ненулевой и растет для сервера ClickHouse, это не повод увеличивать значение по умолчанию, поскольку: и это число растет, для сервера ClickHouse это не повод увеличивать значение по умолчанию, поскольку: - обычно если 4096 недостаточно, это говорит о внутренних проблемах ClickHouse с масштабированием, так что лучше сообщить о проблеме, From d2bcd549f393298453dd8697587e99ea46693369 Mon Sep 17 00:00:00 2001 From: karnevil13 Date: Thu, 21 Oct 2021 23:03:03 +0300 Subject: [PATCH 070/396] line splits fixed --- .../server-configuration-parameters/settings.md | 14 +++++--------- .../server-configuration-parameters/settings.md | 14 +++++--------- 2 files changed, 10 insertions(+), 18 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 425853553ed..975ca139ff0 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -482,16 +482,12 @@ Backlog (queue size of pending connections) of the listen socket. Default value: `4096` (as in linux [5.4+](https://git.kernel.org/pub/scm/linux/kernel/git/torvalds/linux.git/commit/?id=19f92a030ca6d772ab44b22ee6a01378a8cb32d4)). Usually this value does not need to be changed, since: -- default value is large enough, -- and for accepting client's connections server has separate thread. +- default value is large enough, +- and for accepting client's connections server has separate thread. -So even if you have `TcpExtListenOverflows` (from `nstat`) non zero and this -counter grows for ClickHouse server it does not mean that this value need to be -increased, since: -- usually if 4096 is not enough it shows some internal ClickHouse scaling - issue, so it is better to report an issue. -- and it does not mean that the server can handle more connections later (and - even if it can, clients can already goes away / disconnect). +So even if you have `TcpExtListenOverflows` (from `nstat`) non zero and this counter grows for ClickHouse server it does not mean that this value need to be increased, since: +- usually if 4096 is not enough it shows some internal ClickHouse scaling issue, so it is better to report an issue. +- and it does not mean that the server can handle more connections later (and even if it can, clients can already goes away / disconnect). Examples: diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index cb53e245dd9..b2e25e715e5 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -474,16 +474,12 @@ ClickHouse проверяет условия для `min_part_size` и `min_part Значение по умолчанию: `4096` (как в linux [5.4+](https://git.kernel.org/pub/scm/linux/kernel/git/torvalds/linux.git/commit/?id=19f92a030ca6d772ab44b22ee6a01378a8cb32d4)). Обычно это значение незачем менять по следующим причинам: -- значение по умолчанию достаточно велико, -- для принятия соединения клиента у сервера есть отдельный поток. +- значение по умолчанию достаточно велико, +- для принятия соединения клиента у сервера есть отдельный поток. -Так что даже если у вас `TcpExtListenOverflows` (из `nstat`) не на нуле -и это число растет, для сервера ClickHouse это не повод увеличивать значение -по умолчанию, поскольку: -- обычно если 4096 недостаточно, это говорит о внутренних проблемах ClickHouse с масштабированием, - так что лучше сообщить о проблеме, -- и это не значит, что сервер сможет принять еще больше подключений в дальнейшем -(а если и сможет, клиенты, вероятно, отсоединятся). +Так что даже если у вас `TcpExtListenOverflows` (из `nstat`) ненулевой и растет для сервера ClickHouse, это не повод увеличивать значение по умолчанию, поскольку: +- обычно если 4096 недостаточно, это говорит о внутренних проблемах ClickHouse с масштабированием, так что лучше сообщить о проблеме, +- и это не значит, что сервер сможет принять еще больше подключений в дальнейшем (а если и сможет, клиенты, вероятно, уже отсоединятся). Примеры: From 8e2b9eb86017d9eda6fbd1b91d53aa765fb8c3e3 Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Fri, 22 Oct 2021 09:59:41 +0300 Subject: [PATCH 071/396] Update docs/en/operations/server-configuration-parameters/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/en/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 975ca139ff0..aa12c0bea1e 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -487,7 +487,7 @@ Usually this value does not need to be changed, since: So even if you have `TcpExtListenOverflows` (from `nstat`) non zero and this counter grows for ClickHouse server it does not mean that this value need to be increased, since: - usually if 4096 is not enough it shows some internal ClickHouse scaling issue, so it is better to report an issue. -- and it does not mean that the server can handle more connections later (and even if it can, clients can already goes away / disconnect). +- and it does not mean that the server can handle more connections later (and even if it can, clients can already go away / disconnect). Examples: From 0a2437fe954ab37e2288b8a57a02f6d0dd5777c5 Mon Sep 17 00:00:00 2001 From: karnevil13 Date: Fri, 22 Oct 2021 10:10:25 +0300 Subject: [PATCH 072/396] Update settings.md --- docs/en/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index aa12c0bea1e..6590b8f5931 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -487,7 +487,7 @@ Usually this value does not need to be changed, since: So even if you have `TcpExtListenOverflows` (from `nstat`) non zero and this counter grows for ClickHouse server it does not mean that this value need to be increased, since: - usually if 4096 is not enough it shows some internal ClickHouse scaling issue, so it is better to report an issue. -- and it does not mean that the server can handle more connections later (and even if it can, clients can already go away / disconnect). +- and it does not mean that the server can handle more connections later (and even if it could, by that moment clients may be gone or disconnected). Examples: From 28f59a109955d8d0f01e054a24ac6ce79144cbfe Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 22 Oct 2021 15:27:29 +0300 Subject: [PATCH 073/396] Update by dev comments. --- docs/en/operations/settings/settings.md | 17 ++++++++++++----- docs/ru/operations/settings/settings.md | 2 +- 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index f81cbb989ee..940eb0b95f8 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1751,9 +1751,11 @@ Do not merge aggregation states from different servers for distributed query pro Possible values: -- 0 — Disabled (final query processing is done on the initiator node). -- 1 - Do not merge aggregation states from different servers for distributed query processing (query completelly processed on the shard, initiator only proxy the data), can be used in case it is for certain that there are different keys on different shards. -- 2 - Same as `1` but applies `ORDER BY` and `LIMIT` (it is not possible when the query processed completelly on the remote node, like for `distributed_group_by_no_merge=1`) on the initiator (can be used for queries with `ORDER BY` and/or `LIMIT`). +- `0` — Disabled (final query processing is done on the initiator node). +- `1` - Do not merge aggregation states from different servers for distributed query processing (query completelly processed on the shard, initiator only proxy the data), can be used in case it is for certain that there are different keys on different shards. +- `2` - Same as `1` but applies `ORDER BY` and `LIMIT` (it is not possible when the query processed completelly on the remote node, like for `distributed_group_by_no_merge=1`) on the initiator (can be used for queries with `ORDER BY` and/or `LIMIT`). + +Default value: `0` **Example** @@ -1784,11 +1786,15 @@ FORMAT PrettyCompactMonoBlock └───────┘ ``` -Default value: 0 +See also: + +- [distributed_group_by_no_merge](#distributed-group-by-no-merge) ## distributed_push_down_limit {#distributed-push-down-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. +Enables or disables [LIMIT](#limit) applying on each shard separatelly. It works for `GROUP BY`, but only if enabled [optimize_distributed_group_by_sharding_key](#optimize-distributed-group-by-sharding-key) or [distributed_group_by_no_merge](#distributed-group-by-no-merge) settings. + +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: @@ -1906,6 +1912,7 @@ Default value: 0 See also: - [distributed_group_by_no_merge](#distributed-group-by-no-merge) +- [distributed_push_down_limit](#distributed-push-down-limit) - [optimize_skip_unused_shards](#optimize-skip-unused-shards) !!! note "Note" diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 795249b5499..7b7f3143fbc 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1707,7 +1707,7 @@ ClickHouse генерирует исключение ## distributed_push_down_limit {#distributed-push-down-limit} -Включает или отключает [LIMIT](#limit), применяемый к каждому шарду по отдельности. +Включает или отключает [LIMIT](#limit), применяемый к каждому шарду по отдельности. Работает для `GROUP BY`, но только, если включены настройки `optimize_distributed_group_by_sharding_key` или `distributed_group_by_no_merge`. Обычно, включение данной настройки не требуется, так как это будет сделано автоматически, если это возможно, например, для простого запроса SELECT FROM LIMIT. From 510598fc3b35d786310fa427236ae6359de610fa Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 22 Oct 2021 17:58:21 +0300 Subject: [PATCH 074/396] 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 940eb0b95f8..5de29002c34 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1792,7 +1792,7 @@ See also: ## distributed_push_down_limit {#distributed-push-down-limit} -Enables or disables [LIMIT](#limit) applying on each shard separatelly. It works for `GROUP BY`, but only if enabled [optimize_distributed_group_by_sharding_key](#optimize-distributed-group-by-sharding-key) or [distributed_group_by_no_merge](#distributed-group-by-no-merge) settings. +Enables or disables [LIMIT](#limit) applying on each shard separately. It works for `GROUP BY`, but only if at least one of the settings [optimize_distributed_group_by_sharding_key](#optimize-distributed-group-by-sharding-key) or [distributed_group_by_no_merge](#distributed-group-by-no-merge) is enabled. 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. From d590a664ef64c32dbb49f1605e98a930f9365400 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Fri, 22 Oct 2021 19:45:43 +0300 Subject: [PATCH 075/396] Update information_schema.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил ссылку. --- docs/en/operations/system-tables/information_schema.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/information_schema.md b/docs/en/operations/system-tables/information_schema.md index e5c1af7d43a..eb861b5ac6e 100644 --- a/docs/en/operations/system-tables/information_schema.md +++ b/docs/en/operations/system-tables/information_schema.md @@ -114,4 +114,4 @@ is_trigger_insertable_into: NO - [View](../../sql-reference/statements/create/view.md). - [system.tables](../../operations/system-tables/tables.md). - [system.columns](../../operations/system-tables/columns.md). -- [system.views](../../operations/system-tables/views.md). +- [system.query_views_log](../../operations/system-tables/query_views_log.md). From eaca39ba05271bfeb6af9bd7bcaf2aa47bbcc2dc Mon Sep 17 00:00:00 2001 From: Pavel Medvedev Date: Sat, 23 Oct 2021 02:54:58 +0200 Subject: [PATCH 076/396] use cgroup memory limit in getMemoryAmountOrZero Try to read the memory amount from /sys/fs/cgroup/memory/memory.limit_in_bytes See issue # --- base/base/getMemoryAmount.cpp | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/base/base/getMemoryAmount.cpp b/base/base/getMemoryAmount.cpp index 59ad10335ed..52a06389432 100644 --- a/base/base/getMemoryAmount.cpp +++ b/base/base/getMemoryAmount.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -15,6 +16,17 @@ */ uint64_t getMemoryAmountOrZero() { +#if defined(OS_LINUX) + // Try to lookup at the Cgroup limit + std::ifstream cgroup_limit("/sys/fs/cgroup/memory/memory.limit_in_bytes"); + if (cgroup_limit.is_open()) + { + uint64 amount = 0; // in case of read error + cgroup_limit >> amount; + return amount; + } +#endif + int64_t num_pages = sysconf(_SC_PHYS_PAGES); if (num_pages <= 0) return 0; From 705c1b957d89712884e7b100151f826c5b581a6f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 23 Oct 2021 05:27:15 +0300 Subject: [PATCH 077/396] Update getMemoryAmount.cpp --- base/base/getMemoryAmount.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/base/getMemoryAmount.cpp b/base/base/getMemoryAmount.cpp index 52a06389432..8240f82fc67 100644 --- a/base/base/getMemoryAmount.cpp +++ b/base/base/getMemoryAmount.cpp @@ -21,7 +21,7 @@ uint64_t getMemoryAmountOrZero() std::ifstream cgroup_limit("/sys/fs/cgroup/memory/memory.limit_in_bytes"); if (cgroup_limit.is_open()) { - uint64 amount = 0; // in case of read error + uint64_t amount = 0; // in case of read error cgroup_limit >> amount; return amount; } From 2c4d40c26118bc99ec1f79282fd2404997fffa01 Mon Sep 17 00:00:00 2001 From: lehasm Date: Sat, 23 Oct 2021 17:51:20 +0300 Subject: [PATCH 078/396] Update docs/en/sql-reference/statements/describe-table.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/statements/describe-table.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/describe-table.md b/docs/en/sql-reference/statements/describe-table.md index 1fd1befed53..069386e77cc 100644 --- a/docs/en/sql-reference/statements/describe-table.md +++ b/docs/en/sql-reference/statements/describe-table.md @@ -18,7 +18,7 @@ The `DESCRIBE` statement returns a row for each table column with the following - `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. +- `default_expression` — An expression specified after the `DEFAULT` clause. - `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. From 31e1ae6037fb18291cfec361a561125380aec548 Mon Sep 17 00:00:00 2001 From: lehasm Date: Sat, 23 Oct 2021 17:54:24 +0300 Subject: [PATCH 079/396] Update docs/en/sql-reference/statements/describe-table.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/statements/describe-table.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/describe-table.md b/docs/en/sql-reference/statements/describe-table.md index 069386e77cc..69039cf93f0 100644 --- a/docs/en/sql-reference/statements/describe-table.md +++ b/docs/en/sql-reference/statements/describe-table.md @@ -22,7 +22,7 @@ The `DESCRIBE` statement returns a row for each table column with the following - `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. +- `is_subcolumn` - A flag that equals `1` for internal subcolumns. It is included into the result only if subcolumn description is enabled. 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. From 20c842aef58f70fde0e06d40c7cee315f4fc5c8f Mon Sep 17 00:00:00 2001 From: Alexey Date: Sat, 23 Oct 2021 15:09:12 +0000 Subject: [PATCH 080/396] some en updates --- docs/en/sql-reference/statements/describe-table.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/statements/describe-table.md b/docs/en/sql-reference/statements/describe-table.md index 69039cf93f0..b18575d8896 100644 --- a/docs/en/sql-reference/statements/describe-table.md +++ b/docs/en/sql-reference/statements/describe-table.md @@ -24,9 +24,9 @@ The `DESCRIBE` statement returns a row for each table column with the following - `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 only if subcolumn description is enabled. -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. +All columns in [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) data structures 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. +To show internal subcolumns of a [Tuple](../../sql-reference/data-types/tuple.md) or a [Map](../../sql-reference/data-types/map.md#map-subcolumns), use the [describe_include_subcolumns](../../operations/settings/settings.md#describe_include_subcolumns) setting. **Example** @@ -52,7 +52,7 @@ Result: └──────┴───────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ ``` -The second query additionally shows subcolumn information: +The second query additionally shows subcolumns: ``` text ┌─name──────┬─type──────────────────────────┬─default_type─┬─default_expression─┬─comment─┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┐ From 3103e9970de043e49e5338edbe521316fccee19e Mon Sep 17 00:00:00 2001 From: Alexey Date: Sat, 23 Oct 2021 20:23:32 +0000 Subject: [PATCH 081/396] format fixed. LLink added --- docs/en/sql-reference/statements/describe-table.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/statements/describe-table.md b/docs/en/sql-reference/statements/describe-table.md index b18575d8896..97247912233 100644 --- a/docs/en/sql-reference/statements/describe-table.md +++ b/docs/en/sql-reference/statements/describe-table.md @@ -3,7 +3,7 @@ toc_priority: 42 toc_title: DESCRIBE --- -# DESCRIBE TABLE Statement {#misc-describe-table} +# DESCRIBE TABLE {#misc-describe-table} Returns information about table columns. @@ -20,9 +20,9 @@ The `DESCRIBE` statement returns a row for each table column with the following - `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 [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 only if subcolumn description is enabled. +- `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 only if subcolumn description is enabled by the [describe_include_subcolumns](../../operations/settings/settings.md#describe_include_subcolumns) setting. All columns in [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) data structures are described separately. The name of each column is prefixed with a parent column name and a dot. @@ -66,4 +66,4 @@ The second query additionally shows subcolumns: **See Also** -- [describe_include_subcolumns](../../operations/settings/settings.md#describe_include_subcolumns) setting. \ No newline at end of file +- [describe_include_subcolumns](../../operations/settings/settings.md#describe_include_subcolumns) setting. From 0583a4bdd36e6c6db11ca8f56b8be66d50675ac3 Mon Sep 17 00:00:00 2001 From: Alexey Date: Sat, 23 Oct 2021 20:24:03 +0000 Subject: [PATCH 082/396] ru --- docs/ru/operations/settings/settings.md | 15 +++++ .../statements/describe-table.md | 63 ++++++++++++++++--- 2 files changed, 69 insertions(+), 9 deletions(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index e6e697e5eb6..2ec9edb460b 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3603,3 +3603,18 @@ SELECT * FROM positional_arguments ORDER BY 2,3; **См. также** - настройка [optimize_move_to_prewhere](#optimize_move_to_prewhere) + +## describe_include_subcolumns {#describe_include_subcolumns} + +Разрешает описание подстолбцов при выполнении запроса [DESCRIBE](../../sql-reference/statements/describe-table.md). Настройка действует на подстолбцы внутри столбцов типов [Tuple](../../sql-reference/data-types/tuple.md) или [Map](../../sql-reference/data-types/map.md#map-subcolumns). + +Возможные значения: + +- 0 — подстолбцы не включаются в результат запросов `DESCRIBE`. +- 1 — подстолбцы включаются в результат запросов `DESCRIBE`. + +Значение по умолчанию: `0`. + +**Пример** + +Смотрите пример запроса [DESCRIBE](../../sql-reference/statements/describe-table.md). \ No newline at end of file diff --git a/docs/ru/sql-reference/statements/describe-table.md b/docs/ru/sql-reference/statements/describe-table.md index c66dbb66521..3ebb28909ba 100644 --- a/docs/ru/sql-reference/statements/describe-table.md +++ b/docs/ru/sql-reference/statements/describe-table.md @@ -3,21 +3,66 @@ toc_priority: 42 toc_title: DESCRIBE --- -# DESCRIBE TABLE Statement {#misc-describe-table} +# DESCRIBE TABLE {#misc-describe-table} + +Возвращает описание столбцов таблицы. + +**Синтаксис** ``` sql DESC|DESCRIBE TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] ``` -Возвращает описание столбцов таблицы. +Запрос `DESCRIBE` для каждого столбца таблицы возвращает строку со следующими значениями типа [String](../../sql-reference/data-types/string.md): -Результат запроса содержит столбцы (все столбцы имеют тип String): - -- `name` — имя столбца таблицы; -- `type`— тип столбца; -- `default_type` — в каком виде задано [выражение для значения по умолчанию](../../sql-reference/statements/create/table.md#create-default-values): `DEFAULT`, `MATERIALIZED` или `ALIAS`. Столбец содержит пустую строку, если значение по умолчанию не задано. +- `name` — имя столбца; +- `type` — тип столбца; +- `default_type` — вид [выражения для значения по умолчанию](../../sql-reference/statements/create/table.md#create-default-values): `DEFAULT`, `MATERIALIZED` или `ALIAS`. Если значение по умолчанию не задано, то возвращается пустая строка; - `default_expression` — значение, заданное в секции `DEFAULT`; -- `comment_expression` — комментарий к столбцу. +- `comment` — [комментарий](../../sql-reference/statements/alter/column.md#alter_comment-column); +- `codec_expression` — [кодек](../../sql-reference/statements/create/table.md#codecs), который применяется к столбцу; +- `ttl_expression` — выражение [TTL](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl); +- `is_subcolumn` — флаг, который равен `1` для внутренних подстолбцов. Он появляется в результате, только если описание подстолбцов разрешено настройкой [describe_include_subcolumns](../../operations/settings/settings.md#describe_include_subcolumns). -Вложенные структуры данных выводятся в «развёрнутом» виде. То есть, каждый столбец - по отдельности, с именем через точку. +Каждый столбец [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) структур описывается отдельно. Перед его именем ставится имя родительского столбца с точкой. +Чтобы отобразить внутренние подстолбцы [Tuple](../../sql-reference/data-types/tuple.md) или [Map](../../sql-reference/data-types/map.md#map-subcolumns), нужно включить настройку [describe_include_subcolumns](../../operations/settings/settings.md#describe_include_subcolumns). +**Пример** + +Запрос: + +``` 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; +``` + +Результат: + +``` 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) │ │ │ │ │ │ +└──────┴───────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘ +``` + +Второй запрос дополнительно выводит информацию о подстолбцах: + +``` 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 │ +└───────────┴───────────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┴──────────────┘ +``` + +**См. также** + +- настройка [describe_include_subcolumns](../../operations/settings/settings.md#describe_include_subcolumns). From 60a672ab33f222281f4c1c46cd4f7df22b7443ef Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Sun, 24 Oct 2021 13:27:14 +0300 Subject: [PATCH 083/396] token function adding --- .../functions/splitting-merging-functions.md | 26 ++++++++++ .../functions/splitting-merging-functions.md | 52 ++++++++++++++++++- 2 files changed, 77 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/splitting-merging-functions.md b/docs/en/sql-reference/functions/splitting-merging-functions.md index 718d5a977b9..7d31610eec1 100644 --- a/docs/en/sql-reference/functions/splitting-merging-functions.md +++ b/docs/en/sql-reference/functions/splitting-merging-functions.md @@ -232,6 +232,32 @@ SELECT alphaTokens('abca1abc'); └─────────────────────────┘ ``` +## tokens {#tokens} + +Splits a string into tokens using non-alphanumeric ASCII characters as separators. + +**Arguments** + +- `input_string` — The set of bytes. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +Returns an array of tokens from input string. + +Type: [Array](../data-types/array.md). + +**Example** + +``` sql +SELECT tokens('test1,;\\ test2,;\\ test3,;\\ test4') AS tokens; +``` + +``` text +┌─tokens────────────────────────────┐ +│ ['test1','test2','test3','test4'] │ +└───────────────────────────────────┘ +``` + ## extractAllGroups(text, regexp) {#extractallgroups} Extracts all groups from non-overlapping substrings matched by a regular expression. diff --git a/docs/ru/sql-reference/functions/splitting-merging-functions.md b/docs/ru/sql-reference/functions/splitting-merging-functions.md index efe74dba043..228882d8734 100644 --- a/docs/ru/sql-reference/functions/splitting-merging-functions.md +++ b/docs/ru/sql-reference/functions/splitting-merging-functions.md @@ -8,7 +8,31 @@ toc_title: "Функции разбиения и слияния строк и м ## splitByChar(separator, s) {#splitbycharseparator-s} Разбивает строку на подстроки, используя в качестве разделителя `separator`. -separator должен быть константной строкой из ровно одного символа. +separator должен быть константной строкой из ## tokens {#tokens} + +Split string into tokens using non-alpha numeric ASCII characters as separators. + +**Arguments** + +- `input_string` — The set of bytes. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +Returnes an array of tokens from input string. + +Type: [Array(String)](../data-types/array.md). + +**Example** + +``` sql +SELECT tokens('test1,;\\ test2,;\\ test3,;\\ test4') AS tokens; +``` + +``` text +┌─tokens────────────────────────────┐ +│ ['test1','test2','test3','test4'] │ +└───────────────────────────────────┘ +```ровно одного символа. Возвращается массив выделенных подстрок. Могут выделяться пустые подстроки, если разделитель идёт в начале или в конце строки, или если идёт более одного разделителя подряд. **Синтаксис** @@ -232,3 +256,29 @@ SELECT alphaTokens('abca1abc'); │ ['abca','abc'] │ └─────────────────────────┘ ``` + +## tokens {#tokens} + +Разбивает строку на токены, используя в качестве разделителей не буквенно-цифровые символы ASCII. + +**Аргументы** + +- `input_string` — набор байтов. [String](../../sql-reference/data-types/string.md). + +**Возвращаемые значения** + +Возвращает массив токенов. + +Тип: [Array](../data-types/array.md). + +**Пример** + +``` sql +SELECT tokens('test1,;\\ test2,;\\ test3,;\\ test4') AS tokens; +``` + +``` text +┌─tokens────────────────────────────┐ +│ ['test1','test2','test3','test4'] │ +└───────────────────────────────────┘ +``` \ No newline at end of file From 3a4f71d8a333b1df5d81ea5849e102ddb80ebdde Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Sun, 24 Oct 2021 14:06:46 +0300 Subject: [PATCH 084/396] correction of error in md-file --- .../functions/splitting-merging-functions.md | 52 +------------------ 1 file changed, 1 insertion(+), 51 deletions(-) diff --git a/docs/ru/sql-reference/functions/splitting-merging-functions.md b/docs/ru/sql-reference/functions/splitting-merging-functions.md index 228882d8734..efe74dba043 100644 --- a/docs/ru/sql-reference/functions/splitting-merging-functions.md +++ b/docs/ru/sql-reference/functions/splitting-merging-functions.md @@ -8,31 +8,7 @@ toc_title: "Функции разбиения и слияния строк и м ## splitByChar(separator, s) {#splitbycharseparator-s} Разбивает строку на подстроки, используя в качестве разделителя `separator`. -separator должен быть константной строкой из ## tokens {#tokens} - -Split string into tokens using non-alpha numeric ASCII characters as separators. - -**Arguments** - -- `input_string` — The set of bytes. [String](../../sql-reference/data-types/string.md). - -**Returned value** - -Returnes an array of tokens from input string. - -Type: [Array(String)](../data-types/array.md). - -**Example** - -``` sql -SELECT tokens('test1,;\\ test2,;\\ test3,;\\ test4') AS tokens; -``` - -``` text -┌─tokens────────────────────────────┐ -│ ['test1','test2','test3','test4'] │ -└───────────────────────────────────┘ -```ровно одного символа. +separator должен быть константной строкой из ровно одного символа. Возвращается массив выделенных подстрок. Могут выделяться пустые подстроки, если разделитель идёт в начале или в конце строки, или если идёт более одного разделителя подряд. **Синтаксис** @@ -256,29 +232,3 @@ SELECT alphaTokens('abca1abc'); │ ['abca','abc'] │ └─────────────────────────┘ ``` - -## tokens {#tokens} - -Разбивает строку на токены, используя в качестве разделителей не буквенно-цифровые символы ASCII. - -**Аргументы** - -- `input_string` — набор байтов. [String](../../sql-reference/data-types/string.md). - -**Возвращаемые значения** - -Возвращает массив токенов. - -Тип: [Array](../data-types/array.md). - -**Пример** - -``` sql -SELECT tokens('test1,;\\ test2,;\\ test3,;\\ test4') AS tokens; -``` - -``` text -┌─tokens────────────────────────────┐ -│ ['test1','test2','test3','test4'] │ -└───────────────────────────────────┘ -``` \ No newline at end of file From 26ea5a1452e010137070d008c85254a6c6781848 Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Sun, 24 Oct 2021 14:22:34 +0300 Subject: [PATCH 085/396] tokens function in russian --- .../functions/splitting-merging-functions.md | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/docs/ru/sql-reference/functions/splitting-merging-functions.md b/docs/ru/sql-reference/functions/splitting-merging-functions.md index efe74dba043..46a597dd433 100644 --- a/docs/ru/sql-reference/functions/splitting-merging-functions.md +++ b/docs/ru/sql-reference/functions/splitting-merging-functions.md @@ -232,3 +232,29 @@ SELECT alphaTokens('abca1abc'); │ ['abca','abc'] │ └─────────────────────────┘ ``` + +## tokens {#tokens} + +Разбивает строку на токены, используя в качестве разделителей не буквенно-цифровые символы ASCII. + +**Аргументы** + +- `input_string` — набор байтов. [String](../../sql-reference/data-types/string.md). + +**Возвращаемые значения** + +Возвращает массив токенов. + +Тип: [Array](../data-types/array.md). + +**Пример** + +``` sql +SELECT tokens('test1,;\\ test2,;\\ test3,;\\ test4') AS tokens; +``` + +``` text +┌─tokens────────────────────────────┐ +│ ['test1','test2','test3','test4'] │ +└───────────────────────────────────┘ +``` \ No newline at end of file From 43bbc72e17a9a8f964a553b46d46d3502f608849 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Sun, 24 Oct 2021 19:38:22 +0300 Subject: [PATCH 086/396] Update with PR #30577 --- docs/en/operations/settings/settings.md | 20 ++++++++++++++++---- docs/ru/operations/settings/settings.md | 18 ++++++++++++++---- 2 files changed, 30 insertions(+), 8 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 940eb0b95f8..50de46359bb 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1792,9 +1792,18 @@ See also: ## distributed_push_down_limit {#distributed-push-down-limit} -Enables or disables [LIMIT](#limit) applying on each shard separatelly. It works for `GROUP BY`, but only if enabled [optimize_distributed_group_by_sharding_key](#optimize-distributed-group-by-sharding-key) or [distributed_group_by_no_merge](#distributed-group-by-no-merge) settings. +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. +This will allow to avoid: +- sending extra rows over network; +- processing rows behind the limit on the initiator. + +You cannot get inaccurate results anymore, since `distributed_push_down_limit` changes query execution only if at least one of the conditions met: +- [distributed_group_by_no_merge](#distributed-group-by-no-merge) > 0. +- Query **does not have** `GROUP BY`/`DISTINCT`/`LIMIT BY`, but it has `ORDER BY`/`LIMIT`. +- Query **has** `GROUP BY`/`DISTINCT`/`LIMIT BY` with `ORDER BY`/`LIMIT` and: + - [optimize_skip_unused_shards](#optimize-skip-unused-shards) is enabled. + - [optimize_distributed_group_by_sharding_key](#optimize-distributed-group-by-sharding-key) is enabled. Possible values: @@ -1803,8 +1812,11 @@ Possible values: Default value: `1`. -!!! note "Note" - That with this setting the result of the query may be inaccurate. +See also: + +- [distributed_group_by_no_merge](#distributed-group-by-no-merge) +- [optimize_skip_unused_shards](#optimize-skip-unused-shards) +- [optimize_distributed_group_by_sharding_key](#optimize-distributed-group-by-sharding-key) ## optimize_skip_unused_shards_limit {#optimize-skip-unused-shards-limit} diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 7b7f3143fbc..84df665d49d 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1707,9 +1707,18 @@ ClickHouse генерирует исключение ## distributed_push_down_limit {#distributed-push-down-limit} -Включает или отключает [LIMIT](#limit), применяемый к каждому шарду по отдельности. Работает для `GROUP BY`, но только, если включены настройки `optimize_distributed_group_by_sharding_key` или `distributed_group_by_no_merge`. +Включает или отключает [LIMIT](#limit), применяемый к каждому шарду по отдельности. -Обычно, включение данной настройки не требуется, так как это будет сделано автоматически, если это возможно, например, для простого запроса SELECT FROM LIMIT. +Это позволяет избежать: +- отправки дополнительных строк по сети; +- обработки строк за пределами ограничения для инициатора. + +Вы больше не сможете получить неточные результаты, так как `distributed_push_down_limit` изменяет выполнение запроса только в том случае, если выполнено хотя бы одно из условий: +- `distributed_group_by_no_merge` > 0. +- запрос **не содержит** `GROUP BY`/`DISTINCT`/`LIMIT BY`, но содержит `ORDER BY`/`LIMIT`. +- запрос **содержит** `GROUP BY`/`DISTINCT`/`LIMIT BY` с `ORDER BY`/`LIMIT` и: + - включена настройка [optimize_skip_unused_shards](#optimize-skip-unused-shards). + - включена настройка `optimize_distributed_group_by_sharding_key`. Возможные значения: @@ -1718,8 +1727,9 @@ ClickHouse генерирует исключение Значение по умолчанию: `1`. -!!! note "Примечание" - При использованаии этой настройки, результат запроса может быть неточным. +См. также: + +- [optimize_skip_unused_shards](#optimize-skip-unused-shards) ## optimize_skip_unused_shards {#optimize-skip-unused-shards} From 47bd7b94208857ab5228431109afd9af07eda3be Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Sun, 24 Oct 2021 19:57:08 +0300 Subject: [PATCH 087/396] List fix. --- docs/en/operations/settings/settings.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 50de46359bb..287a8b952d1 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1795,15 +1795,15 @@ See also: Enables or disables [LIMIT](#limit) applying on each shard separatelly. This will allow to avoid: -- sending extra rows over network; -- processing rows behind the limit on the initiator. +- Sending extra rows over network; +- Processing rows behind the limit on the initiator. You cannot get inaccurate results anymore, since `distributed_push_down_limit` changes query execution only if at least one of the conditions met: -- [distributed_group_by_no_merge](#distributed-group-by-no-merge) > 0. -- Query **does not have** `GROUP BY`/`DISTINCT`/`LIMIT BY`, but it has `ORDER BY`/`LIMIT`. -- Query **has** `GROUP BY`/`DISTINCT`/`LIMIT BY` with `ORDER BY`/`LIMIT` and: - - [optimize_skip_unused_shards](#optimize-skip-unused-shards) is enabled. - - [optimize_distributed_group_by_sharding_key](#optimize-distributed-group-by-sharding-key) is enabled. +- [distributed_group_by_no_merge](#distributed-group-by-no-merge) > 0. +- Query **does not have** `GROUP BY`/`DISTINCT`/`LIMIT BY`, but it has `ORDER BY`/`LIMIT`. +- Query **has** `GROUP BY`/`DISTINCT`/`LIMIT BY` with `ORDER BY`/`LIMIT` and: + - [optimize_skip_unused_shards](#optimize-skip-unused-shards) is enabled. + - [optimize_distributed_group_by_sharding_key](#optimize-distributed-group-by-sharding-key) is enabled. Possible values: From 1070ad10f365eb33e29a766c9f1e9e739b07ef82 Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Sun, 24 Oct 2021 20:54:29 +0300 Subject: [PATCH 088/396] Update splitting-merging-functions.md --- .../functions/splitting-merging-functions.md | 63 ------------------- 1 file changed, 63 deletions(-) diff --git a/docs/en/sql-reference/functions/splitting-merging-functions.md b/docs/en/sql-reference/functions/splitting-merging-functions.md index 611cc1e03f4..718d5a977b9 100644 --- a/docs/en/sql-reference/functions/splitting-merging-functions.md +++ b/docs/en/sql-reference/functions/splitting-merging-functions.md @@ -232,32 +232,6 @@ SELECT alphaTokens('abca1abc'); └─────────────────────────┘ ``` -## tokens {#tokens} - -Splits a string into tokens using non-alphanumeric ASCII characters as separators. - -**Arguments** - -- `input_string` — The set of bytes. [String](../../sql-reference/data-types/string.md). - -**Returned value** - -Returns an array of tokens from input string. - -Type: [Array](../data-types/array.md). - -**Example** - -``` sql -SELECT tokens('test1,;\\ test2,;\\ test3,;\\ test4') AS tokens; -``` - -``` text -┌─tokens────────────────────────────┐ -│ ['test1','test2','test3','test4'] │ -└───────────────────────────────────┘ -``` - ## extractAllGroups(text, regexp) {#extractallgroups} Extracts all groups from non-overlapping substrings matched by a regular expression. @@ -296,40 +270,3 @@ Result: │ [['abc','123'],['8','"hkl"']] │ └───────────────────────────────────────────────────────────────────────┘ ``` - -## ngrams {#ngrams} - -Splits the UTF-8 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'] │ -└───────────────────────────────────────────────────┘ -``` From 568a47361284406607f63b18fa9670e49a45e571 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 25 Oct 2021 10:11:48 +0300 Subject: [PATCH 089/396] Update ssl-zookeeper.md --- docs/en/operations/ssl-zookeeper.md | 27 +++------------------------ 1 file changed, 3 insertions(+), 24 deletions(-) diff --git a/docs/en/operations/ssl-zookeeper.md b/docs/en/operations/ssl-zookeeper.md index 7124240e231..d6bc970e63f 100644 --- a/docs/en/operations/ssl-zookeeper.md +++ b/docs/en/operations/ssl-zookeeper.md @@ -5,30 +5,9 @@ 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 should specify `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: - -``` xml - - /etc/clickhouse-server/client.crt - /etc/clickhouse-server/client.key - true - true - sslv2,sslv3 - true - none - - RejectCertificateHandler - - -``` - -2. Add `zookeeper.crt` to trusted certificates. +You can add `zookeeper.crt` to trusted certificates. ``` bash sudo cp zookeeper.crt /usr/local/share/ca-certificates/zookeeper.crt @@ -92,4 +71,4 @@ On unencrypted connection you will see in `tcpdump` output something like this: uota. ``` -On encrypted connection you should not see this. \ No newline at end of file +On encrypted connection you should not see this. From e1891d987bb62c5aa2af052bc501e2eb156cdc74 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 25 Oct 2021 12:11:58 +0300 Subject: [PATCH 090/396] Remove see also. --- docs/en/operations/settings/settings.md | 4 ---- 1 file changed, 4 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 3803745fcb5..2dee8f50dda 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1786,10 +1786,6 @@ FORMAT PrettyCompactMonoBlock └───────┘ ``` -See also: - -- [distributed_group_by_no_merge](#distributed-group-by-no-merge) - ## distributed_push_down_limit {#distributed-push-down-limit} Enables or disables [LIMIT](#limit) applying on each shard separatelly. From 63b814671da5e5ae5af98f90b3c3eb1ac435aa96 Mon Sep 17 00:00:00 2001 From: gyuton Date: Mon, 25 Oct 2021 14:18:51 +0300 Subject: [PATCH 091/396] draft --- .../functions/tuple-functions.md | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 39e59ae2ba9..5be75caf173 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -165,3 +165,26 @@ Result: │ 2 │ └─────────────────┘ ``` + +## tupleToNameValuePairs {#tupletonamevaluepairs} + +Turns a named tuple into an array of name-value pairs. For a `Tuple(a T, b T, ..., c T)` return an `Array(Tuple(String, T), ...)` +in which the `Strings` represent the named fields of the tuple and the `T`s are the values associated to those names. + +**Syntax** + +``` sql +tupleToNameValuePairs() +``` + +**Arguments** + +- `x` — Named tuple. + +**Returned value** + +- + +Type: + +**Example** From 65be1e966849ae873ecd4ca96dd693f2d7f5a0e6 Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Mon, 25 Oct 2021 19:14:49 +0300 Subject: [PATCH 092/396] moving tokens from file to file --- .../functions/string-functions.md | 26 ------------------- 1 file changed, 26 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 05c291bb41d..5f20fae0030 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -313,32 +313,6 @@ SELECT toValidUTF8('\x61\xF0\x80\x80\x80b'); └───────────────────────┘ ``` -## tokens {#tokens} - -Split string into tokens using non-alpha numeric ASCII characters as separators. - -**Arguments** - -- `input_string` — Any set of bytes represented as the [String](../../sql-reference/data-types/string.md) data type object. - -**Returned value** - -- The resulting array of tokens from input string. - -Type: [Array](../data-types/array.md). - -**Example** - -``` sql -SELECT tokens('test1,;\\ test2,;\\ test3,;\\ test4') AS tokens; -``` - -``` text -┌─tokens────────────────────────────┐ -│ ['test1','test2','test3','test4'] │ -└───────────────────────────────────┘ -``` - ## repeat {#repeat} Repeats a string as many times as specified and concatenates the replicated values as a single string. From 1bc542980c2c201045028f18b33b2ecb212a98a3 Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Mon, 25 Oct 2021 19:15:28 +0300 Subject: [PATCH 093/396] adding ngrams --- .../functions/splitting-merging-functions.md | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) diff --git a/docs/en/sql-reference/functions/splitting-merging-functions.md b/docs/en/sql-reference/functions/splitting-merging-functions.md index 179d7934bb1..0323834c5aa 100644 --- a/docs/en/sql-reference/functions/splitting-merging-functions.md +++ b/docs/en/sql-reference/functions/splitting-merging-functions.md @@ -270,3 +270,66 @@ Result: │ [['abc','123'],['8','"hkl"']] │ └───────────────────────────────────────────────────────────────────────┘ ``` + +## ngrams {#ngrams} + +Splits the UTF-8 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'] │ +└───────────────────────────────────────────────────┘ +``` + +## tokens {#tokens} + +Splits a string into tokens using non-alphanumeric ASCII characters as separators. + +**Arguments** + +- `input_string` — Any set of bytes represented as the [String](../../sql-reference/data-types/string.md) data type object. + +**Returned value** + +- The resulting array of tokens from input string. + +Type: [Array](../data-types/array.md). + +**Example** + +``` sql +SELECT tokens('test1,;\\ test2,;\\ test3,;\\ test4') AS tokens; +``` + +``` text +┌─tokens────────────────────────────┐ +│ ['test1','test2','test3','test4'] │ +└───────────────────────────────────┘ +``` \ No newline at end of file From 1d743b9259bca8c29770b22ebe90547ff91d8af4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Oct 2021 19:23:44 +0300 Subject: [PATCH 094/396] Partitoned write --- src/Processors/Sinks/SinkToStorage.h | 3 + src/Storages/HDFS/StorageHDFS.cpp | 112 +++++++++++--- src/Storages/HDFS/StorageHDFS.h | 2 + src/Storages/PartitionedSink.cpp | 128 ++++++++++++++++ src/Storages/PartitionedSink.h | 40 +++++ src/Storages/StorageFile.cpp | 140 ++++++++++++++++-- src/Storages/StorageFile.h | 6 +- tests/integration/test_storage_hdfs/test.py | 18 +++ ...able_function_file_partiotion_by.reference | 6 + ...02105_table_function_file_partiotion_by.sh | 30 ++++ 10 files changed, 447 insertions(+), 38 deletions(-) create mode 100644 src/Storages/PartitionedSink.cpp create mode 100644 src/Storages/PartitionedSink.h create mode 100644 tests/queries/0_stateless/02105_table_function_file_partiotion_by.reference create mode 100755 tests/queries/0_stateless/02105_table_function_file_partiotion_by.sh diff --git a/src/Processors/Sinks/SinkToStorage.h b/src/Processors/Sinks/SinkToStorage.h index 76920777bdc..051ecc8f20a 100644 --- a/src/Processors/Sinks/SinkToStorage.h +++ b/src/Processors/Sinks/SinkToStorage.h @@ -8,6 +8,8 @@ namespace DB /// Sink which is returned from Storage::write. class SinkToStorage : public ExceptionKeepingTransform { +friend class PartitionedSink; + public: explicit SinkToStorage(const Block & header); @@ -35,4 +37,5 @@ public: void consume(Chunk) override {} }; +using SinkPtr = std::shared_ptr; } diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 6e81f5577ab..133080321e3 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -2,30 +2,44 @@ #if USE_HDFS -#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 @@ -224,6 +238,43 @@ private: bool is_first_chunk = true; }; + +class PartitionedHDFSSink : public PartitionedSink +{ +public: + PartitionedHDFSSink( + const ASTPtr & partition_by, + const String & uri_, + const String & format_, + const Block & sample_block_, + ContextPtr context_, + const CompressionMethod compression_method_) + : PartitionedSink(partition_by, context_, sample_block_) + , uri(uri_) + , format(format_) + , sample_block(sample_block_) + , context(context_) + , compression_method(compression_method_) + { + } + + SinkPtr createSinkForPartition(const String & partition_id) override + { + auto path = PartitionedSink::replaceWildcards(uri, partition_id); + PartitionedSink::validatePartitionKey(path, true); + return std::make_shared(path, format, sample_block, context, compression_method); + } + +private: + const String uri; + + const String format; + const Block sample_block; + ContextPtr context; + const CompressionMethod compression_method; +}; + + /* Recursive directory listing with matched paths as a result. * Have the same method in StorageFile. */ @@ -315,13 +366,32 @@ Pipe StorageHDFS::read( return Pipe::unitePipes(std::move(pipes)); } -SinkToStoragePtr StorageHDFS::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) +SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) { - return std::make_shared(uri, - format_name, - metadata_snapshot->getSampleBlock(), - getContext(), - chooseCompressionMethod(uri, compression_method)); + bool has_wildcards = uri.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; + const auto * insert_query = dynamic_cast(query.get()); + bool is_partitioned_implementation = insert_query && insert_query->partition_by && has_wildcards; + + if (is_partitioned_implementation) + { + std::cerr << "partitioned implementation\n"; + return std::make_shared( + insert_query->partition_by, + uri, + format_name, + metadata_snapshot->getSampleBlock(), + getContext(), + chooseCompressionMethod(uri, compression_method)); + } + else + { + std::cerr << "non partitioned implementation\n"; + return std::make_shared(uri, + format_name, + metadata_snapshot->getSampleBlock(), + getContext(), + chooseCompressionMethod(uri, compression_method)); + } } void StorageHDFS::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, ContextPtr context_, TableExclusiveLockHolder &) diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index 6ff8bb3f13f..e2ce1f1dcf8 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -38,6 +38,8 @@ public: NamesAndTypesList getVirtuals() const override; + bool supportsPartitionBy() const override { return true; } + protected: StorageHDFS( const String & uri_, diff --git a/src/Storages/PartitionedSink.cpp b/src/Storages/PartitionedSink.cpp new file mode 100644 index 00000000000..1d13cbe5f94 --- /dev/null +++ b/src/Storages/PartitionedSink.cpp @@ -0,0 +1,128 @@ +#include "PartitionedSink.h" + +#include + +#include +#include +#include +#include + +#include +#include +#include + +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_TEXT; +} + +PartitionedSink::PartitionedSink( + const ASTPtr & partition_by, + ContextPtr context_, + const Block & sample_block_) + : SinkToStorage(sample_block_) + , context(context_) + , sample_block(sample_block_) +{ + std::vector arguments(1, partition_by); + ASTPtr partition_by_string = makeASTFunction(FunctionToString::name, std::move(arguments)); + + auto syntax_result = TreeRewriter(context).analyze(partition_by_string, sample_block.getNamesAndTypesList()); + partition_by_expr = ExpressionAnalyzer(partition_by_string, syntax_result, context).getActions(false); + partition_by_column_name = partition_by_string->getColumnName(); +} + + +SinkPtr PartitionedSink::getSinkForPartition(const String & partition_id) +{ + auto it = sinks.find(partition_id); + if (it == sinks.end()) + { + auto sink = createSinkForPartition(partition_id); + std::tie(it, std::ignore) = sinks.emplace(partition_id, sink); + } + + return it->second; +} + + +void PartitionedSink::consume(Chunk chunk) +{ + const auto & columns = chunk.getColumns(); + + Block block_with_partition_by_expr = sample_block.cloneWithoutColumns(); + block_with_partition_by_expr.setColumns(columns); + partition_by_expr->execute(block_with_partition_by_expr); + + const auto * column = block_with_partition_by_expr.getByName(partition_by_column_name).column.get(); + + std::unordered_map sub_chunks_indices; + IColumn::Selector selector; + for (size_t row = 0; row < chunk.getNumRows(); ++row) + { + auto value = column->getDataAt(row); + auto [it, inserted] = sub_chunks_indices.emplace(value, sub_chunks_indices.size()); + selector.push_back(it->second); + } + + Chunks sub_chunks; + sub_chunks.reserve(sub_chunks_indices.size()); + for (size_t column_index = 0; column_index < columns.size(); ++column_index) + { + MutableColumns column_sub_chunks = columns[column_index]->scatter(sub_chunks_indices.size(), selector); + if (column_index == 0) /// Set sizes for sub-chunks. + { + for (const auto & column_sub_chunk : column_sub_chunks) + { + sub_chunks.emplace_back(Columns(), column_sub_chunk->size()); + } + } + for (size_t sub_chunk_index = 0; sub_chunk_index < column_sub_chunks.size(); ++sub_chunk_index) + { + sub_chunks[sub_chunk_index].addColumn(std::move(column_sub_chunks[sub_chunk_index])); + } + } + + for (const auto & [partition_id, sub_chunk_index] : sub_chunks_indices) + { + getSinkForPartition(partition_id)->consume(std::move(sub_chunks[sub_chunk_index])); + } +} + + +void PartitionedSink::onFinish() +{ + for (auto & [partition_id, sink] : sinks) + { + sink->onFinish(); + } +} + + +void PartitionedSink::validatePartitionKey(const String & str, bool allow_slash) +{ + for (const char * i = str.data(); i != str.data() + str.size(); ++i) + { + if (static_cast(*i) < 0x20 || *i == '{' || *i == '}' || *i == '*' || *i == '?' || (!allow_slash && *i == '/')) + { + /// Need to convert to UInt32 because UInt8 can't be passed to format due to "mixing character types is disallowed". + UInt32 invalid_char_byte = static_cast(static_cast(*i)); + throw DB::Exception( + ErrorCodes::CANNOT_PARSE_TEXT, "Illegal character '\\x{:02x}' in partition id starting with '{}'", + invalid_char_byte, std::string(str.data(), i - str.data())); + } + } +} + + +String PartitionedSink::replaceWildcards(const String & haystack, const String & partition_id) +{ + return boost::replace_all_copy(haystack, PartitionedSink::PARTITION_ID_WILDCARD, partition_id); +} + +} diff --git a/src/Storages/PartitionedSink.h b/src/Storages/PartitionedSink.h new file mode 100644 index 00000000000..53654b089ce --- /dev/null +++ b/src/Storages/PartitionedSink.h @@ -0,0 +1,40 @@ +#include +#include +#include + + +namespace DB +{ + +class PartitionedSink : public SinkToStorage +{ +public: + static constexpr auto PARTITION_ID_WILDCARD = "{_partition_id}"; + + PartitionedSink(const ASTPtr & partition_by, ContextPtr context_, const Block & sample_block_); + + String getName() const override { return "PartitionedSink"; } + + void consume(Chunk chunk) override; + + void onFinish() override; + + virtual SinkPtr createSinkForPartition(const String & partition_id) = 0; + + static void validatePartitionKey(const String & str, bool allow_slash); + + static String replaceWildcards(const String & haystack, const String & partition_id); + +private: + ContextPtr context; + Block sample_block; + + ExpressionActionsPtr partition_by_expr; + String partition_by_column_name; + + std::unordered_map sinks; + + SinkPtr getSinkForPartition(const String & partition_id); +}; + +} diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 978d161852b..1f2b6ba6c60 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -25,6 +26,7 @@ #include #include #include +#include #include #include @@ -189,7 +191,11 @@ StorageFile::StorageFile(const std::string & table_path_, const std::string & us : StorageFile(args) { is_db_table = false; - paths = getPathsList(table_path_, user_files_path, args.getContext(), total_bytes_to_read); + bool has_wildcards = table_path_.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; + if (has_wildcards) + paths = {table_path_}; + else + paths = getPathsList(table_path_, user_files_path, args.getContext(), total_bytes_to_read); if (args.format_name == "Distributed") { @@ -541,22 +547,48 @@ Pipe StorageFile::read( class StorageFileSink final : public SinkToStorage { public: - explicit StorageFileSink( + StorageFileSink( StorageFile & storage_, const StorageMetadataPtr & metadata_snapshot_, - std::unique_lock && lock_, - const CompressionMethod compression_method, - ContextPtr context, - const std::optional & format_settings, - int & flags) + const CompressionMethod compression_method_, + ContextPtr context_, + const std::optional & format_settings_, + int flags_) : SinkToStorage(metadata_snapshot_->getSampleBlock()) , storage(storage_) , metadata_snapshot(metadata_snapshot_) + , compression_method(compression_method_) + , context(context_) + , format_settings(format_settings_) + , flags(flags_) + { + initialize(); + } + + StorageFileSink( + StorageFile & storage_, + const StorageMetadataPtr & metadata_snapshot_, + std::unique_lock && lock_, + const CompressionMethod compression_method_, + ContextPtr context_, + const std::optional & format_settings_, + int flags_) + : SinkToStorage(metadata_snapshot_->getSampleBlock()) + , storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , compression_method(compression_method_) + , context(context_) + , format_settings(format_settings_) + , flags(flags_) , lock(std::move(lock_)) { if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + initialize(); + } + void initialize() + { std::unique_ptr naked_buffer = nullptr; if (storage.use_table_fd) { @@ -608,14 +640,66 @@ public: private: StorageFile & storage; StorageMetadataPtr metadata_snapshot; + const CompressionMethod compression_method; + ContextPtr context; + std::optional format_settings; + int flags; + std::unique_lock lock; + std::unique_ptr write_buf; OutputFormatPtr writer; bool prefix_written{false}; }; +class PartitionedStorageFileSink : public PartitionedSink +{ +public: + PartitionedStorageFileSink( + const ASTPtr & partition_by, + StorageFile & storage_, + const StorageMetadataPtr & metadata_snapshot_, + std::unique_lock && lock_, + const CompressionMethod compression_method_, + ContextPtr context_, + const std::optional & format_settings_, + int & flags_) + : PartitionedSink(partition_by, context_, metadata_snapshot_->getSampleBlock()) + , path(storage_.paths[0]) + , storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , lock(std::move(lock_)) + , compression_method(compression_method_) + , context(context_) + , format_settings(format_settings_) + , flags(flags_) + { + } + + SinkPtr createSinkForPartition(const String & partition_id) override + { + auto partition_path = PartitionedSink::replaceWildcards(path, partition_id); + PartitionedSink::validatePartitionKey(partition_path, true); + storage.paths[0] = partition_path; + return std::make_shared( + storage, metadata_snapshot, compression_method, context, format_settings, flags); + } + +private: + const String path; + StorageFile & storage; + StorageMetadataPtr metadata_snapshot; + std::unique_lock lock; + const CompressionMethod compression_method; + + ContextPtr context; + std::optional format_settings; + int flags; +}; + + SinkToStoragePtr StorageFile::write( - const ASTPtr & /*query*/, + const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) { @@ -634,14 +718,38 @@ SinkToStoragePtr StorageFile::write( fs::create_directories(fs::path(path).parent_path()); } - return std::make_shared( - *this, - metadata_snapshot, - std::unique_lock{rwlock, getLockTimeout(context)}, - chooseCompressionMethod(path, compression_method), - context, - format_settings, - flags); + bool has_wildcards = path.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; + const auto * insert_query = dynamic_cast(query.get()); + bool is_partitioned_implementation = insert_query && insert_query->partition_by && has_wildcards; + + if (is_partitioned_implementation) + { + if (paths.size() != 1) + throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, + "Table '{}' is in readonly mode because of globs in filepath", + getStorageID().getNameForLogs()); + + return std::make_shared( + insert_query->partition_by, + *this, + metadata_snapshot, + std::unique_lock{rwlock, getLockTimeout(context)}, + chooseCompressionMethod(path, compression_method), + context, + format_settings, + flags); + } + else + { + return std::make_shared( + *this, + metadata_snapshot, + std::unique_lock{rwlock, getLockTimeout(context)}, + chooseCompressionMethod(path, compression_method), + context, + format_settings, + flags); + } } bool StorageFile::storesDataOnDisk() const diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 09ea44d1405..9a58bb2a251 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -16,7 +16,9 @@ class StorageFileBlockOutputStream; class StorageFile final : public shared_ptr_helper, public IStorage { - friend struct shared_ptr_helper; +friend struct shared_ptr_helper; +friend class PartitionedStorageFileSink; + public: std::string getName() const override { return "File"; } @@ -66,6 +68,8 @@ public: /// format to read only them. Note: this hack cannot be done with ordinary formats like TSV. bool isColumnOriented() const; + bool supportsPartitionBy() const override { return true; } + protected: friend class StorageFileSource; friend class StorageFileSink; diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index f3c83166b46..22fb22cf60c 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -257,6 +257,24 @@ def test_truncate_table(started_cluster): node1.query("drop table test_truncate") +def test_partition_by(started_cluster): + hdfs_api = started_cluster.hdfs_api + + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + file_name = "test_{_partition_id}" + partition_by = "column3" + values = "(1, 2, 3), (3, 2, 1), (1, 3, 2)" + table_function = f"hdfs('hdfs://hdfs1:9000/{file_name}', 'TSV', '{table_format}')" + + node1.query(f"insert into table function {table_function} PARTITION BY {partition_by} values {values}") + result = node1.query(f"select * from hdfs('hdfs://hdfs1:9000/test_1', 'TSV', '{table_format}')") + assert(result.strip() == "3\t2\t1") + result = node1.query(f"select * from hdfs('hdfs://hdfs1:9000/test_2', 'TSV', '{table_format}')") + assert(result.strip() == "1\t3\t2") + result = node1.query(f"select * from hdfs('hdfs://hdfs1:9000/test_3', 'TSV', '{table_format}')") + assert(result.strip() == "1\t2\t3") + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") diff --git a/tests/queries/0_stateless/02105_table_function_file_partiotion_by.reference b/tests/queries/0_stateless/02105_table_function_file_partiotion_by.reference new file mode 100644 index 00000000000..e6d8f69d9eb --- /dev/null +++ b/tests/queries/0_stateless/02105_table_function_file_partiotion_by.reference @@ -0,0 +1,6 @@ +part 1 +3 2 1 +part 2 +1 3 2 +part 3 +1 2 3 diff --git a/tests/queries/0_stateless/02105_table_function_file_partiotion_by.sh b/tests/queries/0_stateless/02105_table_function_file_partiotion_by.sh new file mode 100755 index 00000000000..2e7e0fede94 --- /dev/null +++ b/tests/queries/0_stateless/02105_table_function_file_partiotion_by.sh @@ -0,0 +1,30 @@ +#!/usr/bin/env bash +# Tags: no-parallel, no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# See 01658_read_file_to_string_column.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}') + +mkdir -p "${user_files_path}/" +chmod 777 ${user_files_path} + +FILE_PATH="${user_files_path}/test_table_function_file" + +function cleanup() +{ + rm -r ${FILE_PATH} +} +trap cleanup EXIT + +values="(1, 2, 3), (3, 2, 1), (1, 3, 2)" +${CLICKHOUSE_CLIENT} --query="insert into table function file('${FILE_PATH}/test_{_partition_id}', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32') PARTITION BY column3 values ${values}"; +echo 'part 1' +${CLICKHOUSE_CLIENT} --query="select * from file('${FILE_PATH}/test_1', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')"; +echo 'part 2' +${CLICKHOUSE_CLIENT} --query="select * from file('${FILE_PATH}/test_2', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')"; +echo 'part 3' +${CLICKHOUSE_CLIENT} --query="select * from file('${FILE_PATH}/test_3', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')"; + From a9ef0359d92456d800b7effb3430c7a5b3c0b070 Mon Sep 17 00:00:00 2001 From: gyuton Date: Tue, 26 Oct 2021 00:01:31 +0300 Subject: [PATCH 095/396] documented the function --- .../functions/tuple-functions.md | 65 +++++++++++++++++-- 1 file changed, 59 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 5be75caf173..9a77238f69d 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -168,23 +168,76 @@ Result: ## tupleToNameValuePairs {#tupletonamevaluepairs} -Turns a named tuple into an array of name-value pairs. For a `Tuple(a T, b T, ..., c T)` return an `Array(Tuple(String, T), ...)` -in which the `Strings` represent the named fields of the tuple and the `T`s are the values associated to those names. +Turns a named tuple into an array of (name, value) pairs. For a `Tuple(a T, b T, ..., c T)` returns `Array(Tuple(String, T), ...)` +in which the `Strings` represents the named fields of the tuple and the `T`s are the values associated with those names. All values in the tuple should be of the same type. **Syntax** ``` sql -tupleToNameValuePairs() +tupleToNameValuePairs(tuple) ``` **Arguments** -- `x` — Named tuple. +- `tuple` — Named tuple. [Tuple](../../sql-reference/data-types/tuple.md) with any type of values. **Returned value** -- +- An array with (name, value) pairs. -Type: +Type: [Array](../../sql-reference/data-types/array.md)([Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md), ...)). **Example** + +Query: + +``` sql +CREATE TABLE tupletest (`col` Tuple(user_ID UInt64, session_ID UInt64) ENGINE = Memory; + +INSERT INTO tupletest VALUES (tuple( 100, 2502)), (tuple(1,100)); + +SELECT tupleToNameValuePairs(col) FROM tupletest; +``` + +Result: + +``` text +┌─tupleToNameValuePairs(col)────────────┐ +│ [('user_ID',100),('session_ID',2502)] │ +│ [('user_ID',1),('session_ID',100)] │ +└───────────────────────────────────────┘ +``` + +It is possible to transform colums to rows using this function: + +``` sql +CREATE TABLE tupletest (`col` Tuple(CPU Float64, Memory Float64, Disk Float64)) ENGINE = Memory; + +INSERT INTO tupletest VALUES(tuple(3.3, 5.5, 6.6)); + +SELECT arrayJoin(tupleToNameValuePairs(col))FROM tupletest; +``` + +Result: + +``` text +┌─arrayJoin(tupleToNameValuePairs(col))─┐ +│ ('CPU',3.3) │ +│ ('Memory',5.5) │ +│ ('Disk',6.6) │ +└───────────────────────────────────────┘ +``` + +If you pass a simple tuple to the function, ClickHouse uses the indexes of the values as their names: + +``` sql +SELECT tupleToNameValuePairs(tuple(3, 2, 1)); +``` + +Result: + +``` text +┌─tupleToNameValuePairs(tuple(3, 2, 1))─┐ +│ [('1',3),('2',2),('3',1)] │ +└───────────────────────────────────────┘ +``` From c8f07873f8aa4260709e43d42bd4bcd7771ccdff Mon Sep 17 00:00:00 2001 From: gyuton Date: Tue, 26 Oct 2021 00:04:52 +0300 Subject: [PATCH 096/396] translated --- .../functions/tuple-functions.md | 75 +++++++++++++++++++ 1 file changed, 75 insertions(+) diff --git a/docs/ru/sql-reference/functions/tuple-functions.md b/docs/ru/sql-reference/functions/tuple-functions.md index 381743a450b..ef0dc13c727 100644 --- a/docs/ru/sql-reference/functions/tuple-functions.md +++ b/docs/ru/sql-reference/functions/tuple-functions.md @@ -163,3 +163,78 @@ SELECT tupleHammingDistance(wordShingleMinHash(string), wordShingleMinHashCaseIn │ 2 │ └─────────────────┘ ``` + +## tupleToNameValuePairs {#tupletonamevaluepairs} + +Приводит именованный кортеж к списку пар (имя, значение). Для `Tuple(a T, b T, ..., c T)` возвращает `Array(Tuple(String, T), ...)`, где `Strings` — это названия именованных полей, а `T` — это соответствующие значения. Все значения в кортеже должны быть одинакового типа. + +**Синтаксис** + +``` sql +tupleToNameValuePairs(tuple) +``` + +**Аргументы** + +- `tuple` — именованный кортеж. [Tuple](../../sql-reference/data-types/tuple.md) с любым типом значений. + +**Возвращаемое значение** + +- Список пар (имя, значение). + +Тип: [Array](../../sql-reference/data-types/array.md)([Tuple](../../sql-reference/data-types/tuple.md)([String](../../sql-reference/data-types/string.md), ...)). + +**Пример** + +Запрос: + +``` sql +CREATE TABLE tupletest (`col` Tuple(user_ID UInt64, session_ID UInt64) ENGINE = Memory; + +INSERT INTO tupletest VALUES (tuple( 100, 2502)), (tuple(1,100)); + +SELECT tupleToNameValuePairs(col) FROM tupletest; +``` + +Результат: + +``` text +┌─tupleToNameValuePairs(col)────────────┐ +│ [('user_ID',100),('session_ID',2502)] │ +│ [('user_ID',1),('session_ID',100)] │ +└───────────────────────────────────────┘ +``` + +С помощью этой функции возможно приводить колонки к строкам: + +``` sql +CREATE TABLE tupletest (`col` Tuple(CPU Float64, Memory Float64, Disk Float64)) ENGINE = Memory; + +INSERT INTO tupletest VALUES(tuple(3.3, 5.5, 6.6)); + +SELECT arrayJoin(tupleToNameValuePairs(col))FROM tupletest; +``` + +Результат: + +``` text +┌─arrayJoin(tupleToNameValuePairs(col))─┐ +│ ('CPU',3.3) │ +│ ('Memory',5.5) │ +│ ('Disk',6.6) │ +└───────────────────────────────────────┘ +``` + +При передаче функции обычный кортеж ClickHouse использует индексы значений как их имена: + +``` sql +SELECT tupleToNameValuePairs(tuple(3, 2, 1)); +``` + +Результат: + +``` text +┌─tupleToNameValuePairs(tuple(3, 2, 1))─┐ +│ [('1',3),('2',2),('3',1)] │ +└───────────────────────────────────────┘ +``` From 1f2b08c0bd0fdb308806b5791ea032e1a19aade5 Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Tue, 26 Oct 2021 00:15:25 +0300 Subject: [PATCH 097/396] adding ngrams --- .../functions/splitting-merging-functions.md | 37 +++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/docs/ru/sql-reference/functions/splitting-merging-functions.md b/docs/ru/sql-reference/functions/splitting-merging-functions.md index efe74dba043..47c34e13363 100644 --- a/docs/ru/sql-reference/functions/splitting-merging-functions.md +++ b/docs/ru/sql-reference/functions/splitting-merging-functions.md @@ -232,3 +232,40 @@ SELECT alphaTokens('abca1abc'); │ ['abca','abc'] │ └─────────────────────────┘ ``` + +## ngrams {#ngrams} + +Выделяет из UTF-8 строки отрезки (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 81bd4a0679c42e197bcf899c0c2c725854284094 Mon Sep 17 00:00:00 2001 From: gyuton Date: Tue, 26 Oct 2021 00:15:46 +0300 Subject: [PATCH 098/396] Small fix --- docs/en/sql-reference/functions/tuple-functions.md | 2 +- docs/ru/sql-reference/functions/tuple-functions.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 9a77238f69d..cc69c5717df 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -169,7 +169,7 @@ Result: ## tupleToNameValuePairs {#tupletonamevaluepairs} Turns a named tuple into an array of (name, value) pairs. For a `Tuple(a T, b T, ..., c T)` returns `Array(Tuple(String, T), ...)` -in which the `Strings` represents the named fields of the tuple and the `T`s are the values associated with those names. All values in the tuple should be of the same type. +in which the `Strings` represents the named fields of the tuple and `T` are the values associated with those names. All values in the tuple should be of the same type. **Syntax** diff --git a/docs/ru/sql-reference/functions/tuple-functions.md b/docs/ru/sql-reference/functions/tuple-functions.md index ef0dc13c727..64a49b52ec2 100644 --- a/docs/ru/sql-reference/functions/tuple-functions.md +++ b/docs/ru/sql-reference/functions/tuple-functions.md @@ -205,7 +205,7 @@ SELECT tupleToNameValuePairs(col) FROM tupletest; └───────────────────────────────────────┘ ``` -С помощью этой функции возможно приводить колонки к строкам: +С помощью этой функции возможно приводить столбцы к строкам: ``` sql CREATE TABLE tupletest (`col` Tuple(CPU Float64, Memory Float64, Disk Float64)) ENGINE = Memory; From 0969874a748a59b7ddd43a7de3b34aa02bb44e78 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Tue, 26 Oct 2021 00:19:43 +0300 Subject: [PATCH 099/396] Document columns in views MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Задокументировал столбцы в представлениях information_schema. --- .../system-tables/information_schema.md | 160 ++++++++++++++---- 1 file changed, 130 insertions(+), 30 deletions(-) diff --git a/docs/en/operations/system-tables/information_schema.md b/docs/en/operations/system-tables/information_schema.md index eb861b5ac6e..5aa20515643 100644 --- a/docs/en/operations/system-tables/information_schema.md +++ b/docs/en/operations/system-tables/information_schema.md @@ -19,45 +19,51 @@ SHOW TABLES FROM INFORMATION_SCHEMA; `INFORMATION_SCHEMA` contains the following views: -- `SCHEMATA` — The view that can be used to get all the current database schemas. +- [COLUMNS](#columns) +- [SCHEMATA](#schemata) +- [TABLES](#tables) +- [VIEWS](#views) -``` sql -SELECT * FROM information_schema.schemata WHERE schema_name ILIKE 'information_schema' LIMIT 1 FORMAT Vertical; -``` +## COLUMNS {#columns} -``` text -Row 1: -────── -catalog_name: INFORMATION_SCHEMA -schema_name: INFORMATION_SCHEMA -schema_owner: default -default_character_set_catalog: ᴺᵁᴸᴸ -default_character_set_schema: ᴺᵁᴸᴸ -default_character_set_name: ᴺᵁᴸᴸ -sql_path: ᴺᵁᴸᴸ -``` +Allows to get a list of table columns read using this view. -- `TABLES` — The view that can be used to get all tables in the current database. +Columns: -``` sql -SELECT * FROM INFORMATION_SCHEMA.TABLES WHERE (table_schema = currentDatabase() OR table_schema = '') AND table_name NOT LIKE '%inner%' LIMIT 1 FORMAT Vertical; -``` +- `table_catalog` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the table is located. +- `table_schema` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the schema is located. +- `table_name` ([String](../../sql-reference/data-types/string.md)) — Table name. +- `column_name` ([String](../../sql-reference/data-types/string.md)) — Column name. +- `ordinal_position` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Ordinal position of a column in a table starting with 1. +- `column_default` ([String](../../sql-reference/data-types/string.md)) — Expression for the default value, or an empty string if it is not defined. +- `is_nullable` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column type is `Nullable`. +- `data_type` ([String](../../sql-reference/data-types/string.md)) — Column type. +- `character_maximum_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. +- `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. +- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. Otherwise, the `NULL` value is returned. +- `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. Otherwise, the `NULL` value is returned. +- `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. Otherwise, the `NULL` value is returned. +- `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Subtype code for `DateTime` and `ISO` interval data types. For other data types, the `NULL` value is returned. +- `character_set_catalog` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. +- `character_set_schema` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. +- `character_set_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. +- `collation_catalog` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. +- `collation_schema` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. +- `collation_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. +- `domain_catalog` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. +- `domain_schema` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. +- `domain_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. -``` text -Row 1: -────── -table_catalog: default -table_schema: default -table_name: describe_example -table_type: BASE TABLE -``` +**Example** -- `COLUMNS` — The view that can be used to get a list of table columns in the current database. +Query: ``` sql SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE (table_schema=currentDatabase() OR table_schema='') AND table_name NOT LIKE '%inner%' LIMIT 1 FORMAT Vertical; ``` +Result: + ``` text Row 1: ────── @@ -86,14 +92,108 @@ domain_schema: ᴺᵁᴸᴸ domain_name: ᴺᵁᴸᴸ ``` -- `VIEWS` — The view that can be used to get a list of all views in the current database. +## SCHEMATA {#schemata} + +Allows to get a list of schemas read using this view. + +Columns: + +- `catalog_name` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the table is located. +- `schema_name` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the schema is located. +- `schema_owner` ([String](../../sql-reference/data-types/string.md)) — Schema owner name. +- `default_character_set_catalog` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. +- `default_character_set_schema` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. +- `default_character_set_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. +- `sql_path` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. + +**Example** + +Query: + +``` sql +SELECT * FROM information_schema.schemata WHERE schema_name ILIKE 'information_schema' LIMIT 1 FORMAT Vertical; +``` + +Result: + +``` text +Row 1: +────── +catalog_name: INFORMATION_SCHEMA +schema_name: INFORMATION_SCHEMA +schema_owner: default +default_character_set_catalog: ᴺᵁᴸᴸ +default_character_set_schema: ᴺᵁᴸᴸ +default_character_set_name: ᴺᵁᴸᴸ +sql_path: ᴺᵁᴸᴸ +``` + +## TABLES {#tables} + +Allows to get a list of tables read using this view. + +Columns: + +- `table_catalog` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the table is located. +- `table_schema` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the schema is located. +- `table_name` ([String](../../sql-reference/data-types/string.md)) — Table name. +- `table_type` ([Enum8](../../sql-reference/data-types/enum.md)) — Table type. Possible values: + - `BASE TABLE` + - `VIEW` + - `FOREIGN TABLE` + - `LOCAL TEMPORARY` + - `SYSTEM VIEW` + +**Example** + +Query: + +``` sql +SELECT * FROM INFORMATION_SCHEMA.TABLES WHERE (table_schema = currentDatabase() OR table_schema = '') AND table_name NOT LIKE '%inner%' LIMIT 1 FORMAT Vertical; +``` + +Result: + +``` text +Row 1: +────── +table_catalog: default +table_schema: default +table_name: describe_example +table_type: BASE TABLE +``` + +## VIEWS {#views} + +Allows to get a list of views read using this view. + +Columns: + +- `table_catalog` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the table is located. +- `table_schema` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the schema is located. +- `table_name` ([String](../../sql-reference/data-types/string.md)) — Table name. +- `view_definition` ([String](../../sql-reference/data-types/string.md)) — `SELECT` query for view. +- `check_option` ([String](../../sql-reference/data-types/string.md)) — `NONE`, no checking. +- `is_updatable` ([Enum8](../../sql-reference/data-types/enum.md)) — `NO`, the view is not updated. +- `is_insertable_into` ([Enum8](../../sql-reference/data-types/enum.md)) — Shows whether the created view is materialized. Possible values: + - `NO` — The created view is not materialized. + - `YES` — The created view is materialized. +- `is_trigger_updatable` ([Enum8](../../sql-reference/data-types/enum.md)) — `NO`, the trigger is not updated. +- `is_trigger_deletable` ([Enum8](../../sql-reference/data-types/enum.md)) — `NO`, the trigger is not deleted. +- `is_trigger_insertable_into` ([Enum8](../../sql-reference/data-types/enum.md)) — `NO`, no data is inserted into the trigger. + +**Example** + +Query: ``` sql CREATE VIEW v (n Nullable(Int32), f Float64) AS SELECT n, f FROM t; CREATE MATERIALIZED VIEW mv ENGINE = Null AS SELECT * FROM system.one; -SELECT * FROM information_schema.views WHERE table_schema = currentDatabase(); +SELECT * FROM information_schema.views WHERE table_schema = currentDatabase() LIMIT 1 FORMAT Vertical; ``` +Result: + ``` text Row 1: ────── From b7b5661f15eaf8d706309b0541bcf9d600e15a42 Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Tue, 26 Oct 2021 00:24:38 +0300 Subject: [PATCH 100/396] translate tokens to russian --- .../functions/splitting-merging-functions.md | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/docs/ru/sql-reference/functions/splitting-merging-functions.md b/docs/ru/sql-reference/functions/splitting-merging-functions.md index 47c34e13363..df2f778d6b6 100644 --- a/docs/ru/sql-reference/functions/splitting-merging-functions.md +++ b/docs/ru/sql-reference/functions/splitting-merging-functions.md @@ -269,3 +269,29 @@ SELECT ngrams('ClickHouse', 3); │ ['Cli','lic','ick','ckH','kHo','Hou','ous','use'] │ └───────────────────────────────────────────────────┘ ``` + +## tokens {#tokens} + +Разбивает строку на токены, используя в качестве разделителей не буквенно-цифровые символы ASCII. + +**Аргументы** + +- `input_string` — набор байтов. [String](../../sql-reference/data-types/string.md). + +**Возвращаемые значения** + +Возвращает массив токенов. + +Тип: [Array](../data-types/array.md). + +**Пример** + +``` sql +SELECT tokens('test1,;\\ test2,;\\ test3,;\\ test4') AS tokens; +``` + +``` text +┌─tokens────────────────────────────┐ +│ ['test1','test2','test3','test4'] │ +└───────────────────────────────────┘ +``` From ef7a00f397e462b5308536ab25e98f78aa8c1734 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 25 Oct 2021 21:15:41 +0300 Subject: [PATCH 101/396] clickhouse-local: switch root node from to --- programs/local/LocalServer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 69f7820909a..af8c4a3fe6a 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -354,7 +354,7 @@ static ConfigurationPtr getConfigurationFromXMLString(const char * xml_data) void LocalServer::setupUsers() { static const char * minimal_default_user_xml = - "" + "" " " " " " " @@ -371,7 +371,7 @@ void LocalServer::setupUsers() " " " " " " - ""; + ""; ConfigurationPtr users_config; From e0cd7a8304a2cb2b5a487a3e26417dab4d46cbf8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 26 Oct 2021 08:50:15 +0300 Subject: [PATCH 102/396] docs: switch to --- docs/en/operations/configuration-files.md | 14 +++++++------- .../external-authenticators/kerberos.md | 16 ++++++++-------- .../operations/external-authenticators/ldap.md | 12 ++++++------ .../server-configuration-parameters/settings.md | 12 ++++++------ docs/en/operations/storing-data.md | 12 ++++++------ docs/en/operations/system-tables/index.md | 4 ++-- .../en/operations/utilities/clickhouse-copier.md | 8 ++++---- .../external-dicts-dict-layout.md | 8 ++++---- .../external-dicts-dict-sources.md | 12 ++++++------ .../external-dictionaries/external-dicts.md | 4 ++-- .../functions/ext-dict-functions.md | 8 ++++---- docs/ja/operations/configuration-files.md | 6 +++--- docs/ja/operations/system-tables.md | 4 ++-- .../ja/operations/utilities/clickhouse-copier.md | 8 ++++---- .../external-dicts-dict-layout.md | 8 ++++---- .../external-dicts-dict-sources.md | 12 ++++++------ .../external-dictionaries/external-dicts.md | 4 ++-- .../functions/ext-dict-functions.md | 4 ++-- docs/ru/operations/configuration-files.md | 14 +++++++------- .../external-authenticators/kerberos.md | 16 ++++++++-------- .../operations/external-authenticators/ldap.md | 12 ++++++------ .../server-configuration-parameters/settings.md | 12 ++++++------ docs/ru/operations/storing-data.md | 12 ++++++------ docs/ru/operations/system-tables/index.md | 4 ++-- .../ru/operations/utilities/clickhouse-copier.md | 8 ++++---- .../external-dicts-dict-layout.md | 8 ++++---- .../external-dicts-dict-sources.md | 12 ++++++------ .../external-dictionaries/external-dicts.md | 4 ++-- .../functions/ext-dict-functions.md | 8 ++++---- docs/zh/operations/configuration-files.md | 6 +++--- docs/zh/operations/system-tables/index.md | 4 ++-- docs/zh/operations/system-tables/metric_log.md | 4 ++-- .../zh/operations/utilities/clickhouse-copier.md | 8 ++++---- .../external-dicts-dict-layout.md | 8 ++++---- .../external-dicts-dict-sources.md | 12 ++++++------ .../external-dictionaries/external-dicts.md | 4 ++-- 36 files changed, 156 insertions(+), 156 deletions(-) diff --git a/docs/en/operations/configuration-files.md b/docs/en/operations/configuration-files.md index f5073afcc23..318910b5b42 100644 --- a/docs/en/operations/configuration-files.md +++ b/docs/en/operations/configuration-files.md @@ -7,7 +7,7 @@ toc_title: Configuration Files ClickHouse supports multi-file configuration management. The main server configuration file is `/etc/clickhouse-server/config.xml` or `/etc/clickhouse-server/config.yaml`. Other files must be in the `/etc/clickhouse-server/config.d` directory. Note, that any configuration file can be written either in XML or YAML, but mixing formats in one file is not supported. For example, you can have main configs as `config.xml` and `users.xml` and write additional files in `config.d` and `users.d` directories in `.yaml`. -All XML files should have the same root element, usually ``. As for YAML, `yandex:` should not be present, the parser will insert it automatically. +All XML files should have the same root element, usually ``. As for YAML, `yandex:` should not be present, the parser will insert it automatically. ## Override {#override} @@ -21,13 +21,13 @@ Some settings specified in the main configuration file can be overridden in othe You can also declare attributes as coming from environment variables by using `from_env="VARIABLE_NAME"`: ```xml - + - + ``` ## Substitution {#substitution} @@ -39,7 +39,7 @@ If you want to replace an entire element with a substitution use `include` as el XML substitution example: ```xml - + @@ -48,7 +48,7 @@ XML substitution example: - + ``` Substitutions can also be performed from ZooKeeper. To do this, specify the attribute `from_zk = "/path/to/node"`. The element value is replaced with the contents of the node at `/path/to/node` in ZooKeeper. You can also put an entire XML subtree on the ZooKeeper node and it will be fully inserted into the source element. @@ -72,7 +72,7 @@ $ cat /etc/clickhouse-server/users.d/alice.xml ``` ``` xml - + analytics @@ -83,7 +83,7 @@ $ cat /etc/clickhouse-server/users.d/alice.xml analytics - + ``` ## YAML examples {#example} diff --git a/docs/en/operations/external-authenticators/kerberos.md b/docs/en/operations/external-authenticators/kerberos.md index 5fe0b2bfc37..2e2a88dc7a8 100644 --- a/docs/en/operations/external-authenticators/kerberos.md +++ b/docs/en/operations/external-authenticators/kerberos.md @@ -23,32 +23,32 @@ To enable Kerberos, one should include `kerberos` section in `config.xml`. This Example (goes into `config.xml`): ```xml - + - + ``` With principal specification: ```xml - + HTTP/clickhouse.example.com@EXAMPLE.COM - + ``` With filtering by realm: ```xml - + EXAMPLE.COM - + ``` !!! warning "Note" @@ -80,7 +80,7 @@ Parameters: Example (goes into `users.xml`): ```xml - + @@ -91,7 +91,7 @@ Example (goes into `users.xml`): - + ``` !!! warning "Warning" diff --git a/docs/en/operations/external-authenticators/ldap.md b/docs/en/operations/external-authenticators/ldap.md index c33700f0e31..57e6ec1a087 100644 --- a/docs/en/operations/external-authenticators/ldap.md +++ b/docs/en/operations/external-authenticators/ldap.md @@ -14,7 +14,7 @@ To define LDAP server you must add `ldap_servers` section to the `config.xml`. **Example** ```xml - + @@ -45,7 +45,7 @@ To define LDAP server you must add `ldap_servers` section to the `config.xml`. no - + ``` Note, that you can define multiple LDAP servers inside the `ldap_servers` section using distinct names. @@ -90,7 +90,7 @@ At each login attempt, ClickHouse tries to "bind" to the specified DN defined by **Example** ```xml - + @@ -101,7 +101,7 @@ At each login attempt, ClickHouse tries to "bind" to the specified DN defined by - + ``` Note, that user `my_user` refers to `my_ldap_server`. This LDAP server must be configured in the main `config.xml` file as described previously. @@ -125,7 +125,7 @@ At each login attempt, ClickHouse tries to find the user definition locally and Goes into `config.xml`. ```xml - + @@ -156,7 +156,7 @@ Goes into `config.xml`. - + ``` Note that `my_ldap_server` referred in the `ldap` section inside the `user_directories` section must be a previously defined LDAP server that is configured in the `config.xml` (see [LDAP Server Definition](#ldap-server-definition)). diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index cdf49678570..d6e4008b2a5 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -786,14 +786,14 @@ It is enabled by default. If it`s not, you can do this manually. To manually turn on metrics history collection [`system.metric_log`](../../operations/system-tables/metric_log.md), create `/etc/clickhouse-server/config.d/metric_log.xml` with the following content: ``` xml - + system metric_log
7500 1000
-
+ ``` **Disabling** @@ -801,9 +801,9 @@ To manually turn on metrics history collection [`system.metric_log`](../../opera To disable `metric_log` setting, you should create the following file `/etc/clickhouse-server/config.d/disable_metric_log.xml` with the following content: ``` xml - + - + ``` ## replicated_merge_tree {#server_configuration_parameters-replicated_merge_tree} @@ -1039,7 +1039,7 @@ Parameters: **Example** ```xml - + notice system @@ -1048,7 +1048,7 @@ Parameters: Engine = MergeTree PARTITION BY event_date ORDER BY event_time TTL event_date + INTERVAL 30 day - + ``` diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index bc764a909b8..beffd45bcbd 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -22,7 +22,7 @@ ClickHouse supports zero-copy replication for `S3` and `HDFS` disks, which means Configuration markup: ``` xml - + @@ -44,7 +44,7 @@ Configuration markup: 0 - + ``` Required parameters: @@ -96,7 +96,7 @@ Optional parameters: Example of disk configuration: ``` xml - + @@ -113,7 +113,7 @@ Example of disk configuration: - + ``` ## Storing Data on Web Server {#storing-data-on-webserver} @@ -127,7 +127,7 @@ Web server storage is supported only for the [MergeTree](../engines/table-engine A ready test case. You need to add this configuration to config: ``` xml - + @@ -145,7 +145,7 @@ A ready test case. You need to add this configuration to config: - + ``` And then execute this query: diff --git a/docs/en/operations/system-tables/index.md b/docs/en/operations/system-tables/index.md index 0cc5e834af1..5e8418d0af3 100644 --- a/docs/en/operations/system-tables/index.md +++ b/docs/en/operations/system-tables/index.md @@ -34,7 +34,7 @@ System log tables can be customized by creating a config file with the same name An example: ```xml - + system query_log
@@ -45,7 +45,7 @@ An example: --> 7500
-
+ ``` By default, table growth is unlimited. To control a size of a table, you can use [TTL](../../sql-reference/statements/alter/ttl.md#manipulations-with-table-ttl) settings for removing outdated log records. Also you can use the partitioning feature of `MergeTree`-engine tables. diff --git a/docs/en/operations/utilities/clickhouse-copier.md b/docs/en/operations/utilities/clickhouse-copier.md index 3e729a3a3dd..de3443fb845 100644 --- a/docs/en/operations/utilities/clickhouse-copier.md +++ b/docs/en/operations/utilities/clickhouse-copier.md @@ -47,7 +47,7 @@ Parameters: ## Format of Zookeeper.xml {#format-of-zookeeper-xml} ``` xml - + trace 100M @@ -60,13 +60,13 @@ Parameters: 2181 - + ``` ## Configuration of Copying Tasks {#configuration-of-copying-tasks} ``` xml - + @@ -179,7 +179,7 @@ Parameters: ... - + ``` `clickhouse-copier` tracks the changes in `/task/path/description` and applies them on the fly. For instance, if you change the value of `max_workers`, the number of processes running tasks will also change. 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 40f9db12315..effcc614930 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 @@ -26,7 +26,7 @@ You can view the list of external dictionaries and their statuses in the `system The configuration looks like this: ``` xml - + ... @@ -36,7 +36,7 @@ The configuration looks like this: ... - + ``` Corresponding [DDL-query](../../../sql-reference/statements/create/dictionary.md): @@ -289,7 +289,7 @@ Details of the algorithm: Configuration example: ``` xml - + ... @@ -317,7 +317,7 @@ Configuration example: - + ``` or diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index a1d787a37ea..e9746e59d1d 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -10,7 +10,7 @@ An external dictionary can be connected from many different sources. If dictionary is configured using xml-file, the configuration looks like this: ``` xml - + ... @@ -21,7 +21,7 @@ If dictionary is configured using xml-file, the configuration looks like this: ... ... - + ``` In case of [DDL-query](../../../sql-reference/statements/create/dictionary.md), equal configuration will looks like: @@ -311,7 +311,7 @@ Configuring `/etc/odbc.ini` (or `~/.odbc.ini` if you signed in under a user that The dictionary configuration in ClickHouse: ``` xml - + table_name @@ -340,7 +340,7 @@ The dictionary configuration in ClickHouse: - + ``` or @@ -416,7 +416,7 @@ Remarks: Configuring the dictionary in ClickHouse: ``` xml - + test @@ -446,7 +446,7 @@ Configuring the dictionary in ClickHouse: - + ``` or diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md index d229336c58d..00025c70c60 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -26,7 +26,7 @@ The [dictionaries](../../../operations/system-tables/dictionaries.md#system_tabl The dictionary configuration file has the following format: ``` xml - + An optional element with any content. Ignored by the ClickHouse server. @@ -38,7 +38,7 @@ The dictionary configuration file has the following format: - + ``` You can [configure](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md) any number of dictionaries in the same file. diff --git a/docs/en/sql-reference/functions/ext-dict-functions.md b/docs/en/sql-reference/functions/ext-dict-functions.md index 54b72e77f01..0e8352d2d1e 100644 --- a/docs/en/sql-reference/functions/ext-dict-functions.md +++ b/docs/en/sql-reference/functions/ext-dict-functions.md @@ -53,7 +53,7 @@ The first column is `id`, the second column is `c1`. Configure the external dictionary: ``` xml - + ext-dict-test @@ -77,7 +77,7 @@ Configure the external dictionary: 0 - + ``` Perform the query: @@ -113,7 +113,7 @@ The first column is `id`, the second is `c1`, the third is `c2`. Configure the external dictionary: ``` xml - + ext-dict-mult @@ -142,7 +142,7 @@ Configure the external dictionary: 0 - + ``` Perform the query: diff --git a/docs/ja/operations/configuration-files.md b/docs/ja/operations/configuration-files.md index f170ceab907..e22fc96d3b5 100644 --- a/docs/ja/operations/configuration-files.md +++ b/docs/ja/operations/configuration-files.md @@ -10,7 +10,7 @@ toc_title: "\u8A2D\u5B9A\u30D5\u30A1\u30A4\u30EB" ClickHouseは複数のファイル構成管理をサポートします。 主サーバ設定ファイルで指定することがで `/etc/clickhouse-server/config.xml`. その他のファイルは `/etc/clickhouse-server/config.d` ディレクトリ。 !!! note "注" - すべての構成ファイルはXML形式である必要があります。 また、通常は同じルート要素を持つ必要があります ``. + すべての構成ファイルはXML形式である必要があります。 また、通常は同じルート要素を持つ必要があります ``. メイン構成ファイルで指定された一部の設定は、他の構成ファイルで上書きできます。 その `replace` または `remove` これらの構成ファイルの要素に属性を指定できます。 @@ -36,7 +36,7 @@ $ cat /etc/clickhouse-server/users.d/alice.xml ``` ``` xml - + analytics @@ -47,7 +47,7 @@ $ cat /etc/clickhouse-server/users.d/alice.xml analytics - + ``` 各設定ファイルでは、サーバともある `file-preprocessed.xml` 起動時のファイル。 これらのファイルには、完了したすべての置換と上書きが含まれており、情報提供を目的としています。 設定ファイルでZooKeeperの置換が使用されていても、サーバーの起動時にZooKeeperが使用できない場合、サーバーは前処理されたファイルから設定をロードします。 diff --git a/docs/ja/operations/system-tables.md b/docs/ja/operations/system-tables.md index 4a18028c084..cdeeed7b584 100644 --- a/docs/ja/operations/system-tables.md +++ b/docs/ja/operations/system-tables.md @@ -335,14 +335,14 @@ SELECT * FROM system.metrics LIMIT 10 メトリック履歴の収集を有効にするには `system.metric_log`,作成 `/etc/clickhouse-server/config.d/metric_log.xml` 次の内容を使って: ``` xml - + system metric_log
7500 1000
-
+ ``` **例** diff --git a/docs/ja/operations/utilities/clickhouse-copier.md b/docs/ja/operations/utilities/clickhouse-copier.md index 4d825f02b76..614984af42b 100644 --- a/docs/ja/operations/utilities/clickhouse-copier.md +++ b/docs/ja/operations/utilities/clickhouse-copier.md @@ -46,7 +46,7 @@ $ clickhouse-copier copier --daemon --config zookeeper.xml --task-path /task/pat ## 飼育係の形式。xml {#format-of-zookeeper-xml} ``` xml - + trace 100M @@ -59,13 +59,13 @@ $ clickhouse-copier copier --daemon --config zookeeper.xml --task-path /task/pat 2181 - + ``` ## コピータスクの構成 {#configuration-of-copying-tasks} ``` xml - + @@ -168,7 +168,7 @@ $ clickhouse-copier copier --daemon --config zookeeper.xml --task-path /task/pat ... - + ``` `clickhouse-copier` の変更を追跡します `/task/path/description` そしてその場でそれらを適用します。 たとえば、次の値を変更すると `max_workers`、タスクを実行しているプロセスの数も変更されます。 diff --git a/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 3286cf04113..4aef036b3bd 100644 --- a/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -28,7 +28,7 @@ ClickHouseは、辞書のエラーに対して例外を生成します。 エラ 設定は次のようになります: ``` xml - + ... @@ -38,7 +38,7 @@ ClickHouseは、辞書のエラーに対して例外を生成します。 エラ ... - + ``` 対応する [DDL-クエリ](../../statements/create.md#create-dictionary-query): @@ -208,7 +208,7 @@ dictGetT('dict_name', 'attr_name', id, date) 設定例: ``` xml - + ... @@ -237,7 +237,7 @@ dictGetT('dict_name', 'attr_name', id, date) - + ``` または diff --git a/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index 5aefd7050e3..e61218b8d1d 100644 --- a/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -12,7 +12,7 @@ toc_title: "\u5916\u90E8\u8F9E\u66F8\u306E\u30BD\u30FC\u30B9" 辞書がxml-fileを使用して構成されている場合、構成は次のようになります: ``` xml - + ... @@ -23,7 +23,7 @@ toc_title: "\u5916\u90E8\u8F9E\u66F8\u306E\u30BD\u30FC\u30B9" ... ... - + ``` の場合 [DDL-クエリ](../../statements/create.md#create-dictionary-query)、等しい構成は次のようになります: @@ -272,7 +272,7 @@ $ sudo apt-get install -y unixodbc odbcinst odbc-postgresql ClickHouseの辞書構成: ``` xml - + table_name @@ -301,7 +301,7 @@ ClickHouseの辞書構成: - + ``` または @@ -367,7 +367,7 @@ $ sudo apt-get install tdsodbc freetds-bin sqsh ClickHouseでの辞書の構成: ``` xml - + test @@ -397,7 +397,7 @@ ClickHouseでの辞書の構成: - + ``` または diff --git a/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts.md index 4f90ac18267..725b77af7af 100644 --- a/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/ja/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -28,7 +28,7 @@ toc_title: "\u4E00\u822C\u7684\u306A\u8AAC\u660E" 辞書構成ファイルの形式は次のとおりです: ``` xml - + An optional element with any content. Ignored by the ClickHouse server. @@ -40,7 +40,7 @@ toc_title: "\u4E00\u822C\u7684\u306A\u8AAC\u660E" - + ``` あなたはできる [設定](external-dicts-dict.md) 同じファイル内の任意の数の辞書。 diff --git a/docs/ja/sql-reference/functions/ext-dict-functions.md b/docs/ja/sql-reference/functions/ext-dict-functions.md index 6a90d99a5f0..0d21322eda3 100644 --- a/docs/ja/sql-reference/functions/ext-dict-functions.md +++ b/docs/ja/sql-reference/functions/ext-dict-functions.md @@ -50,7 +50,7 @@ ClickHouseは、属性の値を解析できない場合、または値が属性 外部辞書の構成: ``` xml - + ext-dict-test @@ -74,7 +74,7 @@ ClickHouseは、属性の値を解析できない場合、または値が属性 0 - + ``` クエリの実行: diff --git a/docs/ru/operations/configuration-files.md b/docs/ru/operations/configuration-files.md index 343fffdfaa5..5abb1ece455 100644 --- a/docs/ru/operations/configuration-files.md +++ b/docs/ru/operations/configuration-files.md @@ -8,7 +8,7 @@ toc_title: "Конфигурационные файлы" ClickHouse поддерживает многофайловое управление конфигурацией. Основной конфигурационный файл сервера — `/etc/clickhouse-server/config.xml` или `/etc/clickhouse-server/config.yaml`. Остальные файлы должны находиться в директории `/etc/clickhouse-server/config.d`. Обратите внимание, что конфигурационные файлы могут быть записаны в форматах XML или YAML, но смешение этих форматов в одном файле не поддерживается. Например, можно хранить основные конфигурационные файлы как `config.xml` и `users.xml`, а дополнительные файлы записать в директории `config.d` и `users.d` в формате `.yaml`. -Все XML файлы должны иметь одинаковый корневой элемент, обычно ``. Для YAML элемент `yandex:` должен отсутствовать, так как парсер вставляет его автоматически. +Все XML файлы должны иметь одинаковый корневой элемент, обычно ``. Для YAML элемент `yandex:` должен отсутствовать, так как парсер вставляет его автоматически. ## Переопределение {#override} @@ -22,13 +22,13 @@ ClickHouse поддерживает многофайловое управлен Также возможно указать атрибуты как переменные среды с помощью `from_env="VARIABLE_NAME"`: ```xml - + - + ``` ## Подстановки {#substitution} @@ -40,7 +40,7 @@ ClickHouse поддерживает многофайловое управлен Пример подстановки XML: ```xml - + @@ -49,7 +49,7 @@ ClickHouse поддерживает многофайловое управлен - + ``` Подстановки могут также выполняться из ZooKeeper. Для этого укажите у элемента атрибут `from_zk = "/path/to/node"`. Значение элемента заменится на содержимое узла `/path/to/node` в ZooKeeper. В ZooKeeper-узел также можно положить целое XML-поддерево, оно будет целиком вставлено в исходный элемент. @@ -66,7 +66,7 @@ $ cat /etc/clickhouse-server/users.d/alice.xml ``` ``` xml - + analytics @@ -77,7 +77,7 @@ $ cat /etc/clickhouse-server/users.d/alice.xml analytics - + ``` Для каждого конфигурационного файла, сервер при запуске генерирует также файлы `file-preprocessed.xml`. Эти файлы содержат все выполненные подстановки и переопределения, и предназначены для информационных целей. Если в конфигурационных файлах были использованы ZooKeeper-подстановки, но при старте сервера ZooKeeper недоступен, то сервер загрузит конфигурацию из preprocessed-файла. diff --git a/docs/ru/operations/external-authenticators/kerberos.md b/docs/ru/operations/external-authenticators/kerberos.md index 2d31e355bba..eeff1cb6011 100644 --- a/docs/ru/operations/external-authenticators/kerberos.md +++ b/docs/ru/operations/external-authenticators/kerberos.md @@ -24,32 +24,32 @@ ClickHouse предоставляет возможность аутентифи Примеры, как должен выглядеть файл `config.xml`: ```xml - + - + ``` Или, с указанием принципала: ```xml - + HTTP/clickhouse.example.com@EXAMPLE.COM - + ``` Или, с фильтрацией по реалм: ```xml - + EXAMPLE.COM - + ``` !!! Warning "Важно" @@ -81,7 +81,7 @@ ClickHouse предоставляет возможность аутентифи Пример, как выглядит конфигурация Kerberos в `users.xml`: ```xml - + @@ -92,7 +92,7 @@ ClickHouse предоставляет возможность аутентифи - + ``` diff --git a/docs/ru/operations/external-authenticators/ldap.md b/docs/ru/operations/external-authenticators/ldap.md index 7b4bab0290c..3a8f2c05ae8 100644 --- a/docs/ru/operations/external-authenticators/ldap.md +++ b/docs/ru/operations/external-authenticators/ldap.md @@ -14,7 +14,7 @@ **Пример** ```xml - + @@ -45,7 +45,7 @@ no - + ``` Обратите внимание, что можно определить несколько LDAP серверов внутри секции `ldap_servers`, используя различные имена. @@ -90,7 +90,7 @@ **Пример** ```xml - + @@ -101,7 +101,7 @@ - + ``` Обратите внимание, что пользователь `my_user` ссылается на `my_ldap_server`. Этот LDAP сервер должен быть настроен в основном файле `config.xml`, как это было описано ранее. @@ -125,7 +125,7 @@ CREATE USER my_user IDENTIFIED WITH ldap SERVER 'my_ldap_server'; В `config.xml`. ```xml - + @@ -156,7 +156,7 @@ CREATE USER my_user IDENTIFIED WITH ldap SERVER 'my_ldap_server'; - + ``` Обратите внимание, что `my_ldap_server`, указанный в секции `ldap` внутри секции `user_directories`, должен быть настроен в файле `config.xml`, как это было описано ранее. (см. [Определение LDAP сервера](#ldap-server-definition)). diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 2d4ca1d3dcd..0b62ecf6680 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -754,14 +754,14 @@ ClickHouse проверяет условия для `min_part_size` и `min_part Чтобы вручную включить сбор истории метрик в таблице [`system.metric_log`](../../operations/system-tables/metric_log.md), создайте `/etc/clickhouse-server/config.d/metric_log.xml` следующего содержания: ``` xml - + system metric_log
7500 1000
-
+ ``` **Выключение** @@ -769,9 +769,9 @@ ClickHouse проверяет условия для `min_part_size` и `min_part Чтобы отключить настройку `metric_log` , создайте файл `/etc/clickhouse-server/config.d/disable_metric_log.xml` следующего содержания: ``` xml - + - + ``` ## replicated\_merge\_tree {#server_configuration_parameters-replicated_merge_tree} @@ -1007,7 +1007,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part **Пример** ```xml - + notice system @@ -1016,7 +1016,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part Engine = MergeTree PARTITION BY event_date ORDER BY event_time TTL event_date + INTERVAL 30 day - + ``` diff --git a/docs/ru/operations/storing-data.md b/docs/ru/operations/storing-data.md index ca9b60bded8..a544c7202e1 100644 --- a/docs/ru/operations/storing-data.md +++ b/docs/ru/operations/storing-data.md @@ -19,7 +19,7 @@ toc_title: "Хранение данных на внешних дисках" Пример конфигурации: ``` xml - + @@ -41,7 +41,7 @@ toc_title: "Хранение данных на внешних дисках" 0 - + ``` Обязательные параметры: @@ -93,7 +93,7 @@ toc_title: "Хранение данных на внешних дисках" Пример конфигурации: ``` xml - + @@ -110,7 +110,7 @@ toc_title: "Хранение данных на внешних дисках" - + ``` ## Хранение данных на веб-сервере {#storing-data-on-webserver} @@ -124,7 +124,7 @@ toc_title: "Хранение данных на внешних дисках" Готовый тестовый пример. Добавьте эту конфигурацию в config: ``` xml - + @@ -142,7 +142,7 @@ toc_title: "Хранение данных на внешних дисках" - + ``` А затем выполните этот запрос: diff --git a/docs/ru/operations/system-tables/index.md b/docs/ru/operations/system-tables/index.md index 73b839ddc1f..c560c1b7f49 100644 --- a/docs/ru/operations/system-tables/index.md +++ b/docs/ru/operations/system-tables/index.md @@ -34,7 +34,7 @@ toc_title: "Системные таблицы" Пример: ```xml - + system query_log
@@ -45,7 +45,7 @@ toc_title: "Системные таблицы" --> 7500
-
+ ``` По умолчанию размер таблицы не ограничен. Управлять размером таблицы можно используя [TTL](../../sql-reference/statements/alter/ttl.md#manipuliatsii-s-ttl-tablitsy) для удаления устаревших записей журнала. Также вы можете использовать функцию партиционирования для таблиц `MergeTree`. diff --git a/docs/ru/operations/utilities/clickhouse-copier.md b/docs/ru/operations/utilities/clickhouse-copier.md index 7e1364f9ee1..07467c3e5da 100644 --- a/docs/ru/operations/utilities/clickhouse-copier.md +++ b/docs/ru/operations/utilities/clickhouse-copier.md @@ -44,7 +44,7 @@ $ clickhouse-copier --daemon --config zookeeper.xml --task-path /task/path --bas ## Формат Zookeeper.xml {#format-zookeeper-xml} ``` xml - + trace 100M @@ -57,13 +57,13 @@ $ clickhouse-copier --daemon --config zookeeper.xml --task-path /task/path --bas 2181 - + ``` ## Конфигурация заданий на копирование {#konfiguratsiia-zadanii-na-kopirovanie} ``` xml - + @@ -176,7 +176,7 @@ $ clickhouse-copier --daemon --config zookeeper.xml --task-path /task/path --bas ... - + ``` `clickhouse-copier` отслеживает изменения `/task/path/description` и применяет их «на лету». Если вы поменяете, например, значение `max_workers`, то количество процессов, выполняющих задания, также изменится. 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 647e2c5f5a7..edc8b9bbc11 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 @@ -26,7 +26,7 @@ toc_title: "Хранение словарей в памяти" Общий вид конфигурации: ``` xml - + ... @@ -36,7 +36,7 @@ toc_title: "Хранение словарей в памяти" ... - + ``` Соответствущий [DDL-запрос](../../statements/create/dictionary.md#create-dictionary-query): @@ -284,7 +284,7 @@ RANGE(MIN first MAX last) Пример конфигурации: ``` xml - + ... @@ -313,7 +313,7 @@ RANGE(MIN first MAX last) - + ``` или diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index 828d69cf26b..0486f4931b0 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -10,7 +10,7 @@ toc_title: "Источники внешних словарей" Общий вид XML-конфигурации: ``` xml - + ... @@ -21,7 +21,7 @@ toc_title: "Источники внешних словарей" ... ... - + ``` Аналогичный [DDL-запрос](../../statements/create/dictionary.md#create-dictionary-query): @@ -311,7 +311,7 @@ $ sudo apt-get install -y unixodbc odbcinst odbc-postgresql Конфигурация словаря в ClickHouse: ``` xml - + table_name @@ -340,7 +340,7 @@ $ sudo apt-get install -y unixodbc odbcinst odbc-postgresql - + ``` или @@ -416,7 +416,7 @@ $ sudo apt-get install tdsodbc freetds-bin sqsh Настройка словаря в ClickHouse: ``` xml - + test @@ -446,7 +446,7 @@ $ sudo apt-get install tdsodbc freetds-bin sqsh - + ``` или diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts.md index 04ef24b68c5..55324071efe 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -26,7 +26,7 @@ ClickHouse: Конфигурационный файл словарей имеет вид: ``` xml - + Необязательный элемент с любым содержимым. Игнорируется сервером ClickHouse. @@ -42,7 +42,7 @@ ClickHouse: - + ``` В одном файле можно [сконфигурировать](external-dicts-dict.md) произвольное количество словарей. diff --git a/docs/ru/sql-reference/functions/ext-dict-functions.md b/docs/ru/sql-reference/functions/ext-dict-functions.md index 0e234f1d84e..ae62d98dd63 100644 --- a/docs/ru/sql-reference/functions/ext-dict-functions.md +++ b/docs/ru/sql-reference/functions/ext-dict-functions.md @@ -53,7 +53,7 @@ dictGetOrNull('dict_name', attr_name, id_expr) Настройка внешнего словаря: ``` xml - + ext-dict-test @@ -77,7 +77,7 @@ dictGetOrNull('dict_name', attr_name, id_expr) 0 - + ``` Выполним запрос: @@ -113,7 +113,7 @@ LIMIT 3; Настройка внешнего словаря: ``` xml - + ext-dict-mult @@ -142,7 +142,7 @@ LIMIT 3; 0 - + ``` Выполним запрос: diff --git a/docs/zh/operations/configuration-files.md b/docs/zh/operations/configuration-files.md index da76a3588e3..7998baafb6c 100644 --- a/docs/zh/operations/configuration-files.md +++ b/docs/zh/operations/configuration-files.md @@ -3,7 +3,7 @@ ClickHouse支持多配置文件管理。主配置文件是`/etc/clickhouse-server/config.xml`。其余文件须在目录`/etc/clickhouse-server/config.d`。 !!! 注意: - 所有配置文件必须是XML格式。此外,配置文件须有相同的跟元素,通常是``。 + 所有配置文件必须是XML格式。此外,配置文件须有相同的跟元素,通常是``。 主配置文件中的一些配置可以通过`replace`或`remove`属性被配置文件覆盖。 @@ -26,7 +26,7 @@ $ cat /etc/clickhouse-server/users.d/alice.xml ``` ``` xml - + analytics @@ -37,7 +37,7 @@ $ cat /etc/clickhouse-server/users.d/alice.xml analytics - + ``` 对于每个配置文件,服务器还会在启动时生成 `file-preprocessed.xml` 文件。这些文件包含所有已完成的替换和复盖,并且它们旨在提供信息。如果zookeeper替换在配置文件中使用,但ZooKeeper在服务器启动时不可用,则服务器将从预处理的文件中加载配置。 diff --git a/docs/zh/operations/system-tables/index.md b/docs/zh/operations/system-tables/index.md index 21e0bccc693..eb7d45f9388 100644 --- a/docs/zh/operations/system-tables/index.md +++ b/docs/zh/operations/system-tables/index.md @@ -36,7 +36,7 @@ toc_title: "\u7CFB\u7EDF\u8868" 配置定义的示例如下: ``` - + system query_log
@@ -47,7 +47,7 @@ toc_title: "\u7CFB\u7EDF\u8868" --> 7500
-
+ ``` 默认情况下,表增长是无限的。可以通过TTL 删除过期日志记录的设置来控制表的大小。 你也可以使用分区功能 `MergeTree`-引擎表。 diff --git a/docs/zh/operations/system-tables/metric_log.md b/docs/zh/operations/system-tables/metric_log.md index aaf04d74b0d..5eb038feea7 100644 --- a/docs/zh/operations/system-tables/metric_log.md +++ b/docs/zh/operations/system-tables/metric_log.md @@ -9,14 +9,14 @@ machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 打开指标历史记录收集 `system.metric_log`,创建 `/etc/clickhouse-server/config.d/metric_log.xml` 具有以下内容: ``` xml - + system metric_log
7500 1000
-
+ ``` **示例** diff --git a/docs/zh/operations/utilities/clickhouse-copier.md b/docs/zh/operations/utilities/clickhouse-copier.md index b68390fce04..537006ecf0d 100644 --- a/docs/zh/operations/utilities/clickhouse-copier.md +++ b/docs/zh/operations/utilities/clickhouse-copier.md @@ -41,7 +41,7 @@ clickhouse-copier --daemon --config zookeeper.xml --task-path /task/path --base- ## Zookeeper.xml格式 {#format-of-zookeeper-xml} ``` xml - + trace 100M @@ -54,13 +54,13 @@ clickhouse-copier --daemon --config zookeeper.xml --task-path /task/path --base- 2181 - + ``` ## 复制任务的配置 {#configuration-of-copying-tasks} ``` xml - + @@ -163,7 +163,7 @@ clickhouse-copier --daemon --config zookeeper.xml --task-path /task/path --base- ... - + ``` `clickhouse-copier` 跟踪更改 `/task/path/description` 并在飞行中应用它们。 例如,如果你改变的值 `max_workers`,运行任务的进程数也会发生变化。 diff --git a/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 78d4c1ea236..083ed0318ba 100644 --- a/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -28,7 +28,7 @@ ClickHouse为字典中的错误生成异常。 错误示例: 配置如下所示: ``` xml - + ... @@ -38,7 +38,7 @@ ClickHouse为字典中的错误生成异常。 错误示例: ... - + ``` 相应的 [DDL-查询](../../statements/create.md#create-dictionary-query): @@ -208,7 +208,7 @@ dictGetT('dict_name', 'attr_name', id, date) 配置示例: ``` xml - + ... @@ -237,7 +237,7 @@ dictGetT('dict_name', 'attr_name', id, date) - + ``` 或 diff --git a/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index 399224cb8c4..c0a31535b08 100644 --- a/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -12,7 +12,7 @@ toc_title: "\u5916\u90E8\u5B57\u5178\u7684\u6765\u6E90" 如果使用xml-file配置字典,则配置如下所示: ``` xml - + ... @@ -23,7 +23,7 @@ toc_title: "\u5916\u90E8\u5B57\u5178\u7684\u6765\u6E90" ... ... - + ``` 在情况下 [DDL-查询](../../statements/create.md#create-dictionary-query),相等的配置将看起来像: @@ -272,7 +272,7 @@ $ sudo apt-get install -y unixodbc odbcinst odbc-postgresql ClickHouse中的字典配置: ``` xml - + table_name @@ -301,7 +301,7 @@ ClickHouse中的字典配置: - + ``` 或 @@ -367,7 +367,7 @@ $ sudo apt-get install tdsodbc freetds-bin sqsh 在ClickHouse中配置字典: ``` xml - + test @@ -397,7 +397,7 @@ $ sudo apt-get install tdsodbc freetds-bin sqsh - + ``` 或 diff --git a/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts.md index 609d1c11f56..036ba3d707f 100644 --- a/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/zh/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -28,7 +28,7 @@ ClickHouse: 字典配置文件具有以下格式: ``` xml - + An optional element with any content. Ignored by the ClickHouse server. @@ -40,7 +40,7 @@ ClickHouse: - + ``` 你可以 [配置](external-dicts-dict.md) 同一文件中的任意数量的字典。 From 573994a4ed15060e729211ab48439436372bc316 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 26 Oct 2021 08:51:22 +0300 Subject: [PATCH 103/396] clickhouse-install: switch to --- programs/install/Install.cpp | 28 ++++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 606af7ecd0d..f423ae4ac3e 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -495,12 +495,12 @@ int mainEntryClickHouseInstall(int argc, char ** argv) { std::string data_file = config_d / "data-paths.xml"; WriteBufferFromFile out(data_file); - out << "\n" + out << "\n" " " << data_path.string() << "\n" " " << (data_path / "tmp").string() << "\n" " " << (data_path / "user_files").string() << "\n" " " << (data_path / "format_schemas").string() << "\n" - "\n"; + "\n"; out.sync(); out.finalize(); fmt::print("Data path configuration override is saved to file {}.\n", data_file); @@ -510,12 +510,12 @@ int mainEntryClickHouseInstall(int argc, char ** argv) { std::string logger_file = config_d / "logger.xml"; WriteBufferFromFile out(logger_file); - out << "\n" + out << "\n" " \n" " " << (log_path / "clickhouse-server.log").string() << "\n" " " << (log_path / "clickhouse-server.err.log").string() << "\n" " \n" - "\n"; + "\n"; out.sync(); out.finalize(); fmt::print("Log path configuration override is saved to file {}.\n", logger_file); @@ -525,13 +525,13 @@ int mainEntryClickHouseInstall(int argc, char ** argv) { std::string user_directories_file = config_d / "user-directories.xml"; WriteBufferFromFile out(user_directories_file); - out << "\n" + out << "\n" " \n" " \n" " " << (data_path / "access").string() << "\n" " \n" " \n" - "\n"; + "\n"; out.sync(); out.finalize(); fmt::print("User directory path configuration override is saved to file {}.\n", user_directories_file); @@ -541,7 +541,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) { std::string openssl_file = config_d / "openssl.xml"; WriteBufferFromFile out(openssl_file); - out << "\n" + out << "\n" " \n" " \n" " " << (config_dir / "server.crt").string() << "\n" @@ -549,7 +549,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) " " << (config_dir / "dhparam.pem").string() << "\n" " \n" " \n" - "\n"; + "\n"; out.sync(); out.finalize(); fmt::print("OpenSSL path configuration override is saved to file {}.\n", openssl_file); @@ -716,25 +716,25 @@ int mainEntryClickHouseInstall(int argc, char ** argv) hash_hex.resize(64); for (size_t i = 0; i < 32; ++i) writeHexByteLowercase(hash[i], &hash_hex[2 * i]); - out << "\n" + out << "\n" " \n" " \n" " \n" " " << hash_hex << "\n" " \n" " \n" - "\n"; + "\n"; out.sync(); out.finalize(); fmt::print(HILITE "Password for default user is saved in file {}." END_HILITE "\n", password_file); #else - out << "\n" + out << "\n" " \n" " \n" " \n" " \n" " \n" - "\n"; + "\n"; out.sync(); out.finalize(); fmt::print(HILITE "Password for default user is saved in plaintext in file {}." END_HILITE "\n", password_file); @@ -777,9 +777,9 @@ int mainEntryClickHouseInstall(int argc, char ** argv) { std::string listen_file = config_d / "listen.xml"; WriteBufferFromFile out(listen_file); - out << "\n" + out << "\n" " ::\n" - "\n"; + "\n"; out.sync(); out.finalize(); fmt::print("The choice is saved in file {}.\n", listen_file); From e60c1f1061266ca5ecc98ee5d34578c092bcdc1b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 26 Oct 2021 08:51:37 +0300 Subject: [PATCH 104/396] programs: switch left configs from to --- programs/server/config.d/listen.xml.disabled | 4 ++-- programs/server/config.d/tls.xml.disabled | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/programs/server/config.d/listen.xml.disabled b/programs/server/config.d/listen.xml.disabled index de8c737ff75..f94e5c88568 100644 --- a/programs/server/config.d/listen.xml.disabled +++ b/programs/server/config.d/listen.xml.disabled @@ -1,3 +1,3 @@ - + :: - + diff --git a/programs/server/config.d/tls.xml.disabled b/programs/server/config.d/tls.xml.disabled index c81f005f2ed..b88b4b07231 100644 --- a/programs/server/config.d/tls.xml.disabled +++ b/programs/server/config.d/tls.xml.disabled @@ -1,4 +1,4 @@ - + 8443 9440 @@ -6,4 +6,4 @@ - + From 3a6f876abc30b254b7608d78682b92b024aa1275 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 26 Oct 2021 08:52:46 +0300 Subject: [PATCH 105/396] gtest_getMultipleValuesFromConfig: switch to --- src/Common/tests/gtest_getMultipleValuesFromConfig.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp b/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp index 30aba9418ba..e813cf4c75c 100644 --- a/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp +++ b/src/Common/tests/gtest_getMultipleValuesFromConfig.cpp @@ -11,14 +11,14 @@ TEST(Common, getMultipleValuesFromConfig) { std::istringstream // STYLE_CHECK_ALLOW_STD_STRING_STREAM xml_isteam(R"END( - + 0 1 2 3 -)END"); +)END"); Poco::AutoPtr config = new Poco::Util::XMLConfiguration(xml_isteam); std::vector answer = getMultipleValuesFromConfig(*config, "first_level", "second_level"); From da90df565826af4e48f247583d41eab375f7153a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 25 Oct 2021 21:15:42 +0300 Subject: [PATCH 106/396] docker: switch to --- docker/server/entrypoint.sh | 4 ++-- docker/test/stress/run.sh | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docker/server/entrypoint.sh b/docker/server/entrypoint.sh index 4dd54074d55..103f2fdb47f 100755 --- a/docker/server/entrypoint.sh +++ b/docker/server/entrypoint.sh @@ -86,7 +86,7 @@ done if [ -n "$CLICKHOUSE_USER" ] && [ "$CLICKHOUSE_USER" != "default" ] || [ -n "$CLICKHOUSE_PASSWORD" ]; then echo "$0: create new user '$CLICKHOUSE_USER' instead 'default'" cat < /etc/clickhouse-server/users.d/default-user.xml - + @@ -103,7 +103,7 @@ if [ -n "$CLICKHOUSE_USER" ] && [ "$CLICKHOUSE_USER" != "default" ] || [ -n "$CL ${CLICKHOUSE_ACCESS_MANAGEMENT} - + EOT fi diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index f42a3af17c9..a4e2f797210 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -46,11 +46,11 @@ function configure() sudo chown root: /var/lib/clickhouse # Set more frequent update period of asynchronous metrics to more frequently update information about real memory usage (less chance of OOM). - echo "1" \ + echo "1" \ > /etc/clickhouse-server/config.d/asynchronous_metrics_update_period_s.xml # Set maximum memory usage as half of total memory (less chance of OOM). - echo "0.5" \ + echo "0.5" \ > /etc/clickhouse-server/config.d/max_server_memory_usage_to_ram_ratio.xml } From 81436089dc7685677391fd83da5e8c60e646ca7c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 26 Oct 2021 08:53:23 +0300 Subject: [PATCH 107/396] tests: switch to --- tests/config/users.d/memory_profiler.xml | 4 ++-- tests/integration/helpers/0_common_enable_dictionaries.xml | 4 ++-- tests/integration/helpers/0_common_instance_config.xml | 4 ++-- tests/integration/helpers/0_common_instance_users.xml | 4 ++-- tests/integration/helpers/zookeeper_config.xml | 4 ++-- tests/integration/test_backup_with_other_granularity/test.py | 2 +- .../test_config_substitutions/configs/include_from_source.xml | 4 ++-- .../configs/default_compression.xml | 4 ++-- .../configs/wide_parts_only.xml | 4 ++-- .../config/dictionaries_config.xml | 4 ++-- tests/integration/test_dictionaries_config_reload/test.py | 4 ++-- .../configs/remote_servers.xml | 4 ++-- .../config/executable_user_defined_functions_config.xml | 4 ++-- .../test.py | 4 ++-- tests/integration/test_log_levels_update/configs/log.xml | 4 ++-- tests/integration/test_log_levels_update/test.py | 4 ++-- .../test_old_versions/configs/config.d/test_cluster.xml | 4 ++-- .../configs_old/users.d/not_optimize_count.xml | 4 ++-- .../configs/test_cluster.xml | 4 ++-- tests/integration/test_storage_rabbitmq/configs/macros.xml | 4 ++-- 20 files changed, 39 insertions(+), 39 deletions(-) diff --git a/tests/config/users.d/memory_profiler.xml b/tests/config/users.d/memory_profiler.xml index 56e9ba28347..a165ac701b0 100644 --- a/tests/config/users.d/memory_profiler.xml +++ b/tests/config/users.d/memory_profiler.xml @@ -1,8 +1,8 @@ - + 1Mi 1Mi - + diff --git a/tests/integration/helpers/0_common_enable_dictionaries.xml b/tests/integration/helpers/0_common_enable_dictionaries.xml index b6e52983db2..c7a763eb942 100644 --- a/tests/integration/helpers/0_common_enable_dictionaries.xml +++ b/tests/integration/helpers/0_common_enable_dictionaries.xml @@ -1,4 +1,4 @@ - + /etc/clickhouse-server/dictionaries/*.xml - + diff --git a/tests/integration/helpers/0_common_instance_config.xml b/tests/integration/helpers/0_common_instance_config.xml index c848ebdf45c..f4a9a0b6296 100644 --- a/tests/integration/helpers/0_common_instance_config.xml +++ b/tests/integration/helpers/0_common_instance_config.xml @@ -1,4 +1,4 @@ - + Europe/Moscow 0.0.0.0 custom_ @@ -17,4 +17,4 @@ /var/log/clickhouse-server/stderr.log /var/log/clickhouse-server/stdout.log - + diff --git a/tests/integration/helpers/0_common_instance_users.xml b/tests/integration/helpers/0_common_instance_users.xml index 7e799cb7b10..3399ef5915a 100644 --- a/tests/integration/helpers/0_common_instance_users.xml +++ b/tests/integration/helpers/0_common_instance_users.xml @@ -1,7 +1,7 @@ - + 1 - + diff --git a/tests/integration/helpers/zookeeper_config.xml b/tests/integration/helpers/zookeeper_config.xml index 9f864fef276..7a0d7c1de92 100644 --- a/tests/integration/helpers/zookeeper_config.xml +++ b/tests/integration/helpers/zookeeper_config.xml @@ -1,4 +1,4 @@ - + zoo1 @@ -14,4 +14,4 @@ 3000 - + diff --git a/tests/integration/test_backup_with_other_granularity/test.py b/tests/integration/test_backup_with_other_granularity/test.py index 832c1cf35ce..0f35c0f849e 100644 --- a/tests/integration/test_backup_with_other_granularity/test.py +++ b/tests/integration/test_backup_with_other_granularity/test.py @@ -111,7 +111,7 @@ def test_backup_from_old_version_config(started_cluster): def callback(n): n.replace_config("/etc/clickhouse-server/merge_tree_settings.xml", - "1") + "1") node3.restart_with_latest_version(callback_onstop=callback) diff --git a/tests/integration/test_config_substitutions/configs/include_from_source.xml b/tests/integration/test_config_substitutions/configs/include_from_source.xml index 6095180bb59..88a8b9d6e3d 100644 --- a/tests/integration/test_config_substitutions/configs/include_from_source.xml +++ b/tests/integration/test_config_substitutions/configs/include_from_source.xml @@ -1,4 +1,4 @@ - + 99999 @@ -14,4 +14,4 @@ default - + diff --git a/tests/integration/test_default_compression_codec/configs/default_compression.xml b/tests/integration/test_default_compression_codec/configs/default_compression.xml index abb2108f780..c0af422a4e6 100644 --- a/tests/integration/test_default_compression_codec/configs/default_compression.xml +++ b/tests/integration/test_default_compression_codec/configs/default_compression.xml @@ -1,4 +1,4 @@ - + @@ -26,4 +26,4 @@ - + diff --git a/tests/integration/test_default_compression_codec/configs/wide_parts_only.xml b/tests/integration/test_default_compression_codec/configs/wide_parts_only.xml index 42e2173f718..10b9edef36d 100644 --- a/tests/integration/test_default_compression_codec/configs/wide_parts_only.xml +++ b/tests/integration/test_default_compression_codec/configs/wide_parts_only.xml @@ -1,6 +1,6 @@ - + 0 0 - + diff --git a/tests/integration/test_dictionaries_config_reload/config/dictionaries_config.xml b/tests/integration/test_dictionaries_config_reload/config/dictionaries_config.xml index 7e62283a83c..3cbf717bb67 100644 --- a/tests/integration/test_dictionaries_config_reload/config/dictionaries_config.xml +++ b/tests/integration/test_dictionaries_config_reload/config/dictionaries_config.xml @@ -1,2 +1,2 @@ - - + + diff --git a/tests/integration/test_dictionaries_config_reload/test.py b/tests/integration/test_dictionaries_config_reload/test.py index 6f6e2488abc..4f338767304 100644 --- a/tests/integration/test_dictionaries_config_reload/test.py +++ b/tests/integration/test_dictionaries_config_reload/test.py @@ -17,9 +17,9 @@ def copy_file_to_container(local_path, dist_path, container_id): os.system("docker cp {local} {cont_id}:{dist}".format(local=local_path, cont_id=container_id, dist=dist_path)) -config = ''' +config = ''' /etc/clickhouse-server/dictionaries/{dictionaries_config} -''' +''' @pytest.fixture(scope="module") diff --git a/tests/integration/test_distributed_backward_compatability/configs/remote_servers.xml b/tests/integration/test_distributed_backward_compatability/configs/remote_servers.xml index ebce4697529..68b420f36b4 100644 --- a/tests/integration/test_distributed_backward_compatability/configs/remote_servers.xml +++ b/tests/integration/test_distributed_backward_compatability/configs/remote_servers.xml @@ -1,4 +1,4 @@ - + @@ -15,4 +15,4 @@ - + diff --git a/tests/integration/test_executable_user_defined_functions_config_reload/config/executable_user_defined_functions_config.xml b/tests/integration/test_executable_user_defined_functions_config_reload/config/executable_user_defined_functions_config.xml index 7e62283a83c..3cbf717bb67 100644 --- a/tests/integration/test_executable_user_defined_functions_config_reload/config/executable_user_defined_functions_config.xml +++ b/tests/integration/test_executable_user_defined_functions_config_reload/config/executable_user_defined_functions_config.xml @@ -1,2 +1,2 @@ - - + + diff --git a/tests/integration/test_executable_user_defined_functions_config_reload/test.py b/tests/integration/test_executable_user_defined_functions_config_reload/test.py index e5c4f4edb4e..3117b3e72b1 100644 --- a/tests/integration/test_executable_user_defined_functions_config_reload/test.py +++ b/tests/integration/test_executable_user_defined_functions_config_reload/test.py @@ -17,9 +17,9 @@ def copy_file_to_container(local_path, dist_path, container_id): os.system("docker cp {local} {cont_id}:{dist}".format(local=local_path, cont_id=container_id, dist=dist_path)) -config = ''' +config = ''' /etc/clickhouse-server/functions/{user_defined_executable_functions_config} -''' +''' @pytest.fixture(scope="module") diff --git a/tests/integration/test_log_levels_update/configs/log.xml b/tests/integration/test_log_levels_update/configs/log.xml index 668a15f6afd..a85417d05b8 100644 --- a/tests/integration/test_log_levels_update/configs/log.xml +++ b/tests/integration/test_log_levels_update/configs/log.xml @@ -1,6 +1,6 @@ - + trace /var/log/clickhouse-server/clickhouse-server.log - \ No newline at end of file + \ No newline at end of file diff --git a/tests/integration/test_log_levels_update/test.py b/tests/integration/test_log_levels_update/test.py index dca660a2982..f631677a400 100644 --- a/tests/integration/test_log_levels_update/test.py +++ b/tests/integration/test_log_levels_update/test.py @@ -6,12 +6,12 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__, name="log_quries_probability") node = cluster.add_instance('node', with_zookeeper=False) -config = ''' +config = ''' information /var/log/clickhouse-server/clickhouse-server.log -''' +''' @pytest.fixture(scope="module") diff --git a/tests/integration/test_old_versions/configs/config.d/test_cluster.xml b/tests/integration/test_old_versions/configs/config.d/test_cluster.xml index 56d53d63d78..f97538b35e3 100644 --- a/tests/integration/test_old_versions/configs/config.d/test_cluster.xml +++ b/tests/integration/test_old_versions/configs/config.d/test_cluster.xml @@ -1,4 +1,4 @@ - + @@ -10,4 +10,4 @@ - + diff --git a/tests/integration/test_polymorphic_parts/configs_old/users.d/not_optimize_count.xml b/tests/integration/test_polymorphic_parts/configs_old/users.d/not_optimize_count.xml index 5a06453b214..7f8036c4f87 100644 --- a/tests/integration/test_polymorphic_parts/configs_old/users.d/not_optimize_count.xml +++ b/tests/integration/test_polymorphic_parts/configs_old/users.d/not_optimize_count.xml @@ -1,7 +1,7 @@ - + 0 - + diff --git a/tests/integration/test_sharding_key_from_default_column/configs/test_cluster.xml b/tests/integration/test_sharding_key_from_default_column/configs/test_cluster.xml index 0437e047fad..a378e397011 100644 --- a/tests/integration/test_sharding_key_from_default_column/configs/test_cluster.xml +++ b/tests/integration/test_sharding_key_from_default_column/configs/test_cluster.xml @@ -1,4 +1,4 @@ - + @@ -15,4 +15,4 @@ - + diff --git a/tests/integration/test_storage_rabbitmq/configs/macros.xml b/tests/integration/test_storage_rabbitmq/configs/macros.xml index 6e9f3390b39..8f27ff51ed1 100644 --- a/tests/integration/test_storage_rabbitmq/configs/macros.xml +++ b/tests/integration/test_storage_rabbitmq/configs/macros.xml @@ -1,8 +1,8 @@ - + rabbitmq1 5672 macro JSONEachRow - + From e3b8557dd643471141bedb8672e2210df26e917c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 26 Oct 2021 08:58:50 +0300 Subject: [PATCH 108/396] tests/integration/cluster: switch to --- tests/integration/helpers/cluster.py | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 3854cadaba5..1bf2ce8c4c5 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -760,7 +760,7 @@ class ClickHouseCluster: hostname=None, env_variables=None, image="clickhouse/integration-test", tag=None, stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None, zookeeper_docker_compose_path=None, minio_certs_dir=None, use_keeper=True, - main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True, config_root_name="yandex"): + main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True, config_root_name="clickhouse"): """Add an instance to the cluster. @@ -1827,7 +1827,7 @@ class ClickHouseInstance: main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True, hostname=None, env_variables=None, image="clickhouse/integration-test", tag="latest", - stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None, config_root_name="yandex"): + stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=None, config_root_name="clickhouse"): self.name = name self.base_cmd = cluster.base_cmd @@ -2341,11 +2341,6 @@ class ClickHouseInstance: shutil.copyfile(p.join(self.base_config_dir, self.main_config_name), p.join(instance_config_dir, self.main_config_name)) shutil.copyfile(p.join(self.base_config_dir, self.users_config_name), p.join(instance_config_dir, self.users_config_name)) - # For old images, keep 'yandex' as root element name. - if self.image.startswith('yandex/'): - os.system("sed -i 's!!!; s!!!;' '{}'".format(p.join(instance_config_dir, self.main_config_name))) - os.system("sed -i 's!!!; s!!!;' '{}'".format(p.join(instance_config_dir, self.users_config_name))) - logging.debug("Create directory for configuration generated in this helper") # used by all utils with any config conf_d_dir = p.abspath(p.join(instance_config_dir, 'conf.d')) @@ -2363,7 +2358,7 @@ class ClickHouseInstance: def write_embedded_config(name, dest_dir, fix_log_level=False): with open(p.join(HELPERS_DIR, name), 'r') as f: data = f.read() - data = data.replace('yandex', self.config_root_name) + data = data.replace('clickhouse', self.config_root_name) if fix_log_level: data = data.replace('test', 'trace') with open(p.join(dest_dir, name), 'w') as r: From 950d3acf3a6fb99fb8ba182f9954736cc7c4bf92 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 26 Oct 2021 12:26:44 +0300 Subject: [PATCH 109/396] Update configuration-files.md --- docs/en/operations/configuration-files.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/configuration-files.md b/docs/en/operations/configuration-files.md index 318910b5b42..cbc139dd958 100644 --- a/docs/en/operations/configuration-files.md +++ b/docs/en/operations/configuration-files.md @@ -7,7 +7,7 @@ toc_title: Configuration Files ClickHouse supports multi-file configuration management. The main server configuration file is `/etc/clickhouse-server/config.xml` or `/etc/clickhouse-server/config.yaml`. Other files must be in the `/etc/clickhouse-server/config.d` directory. Note, that any configuration file can be written either in XML or YAML, but mixing formats in one file is not supported. For example, you can have main configs as `config.xml` and `users.xml` and write additional files in `config.d` and `users.d` directories in `.yaml`. -All XML files should have the same root element, usually ``. As for YAML, `yandex:` should not be present, the parser will insert it automatically. +All XML files should have the same root element, usually ``. As for YAML, `clickhouse:` should not be present, the parser will insert it automatically. ## Override {#override} From 49c776e77c791f3747b48ac4820220a6b0637fd0 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 26 Oct 2021 12:27:13 +0300 Subject: [PATCH 110/396] Update configuration-files.md --- docs/ru/operations/configuration-files.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/configuration-files.md b/docs/ru/operations/configuration-files.md index 5abb1ece455..9a3beef22ab 100644 --- a/docs/ru/operations/configuration-files.md +++ b/docs/ru/operations/configuration-files.md @@ -8,7 +8,7 @@ toc_title: "Конфигурационные файлы" ClickHouse поддерживает многофайловое управление конфигурацией. Основной конфигурационный файл сервера — `/etc/clickhouse-server/config.xml` или `/etc/clickhouse-server/config.yaml`. Остальные файлы должны находиться в директории `/etc/clickhouse-server/config.d`. Обратите внимание, что конфигурационные файлы могут быть записаны в форматах XML или YAML, но смешение этих форматов в одном файле не поддерживается. Например, можно хранить основные конфигурационные файлы как `config.xml` и `users.xml`, а дополнительные файлы записать в директории `config.d` и `users.d` в формате `.yaml`. -Все XML файлы должны иметь одинаковый корневой элемент, обычно ``. Для YAML элемент `yandex:` должен отсутствовать, так как парсер вставляет его автоматически. +Все XML файлы должны иметь одинаковый корневой элемент, обычно ``. Для YAML элемент `clickhouse:` должен отсутствовать, так как парсер вставляет его автоматически. ## Переопределение {#override} From d6b0ce1e94bbbd049711f9cc3f3e591dd1948e4d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Tue, 26 Oct 2021 17:37:11 +0800 Subject: [PATCH 111/396] delete crash-log.md symbolic link --- docs/zh/operations/system-tables/crash-log.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/crash-log.md diff --git a/docs/zh/operations/system-tables/crash-log.md b/docs/zh/operations/system-tables/crash-log.md deleted file mode 120000 index d1aa67601bc..00000000000 --- a/docs/zh/operations/system-tables/crash-log.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/crash-log.md \ No newline at end of file From aaa08e91b56b5e55eef9bbef85f6e931c5c5eefc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Tue, 26 Oct 2021 17:50:48 +0800 Subject: [PATCH 112/396] chinese translation for crash_log 1.Regenerate the crash-log.md file. 2.Supplement the internal translation content --- docs/zh/operations/system-tables/crash-log.md | 48 +++++++++++++++++++ 1 file changed, 48 insertions(+) create mode 100644 docs/zh/operations/system-tables/crash-log.md diff --git a/docs/zh/operations/system-tables/crash-log.md b/docs/zh/operations/system-tables/crash-log.md new file mode 100644 index 00000000000..54ba9c570bd --- /dev/null +++ b/docs/zh/operations/system-tables/crash-log.md @@ -0,0 +1,48 @@ +# system.crash_log {#system-tables_crash_log} + +包含有关致命错误堆栈跟踪的信息.该表默认不存在于数据库中, 仅在发生致命错误时才创建. + +列信息: + +- `event_date` ([Datetime](../../sql-reference/data-types/datetime.md)) — 事件日期. +- `event_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — 事件时间. +- `timestamp_ns` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 以纳秒为单位的事件时间戳. +- `signal` ([Int32](../../sql-reference/data-types/int-uint.md)) — 信号编号. +- `thread_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 线程ID. +- `query_id` ([String](../../sql-reference/data-types/string.md)) — 查询ID. +- `trace` ([Array](../../sql-reference/data-types/array.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 崩溃时的堆栈跟踪.每个元素都是 ClickHouse 服务器进程内的一个虚拟内存地址. +- `trace_full` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — 崩溃时的堆栈跟踪.每个元素在 ClickHouse 服务器进程中包含一个被调用的方法. +- `version` ([String](../../sql-reference/data-types/string.md)) — ClickHouse 服务器版本. +- `revision` ([UInt32](../../sql-reference/data-types/int-uint.md)) — ClickHouse 服务器订正版本. +- `build_id` ([String](../../sql-reference/data-types/string.md)) — 编译器生成的 BuildID. + +**Example** + +查询: + +``` sql +SELECT * FROM system.crash_log ORDER BY event_time DESC LIMIT 1; +``` + +结果 (部分): + +``` text +Row 1: +────── +event_date: 2020-10-14 +event_time: 2020-10-14 15:47:40 +timestamp_ns: 1602679660271312710 +signal: 11 +thread_id: 23624 +query_id: 428aab7c-8f5c-44e9-9607-d16b44467e69 +trace: [188531193,...] +trace_full: ['3. DB::(anonymous namespace)::FunctionFormatReadableTimeDelta::executeImpl(std::__1::vector >&, std::__1::vector > const&, unsigned long, unsigned long) const @ 0xb3cc1f9 in /home/username/work/ClickHouse/build/programs/clickhouse',...] +version: ClickHouse 20.11.1.1 +revision: 54442 +build_id: +``` + +**另请参阅** +- [trace_log](../../operations/system-tables/trace_log.md) 系统表 + +[Original article](https://clickhouse.com/docs/en/operations/system-tables/crash-log) From 2ba3ee830e773a5603391f9e81cb7e507b294532 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 26 Oct 2021 12:31:01 +0300 Subject: [PATCH 113/396] Partitioned write part 2 --- .../ExternalDataSourceConfiguration.cpp | 15 ++ .../ExternalDataSourceConfiguration.h | 1 + src/Storages/HDFS/StorageHDFS.cpp | 2 - src/Storages/StorageExternalDistributed.cpp | 3 +- src/Storages/StorageS3.cpp | 129 ++---------- src/Storages/StorageURL.cpp | 198 ++++++++++++++---- src/Storages/StorageURL.h | 22 +- src/Storages/StorageXDBC.cpp | 10 +- src/Storages/StorageXDBC.h | 1 + src/TableFunctions/TableFunctionURL.cpp | 50 +++-- src/TableFunctions/TableFunctionURL.h | 3 + .../integration/test_storage_url/__init__.py | 0 .../test_storage_url/configs/conf.xml | 11 + tests/integration/test_storage_url/test.py | 29 +++ 14 files changed, 283 insertions(+), 191 deletions(-) create mode 100644 tests/integration/test_storage_url/__init__.py create mode 100644 tests/integration/test_storage_url/configs/conf.xml create mode 100644 tests/integration/test_storage_url/test.py diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index 71be37e4b5b..c77acead06d 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -259,6 +259,7 @@ void URLBasedDataSourceConfiguration::set(const URLBasedDataSourceConfiguration format = conf.format; compression_method = conf.compression_method; structure = conf.structure; + method = conf.method; } @@ -286,6 +287,18 @@ std::optional getURLBasedDataSourceConfiguration(const { configuration.url = config.getString(config_prefix + ".url", ""); } + else if (key == "method") + { + configuration.method = config.getString(config_prefix + ".method", ""); + } + else if (key == "format") + { + configuration.format = config.getString(config_prefix + ".format", ""); + } + else if (key == "structure") + { + configuration.structure = config.getString(config_prefix + ".structure", ""); + } else if (key == "headers") { Poco::Util::AbstractConfiguration::Keys header_keys; @@ -319,6 +332,8 @@ std::optional getURLBasedDataSourceConfiguration(const if (arg_name == "url") configuration.url = arg_value.safeGet(); + if (arg_name == "method") + configuration.method = arg_value.safeGet(); else if (arg_name == "format") configuration.format = arg_value.safeGet(); else if (arg_name == "compression_method") diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index e3b589b1bb4..203b4933336 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -93,6 +93,7 @@ struct URLBasedDataSourceConfiguration String structure; std::vector> headers; + String method; void set(const URLBasedDataSourceConfiguration & conf); }; diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 133080321e3..c5ad0fcc30c 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -374,7 +374,6 @@ SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataP if (is_partitioned_implementation) { - std::cerr << "partitioned implementation\n"; return std::make_shared( insert_query->partition_by, uri, @@ -385,7 +384,6 @@ SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataP } else { - std::cerr << "non partitioned implementation\n"; return std::make_shared(uri, format_name, metadata_snapshot->getSampleBlock(), diff --git a/src/Storages/StorageExternalDistributed.cpp b/src/Storages/StorageExternalDistributed.cpp index 021cd9815d7..927c070826b 100644 --- a/src/Storages/StorageExternalDistributed.cpp +++ b/src/Storages/StorageExternalDistributed.cpp @@ -159,9 +159,8 @@ StorageExternalDistributed::StorageExternalDistributed( } else { - Poco::URI uri(url_description); shard = std::make_shared( - uri, table_id, format_name, format_settings, columns, constraints, String{}, context, compression_method); + url_description, table_id, format_name, format_settings, columns, constraints, String{}, context, compression_method); LOG_DEBUG(&Poco::Logger::get("StorageURLDistributed"), "Adding URL: {}", url_description); } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index aa036e9b955..a17a8512fc2 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #include @@ -353,7 +354,7 @@ private: }; -class PartitionedStorageS3Sink : public SinkToStorage +class PartitionedStorageS3Sink : public PartitionedSink { public: PartitionedStorageS3Sink( @@ -368,7 +369,7 @@ public: const String & key_, size_t min_upload_part_size_, size_t max_single_part_upload_size_) - : SinkToStorage(sample_block_) + : PartitionedSink(partition_by, context_, sample_block_) , format(format_) , sample_block(sample_block_) , context(context_) @@ -380,74 +381,36 @@ public: , max_single_part_upload_size(max_single_part_upload_size_) , format_settings(format_settings_) { - std::vector arguments(1, partition_by); - ASTPtr partition_by_string = makeASTFunction(FunctionToString::name, std::move(arguments)); - - auto syntax_result = TreeRewriter(context).analyze(partition_by_string, sample_block.getNamesAndTypesList()); - partition_by_expr = ExpressionAnalyzer(partition_by_string, syntax_result, context).getActions(false); - partition_by_column_name = partition_by_string->getColumnName(); } - String getName() const override { return "PartitionedStorageS3Sink"; } - - void consume(Chunk chunk) override + SinkPtr createSinkForPartition(const String & partition_id) override { - const auto & columns = chunk.getColumns(); + auto partition_bucket = replaceWildcards(bucket, partition_id); + validateBucket(partition_bucket); - Block block_with_partition_by_expr = sample_block.cloneWithoutColumns(); - block_with_partition_by_expr.setColumns(columns); - partition_by_expr->execute(block_with_partition_by_expr); + auto partition_key = replaceWildcards(key, partition_id); + validateKey(partition_key); - const auto * column = block_with_partition_by_expr.getByName(partition_by_column_name).column.get(); - - std::unordered_map sub_chunks_indices; - IColumn::Selector selector; - for (size_t row = 0; row < chunk.getNumRows(); ++row) - { - auto value = column->getDataAt(row); - auto [it, inserted] = sub_chunks_indices.emplace(value, sub_chunks_indices.size()); - selector.push_back(it->second); - } - - Chunks sub_chunks; - sub_chunks.reserve(sub_chunks_indices.size()); - for (size_t column_index = 0; column_index < columns.size(); ++column_index) - { - MutableColumns column_sub_chunks = columns[column_index]->scatter(sub_chunks_indices.size(), selector); - if (column_index == 0) /// Set sizes for sub-chunks. - { - for (const auto & column_sub_chunk : column_sub_chunks) - { - sub_chunks.emplace_back(Columns(), column_sub_chunk->size()); - } - } - for (size_t sub_chunk_index = 0; sub_chunk_index < column_sub_chunks.size(); ++sub_chunk_index) - { - sub_chunks[sub_chunk_index].addColumn(std::move(column_sub_chunks[sub_chunk_index])); - } - } - - for (const auto & [partition_id, sub_chunk_index] : sub_chunks_indices) - { - getSinkForPartition(partition_id)->consume(std::move(sub_chunks[sub_chunk_index])); - } - } - - void onFinish() override - { - for (auto & [partition_id, sink] : sinks) - { - sink->onFinish(); - } + return std::make_shared( + format, + sample_block, + context, + format_settings, + compression_method, + client, + partition_bucket, + partition_key, + min_upload_part_size, + max_single_part_upload_size + ); } private: - using SinkPtr = std::shared_ptr; - const String format; const Block sample_block; ContextPtr context; const CompressionMethod compression_method; + std::shared_ptr client; const String bucket; const String key; @@ -458,41 +421,6 @@ private: ExpressionActionsPtr partition_by_expr; String partition_by_column_name; - std::unordered_map sinks; - - static String replaceWildcards(const String & haystack, const String & partition_id) - { - return boost::replace_all_copy(haystack, PARTITION_ID_WILDCARD, partition_id); - } - - SinkPtr getSinkForPartition(const String & partition_id) - { - auto it = sinks.find(partition_id); - if (it == sinks.end()) - { - auto partition_bucket = replaceWildcards(bucket, partition_id); - validateBucket(partition_bucket); - - auto partition_key = replaceWildcards(key, partition_id); - validateKey(partition_key); - - std::tie(it, std::ignore) = sinks.emplace(partition_id, std::make_shared( - format, - sample_block, - context, - format_settings, - compression_method, - client, - partition_bucket, - partition_key, - min_upload_part_size, - max_single_part_upload_size - )); - } - - return it->second; - } - static void validateBucket(const String & str) { S3::URI::validateBucket(str, {}); @@ -517,21 +445,6 @@ private: validatePartitionKey(str, true); } - - static void validatePartitionKey(const StringRef & str, bool allow_slash) - { - for (const char * i = str.data; i != str.data + str.size; ++i) - { - if (static_cast(*i) < 0x20 || *i == '{' || *i == '}' || *i == '*' || *i == '?' || (!allow_slash && *i == '/')) - { - /// Need to convert to UInt32 because UInt8 can't be passed to format due to "mixing character types is disallowed". - UInt32 invalid_char_byte = static_cast(static_cast(*i)); - throw DB::Exception( - ErrorCodes::CANNOT_PARSE_TEXT, "Illegal character '\\x{:02x}' in partition id starting with '{}'", - invalid_char_byte, StringRef(str.data, i - str.data)); - } - } - } }; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 174ee58ee42..46cc13b450d 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -15,7 +16,9 @@ #include #include +#include #include +#include #include #include @@ -36,7 +39,7 @@ namespace ErrorCodes IStorageURLBase::IStorageURLBase( - const Poco::URI & uri_, + const String & uri_, ContextPtr /*context_*/, const StorageID & table_id_, const String & format_name_, @@ -45,8 +48,15 @@ IStorageURLBase::IStorageURLBase( const ConstraintsDescription & constraints_, const String & comment, const String & compression_method_, - const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_) - : IStorage(table_id_), uri(uri_), compression_method(compression_method_), format_name(format_name_), format_settings(format_settings_), headers(headers_) + const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_, + const String & method_) + : IStorage(table_id_) + , uri(uri_) + , compression_method(compression_method_) + , format_name(format_name_) + , format_settings(format_settings_) + , headers(headers_) + , method(method_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); @@ -80,13 +90,14 @@ namespace return headers; } + class StorageURLSource : public SourceWithProgress { using URIParams = std::vector>; public: StorageURLSource( - const std::vector & uri_options, + const std::vector & uri_options, const std::string & method, std::function callback, const String & format, @@ -109,7 +120,7 @@ namespace WriteBufferFromOwnString error_message; for (auto option = uri_options.begin(); option < uri_options.end(); ++option) { - auto request_uri = *option; + auto request_uri = Poco::URI(*option); for (const auto & [param, value] : params) request_uri.addQueryParameter(param, value); @@ -136,7 +147,7 @@ namespace if (option == uri_options.end() - 1) throw Exception(ErrorCodes::NETWORK_ERROR, "All uri options are unreachable. {}", error_message.str()); - error_message << option->toString() << " error: " << getCurrentExceptionMessage(false) << "\n"; + error_message << *option << " error: " << getCurrentExceptionMessage(false) << "\n"; tryLogCurrentException(__PRETTY_FUNCTION__); } } @@ -192,17 +203,18 @@ namespace } StorageURLSink::StorageURLSink( - const Poco::URI & uri, + const String & uri, const String & format, const std::optional & format_settings, const Block & sample_block, ContextPtr context, const ConnectionTimeouts & timeouts, - const CompressionMethod compression_method) + const CompressionMethod compression_method, + const String & method) : SinkToStorage(sample_block) { write_buf = wrapWriteBufferWithCompressionMethod( - std::make_unique(uri, Poco::Net::HTTPRequest::HTTP_POST, timeouts), + std::make_unique(Poco::URI(uri), method, timeouts), compression_method, 3); writer = FormatFactory::instance().getOutputFormat(format, *write_buf, sample_block, context, {} /* write callback */, format_settings); @@ -227,6 +239,50 @@ void StorageURLSink::onFinish() write_buf->finalize(); } +class PartitionedStorageURLSink : public PartitionedSink +{ +public: + PartitionedStorageURLSink( + const ASTPtr & partition_by, + const String & uri_, + const String & format_, + const std::optional & format_settings_, + const Block & sample_block_, + ContextPtr context_, + const ConnectionTimeouts & timeouts_, + const CompressionMethod compression_method_, + const String & method_) + : PartitionedSink(partition_by, context_, sample_block_) + , uri(uri_) + , format(format_) + , format_settings(format_settings_) + , sample_block(sample_block_) + , context(context_) + , timeouts(timeouts_) + , compression_method(compression_method_) + , method(method_) + { + } + + SinkPtr createSinkForPartition(const String & partition_id) override + { + auto partition_path = PartitionedSink::replaceWildcards(uri, partition_id); + context->getRemoteHostFilter().checkURL(Poco::URI(partition_path)); + return std::make_shared(partition_path, format, + format_settings, sample_block, context, timeouts, compression_method, method); + } + +private: + const String uri; + const String format; + const std::optional format_settings; + const Block sample_block; + ContextPtr context; + const ConnectionTimeouts timeouts; + + const CompressionMethod compression_method; + const String method; +}; std::string IStorageURLBase::getReadMethod() const { @@ -266,22 +322,57 @@ Pipe IStorageURLBase::read( unsigned /*num_streams*/) { auto params = getReadURIParams(column_names, metadata_snapshot, query_info, local_context, processed_stage, max_block_size); - std::vector uri_options{uri}; - return Pipe(std::make_shared( - uri_options, - getReadMethod(), - getReadPOSTDataCallback( - column_names, metadata_snapshot, query_info, - local_context, processed_stage, max_block_size), - format_name, - format_settings, - getName(), - getHeaderBlock(column_names, metadata_snapshot), - local_context, - metadata_snapshot->getColumns(), - max_block_size, - ConnectionTimeouts::getHTTPTimeouts(local_context), - compression_method, headers, params)); + auto with_globs = (uri.find('{') != std::string::npos && uri.find('}') != std::string::npos) || uri.find('|') == std::string::npos; + if (with_globs) + { + size_t max_addresses = local_context->getSettingsRef().glob_expansion_max_elements; + std::vector url_descriptions = parseRemoteDescription(uri, 0, uri.size(), ',', max_addresses); + std::vector uri_options; + + Pipes pipes; + for (const auto & url_description : url_descriptions) + { + /// For each uri (which acts like shard) check if it has failover options + uri_options = parseRemoteDescription(url_description, 0, url_description.size(), '|', max_addresses); + StoragePtr shard; + + pipes.emplace_back(std::make_shared( + uri_options, + getReadMethod(), + getReadPOSTDataCallback( + column_names, metadata_snapshot, query_info, + local_context, processed_stage, max_block_size), + format_name, + format_settings, + getName(), + getHeaderBlock(column_names, metadata_snapshot), + local_context, + metadata_snapshot->getColumns(), + max_block_size, + ConnectionTimeouts::getHTTPTimeouts(local_context), + compression_method, headers, params)); + } + return Pipe::unitePipes(std::move(pipes)); + } + else + { + std::vector uri_options{uri}; + return Pipe(std::make_shared( + uri_options, + getReadMethod(), + getReadPOSTDataCallback( + column_names, metadata_snapshot, query_info, + local_context, processed_stage, max_block_size), + format_name, + format_settings, + getName(), + getHeaderBlock(column_names, metadata_snapshot), + local_context, + metadata_snapshot->getColumns(), + max_block_size, + ConnectionTimeouts::getHTTPTimeouts(local_context), + compression_method, headers, params)); + } } @@ -315,16 +406,35 @@ Pipe StorageURLWithFailover::read( } -SinkToStoragePtr IStorageURLBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) +SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) { - return std::make_shared(uri, format_name, - format_settings, metadata_snapshot->getSampleBlock(), context, - ConnectionTimeouts::getHTTPTimeouts(context), - chooseCompressionMethod(uri.toString(), compression_method)); + if (method.empty()) + method = Poco::Net::HTTPRequest::HTTP_POST; + + bool has_wildcards = uri.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; + const auto * insert_query = dynamic_cast(query.get()); + bool is_partitioned_implementation = insert_query && insert_query->partition_by && has_wildcards; + + if (is_partitioned_implementation) + { + return std::make_shared( + insert_query->partition_by, + uri, format_name, + format_settings, metadata_snapshot->getSampleBlock(), context, + ConnectionTimeouts::getHTTPTimeouts(context), + chooseCompressionMethod(uri, compression_method), method); + } + else + { + return std::make_shared(uri, format_name, + format_settings, metadata_snapshot->getSampleBlock(), context, + ConnectionTimeouts::getHTTPTimeouts(context), + chooseCompressionMethod(uri, compression_method), method); + } } StorageURL::StorageURL( - const Poco::URI & uri_, + const String & uri_, const StorageID & table_id_, const String & format_name_, const std::optional & format_settings_, @@ -333,10 +443,11 @@ StorageURL::StorageURL( const String & comment, ContextPtr context_, const String & compression_method_, - const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_) - : IStorageURLBase(uri_, context_, table_id_, format_name_, format_settings_, columns_, constraints_, comment, compression_method_, headers_) + const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_, + const String & method_) + : IStorageURLBase(uri_, context_, table_id_, format_name_, format_settings_, columns_, constraints_, comment, compression_method_, headers_, method_) { - context_->getRemoteHostFilter().checkURL(uri); + context_->getRemoteHostFilter().checkURL(Poco::URI(uri)); } @@ -349,13 +460,13 @@ StorageURLWithFailover::StorageURLWithFailover( const ConstraintsDescription & constraints_, ContextPtr context_, const String & compression_method_) - : StorageURL(Poco::URI(), table_id_, format_name_, format_settings_, columns_, constraints_, String{}, context_, compression_method_) + : StorageURL("", table_id_, format_name_, format_settings_, columns_, constraints_, String{}, context_, compression_method_) { for (const auto & uri_option : uri_options_) { Poco::URI poco_uri(uri_option); context_->getRemoteHostFilter().checkURL(poco_uri); - uri_options.emplace_back(std::move(poco_uri)); + uri_options.emplace_back(std::move(uri_option)); LOG_DEBUG(&Poco::Logger::get("StorageURLDistributed"), "Adding URL option: {}", uri_option); } } @@ -405,6 +516,13 @@ URLBasedDataSourceConfiguration StorageURL::getConfiguration(ASTs & args, Contex auto [common_configuration, storage_specific_args] = named_collection.value(); configuration.set(common_configuration); + if (!configuration.method.empty() + && configuration.method != Poco::Net::HTTPRequest::HTTP_POST + && configuration.method != Poco::Net::HTTPRequest::HTTP_PUT) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Method can be POST or PUT (current: {}). For insert default is POST, for select GET", + configuration.method); + if (!storage_specific_args.empty()) { String illegal_args; @@ -421,7 +539,8 @@ URLBasedDataSourceConfiguration StorageURL::getConfiguration(ASTs & args, Contex { if (args.size() != 2 && args.size() != 3) throw Exception( - "Storage URL requires 2 or 3 arguments: url, name of used format and optional compression method.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + "Storage URL requires 2 or 3 arguments: url, name of used format and optional compression method.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); for (auto & arg : args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, local_context); @@ -443,7 +562,6 @@ void registerStorageURL(StorageFactory & factory) ASTs & engine_args = args.engine_args; auto configuration = StorageURL::getConfiguration(engine_args, args.getLocalContext()); auto format_settings = StorageURL::getFormatSettingsFromArgs(args); - Poco::URI uri(configuration.url); ReadWriteBufferFromHTTP::HTTPHeaderEntries headers; for (const auto & [header, value] : configuration.headers) @@ -453,7 +571,7 @@ void registerStorageURL(StorageFactory & factory) } return StorageURL::create( - uri, + configuration.url, args.table_id, configuration.format, format_settings, @@ -462,7 +580,7 @@ void registerStorageURL(StorageFactory & factory) args.comment, args.getContext(), configuration.compression_method, - headers); + headers, configuration.method); }, { .supports_settings = true, diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 1f2cac239e1..42110cafd75 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -39,9 +39,11 @@ public: SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; + bool supportsPartitionBy() const override { return true; } + protected: IStorageURLBase( - const Poco::URI & uri_, + const String & uri_, ContextPtr context_, const StorageID & id_, const String & format_name_, @@ -50,9 +52,10 @@ protected: const ConstraintsDescription & constraints_, const String & comment, const String & compression_method_, - const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_ = {}); + const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_ = {}, + const String & method_ = ""); - Poco::URI uri; + String uri; String compression_method; String format_name; // For URL engine, we use format settings from server context + `SETTINGS` @@ -61,6 +64,7 @@ protected: // In this case, format_settings is not set. std::optional format_settings; ReadWriteBufferFromHTTP::HTTPHeaderEntries headers; + String method; /// For insert can choose Put instead of default Post. virtual std::string getReadMethod() const; @@ -88,13 +92,14 @@ class StorageURLSink : public SinkToStorage { public: StorageURLSink( - const Poco::URI & uri, + const String & uri, const String & format, const std::optional & format_settings, const Block & sample_block, ContextPtr context, const ConnectionTimeouts & timeouts, - CompressionMethod compression_method); + CompressionMethod compression_method, + const String & method = Poco::Net::HTTPRequest::HTTP_POST); std::string getName() const override { return "StorageURLSink"; } void consume(Chunk chunk) override; @@ -112,7 +117,7 @@ class StorageURL : public shared_ptr_helper, public IStorageURLBase friend struct shared_ptr_helper; public: StorageURL( - const Poco::URI & uri_, + const String & uri_, const StorageID & table_id_, const String & format_name_, const std::optional & format_settings_, @@ -121,7 +126,8 @@ public: const String & comment, ContextPtr context_, const String & compression_method_, - const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_ = {}); + const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_ = {}, + const String & method_ = ""); String getName() const override { @@ -170,6 +176,6 @@ public: }; private: - std::vector uri_options; + std::vector uri_options; }; } diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index a90e21a2edb..3ccde3a3597 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -33,7 +33,7 @@ StorageXDBC::StorageXDBC( const BridgeHelperPtr bridge_helper_) /// Please add support for constraints as soon as StorageODBC or JDBC will support insertion. : IStorageURLBase( - Poco::URI(), + "", context_, table_id_, IXDBCBridgeHelper::DEFAULT_FORMAT, @@ -47,7 +47,7 @@ StorageXDBC::StorageXDBC( , remote_table_name(remote_table_name_) , log(&Poco::Logger::get("Storage" + bridge_helper->getName())) { - uri = bridge_helper->getMainURI(); + poco_uri = bridge_helper->getMainURI(); } std::string StorageXDBC::getReadMethod() const @@ -118,7 +118,7 @@ SinkToStoragePtr StorageXDBC::write(const ASTPtr & /*query*/, const StorageMetad { bridge_helper->startBridgeSync(); - Poco::URI request_uri = uri; + Poco::URI request_uri = poco_uri; request_uri.setPath("/write"); auto url_params = bridge_helper->getURLParams(65536); @@ -131,13 +131,13 @@ SinkToStoragePtr StorageXDBC::write(const ASTPtr & /*query*/, const StorageMetad request_uri.addQueryParameter("sample_block", metadata_snapshot->getSampleBlock().getNamesAndTypesList().toString()); return std::make_shared( - request_uri, + request_uri.toString(), format_name, getFormatSettings(local_context), metadata_snapshot->getSampleBlock(), local_context, ConnectionTimeouts::getHTTPTimeouts(local_context), - chooseCompressionMethod(uri.toString(), compression_method)); + chooseCompressionMethod(poco_uri.toString(), compression_method)); } Block StorageXDBC::getHeaderBlock(const Names & column_names, const StorageMetadataPtr & metadata_snapshot) const diff --git a/src/Storages/StorageXDBC.h b/src/Storages/StorageXDBC.h index 89bd7856b06..34207dcbb4c 100644 --- a/src/Storages/StorageXDBC.h +++ b/src/Storages/StorageXDBC.h @@ -41,6 +41,7 @@ public: std::string getName() const override; private: + Poco::URI poco_uri; BridgeHelperPtr bridge_helper; std::string remote_database_name; diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index bbaa1b5f048..3fb6fcc2433 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -24,12 +24,18 @@ void TableFunctionURL::parseArguments(const ASTPtr & ast_function, ContextPtr co if (!func_args.arguments) throw Exception("Table function 'URL' must have arguments.", ErrorCodes::BAD_ARGUMENTS); - URLBasedDataSourceConfiguration configuration; if (auto with_named_collection = getURLBasedDataSourceConfiguration(func_args.arguments->children, context)) { auto [common_configuration, storage_specific_args] = with_named_collection.value(); configuration.set(common_configuration); + if (!configuration.method.empty() + && configuration.method != Poco::Net::HTTPRequest::HTTP_POST + && configuration.method != Poco::Net::HTTPRequest::HTTP_PUT) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Method can be POST or PUT (current: {}). For insert default is POST, for select GET", + configuration.method); + if (!storage_specific_args.empty()) { String illegal_args; @@ -58,33 +64,25 @@ StoragePtr TableFunctionURL::getStorage( const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, const std::string & table_name, const String & compression_method_) const { - /// If url contains {1..k} or failover options with separator `|`, use a separate storage - if ((source.find('{') == std::string::npos || source.find('}') == std::string::npos) && source.find('|') == std::string::npos) + ReadWriteBufferFromHTTP::HTTPHeaderEntries headers; + for (const auto & [header, value] : configuration.headers) { - Poco::URI uri(source); - return StorageURL::create( - uri, - StorageID(getDatabaseName(), table_name), - format_, - std::nullopt /*format settings*/, - columns, - ConstraintsDescription{}, - String{}, - global_context, - compression_method_); - } - else - { - return StorageExternalDistributed::create( - source, - StorageID(getDatabaseName(), table_name), - format_, - std::nullopt, - compression_method_, - columns, - ConstraintsDescription{}, - global_context); + auto value_literal = value.safeGet(); + headers.emplace_back(std::make_pair(header, value_literal)); } + + return StorageURL::create( + source, + StorageID(getDatabaseName(), table_name), + format_, + std::nullopt /*format settings*/, + columns, + ConstraintsDescription{}, + String{}, + global_context, + compression_method_, + headers, + configuration.method); } void registerTableFunctionURL(TableFunctionFactory & factory) diff --git a/src/TableFunctions/TableFunctionURL.h b/src/TableFunctions/TableFunctionURL.h index c35db9f9c8b..9425112acb2 100644 --- a/src/TableFunctions/TableFunctionURL.h +++ b/src/TableFunctions/TableFunctionURL.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB @@ -27,6 +28,8 @@ private: const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, const std::string & table_name, const String & compression_method_) const override; const char * getStorageTypeName() const override { return "URL"; } + + URLBasedDataSourceConfiguration configuration; }; } diff --git a/tests/integration/test_storage_url/__init__.py b/tests/integration/test_storage_url/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_url/configs/conf.xml b/tests/integration/test_storage_url/configs/conf.xml new file mode 100644 index 00000000000..e3e8627d95e --- /dev/null +++ b/tests/integration/test_storage_url/configs/conf.xml @@ -0,0 +1,11 @@ + + + + + http://nginx:80/test_{_partition_id} + PUT + TSV + column1 UInt32, column2 UInt32, column3 UInt32 + + + diff --git a/tests/integration/test_storage_url/test.py b/tests/integration/test_storage_url/test.py new file mode 100644 index 00000000000..1ced71bc849 --- /dev/null +++ b/tests/integration/test_storage_url/test.py @@ -0,0 +1,29 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +uuids = [] + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance("node1", main_configs=["configs/conf.xml"], with_nginx=True) + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def test_partition_by(cluster): + node1 = cluster.instances["node1"] + + node1.query(f"insert into table function url(url1) partition by column3 values (1, 2, 3), (3, 2, 1), (1, 3, 2)") + result = node1.query(f"select * from url('http://nginx:80/test_1', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')") + assert(result.strip() == "3\t2\t1") + result = node1.query(f"select * from url('http://nginx:80/test_2', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')") + assert(result.strip() == "1\t3\t2") + result = node1.query(f"select * from url('http://nginx:80/test_3', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32')") + assert(result.strip() == "1\t2\t3") From 5a5fb50c9f2b484d969d536ddc6bf07da58bde6c Mon Sep 17 00:00:00 2001 From: lehasm Date: Tue, 26 Oct 2021 20:04:22 +0300 Subject: [PATCH 114/396] Update docs/ru/operations/settings/settings.md Co-authored-by: olgarev <56617294+olgarev@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 2ec9edb460b..27172dbd91b 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3606,7 +3606,7 @@ SELECT * FROM positional_arguments ORDER BY 2,3; ## describe_include_subcolumns {#describe_include_subcolumns} -Разрешает описание подстолбцов при выполнении запроса [DESCRIBE](../../sql-reference/statements/describe-table.md). Настройка действует на подстолбцы внутри столбцов типов [Tuple](../../sql-reference/data-types/tuple.md) или [Map](../../sql-reference/data-types/map.md#map-subcolumns). +Включает или отключает описание подстолбцов при выполнении запроса [DESCRIBE](../../sql-reference/statements/describe-table.md). Настройка действует на подстолбцы внутри столбцов типов [Tuple](../../sql-reference/data-types/tuple.md) или [Map](../../sql-reference/data-types/map.md#map-subcolumns). Возможные значения: From 0f305136522373cbaaa9e7d4ee1c9a9f2e487956 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 26 Oct 2021 20:17:02 +0300 Subject: [PATCH 115/396] Revert "Revert "Improve usability of `remote_url_allow_hosts`"" --- src/Common/RemoteHostFilter.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/RemoteHostFilter.cpp b/src/Common/RemoteHostFilter.cpp index 73c84364f3c..ba7f163fd16 100644 --- a/src/Common/RemoteHostFilter.cpp +++ b/src/Common/RemoteHostFilter.cpp @@ -18,14 +18,14 @@ void RemoteHostFilter::checkURL(const Poco::URI & uri) const { if (!checkForDirectEntry(uri.getHost()) && !checkForDirectEntry(uri.getHost() + ":" + toString(uri.getPort()))) - throw Exception("URL \"" + uri.toString() + "\" is not allowed in config.xml", ErrorCodes::UNACCEPTABLE_URL); + throw Exception("URL \"" + uri.toString() + "\" is not allowed in configuration file, see ", ErrorCodes::UNACCEPTABLE_URL); } void RemoteHostFilter::checkHostAndPort(const std::string & host, const std::string & port) const { if (!checkForDirectEntry(host) && !checkForDirectEntry(host + ":" + port)) - throw Exception("URL \"" + host + ":" + port + "\" is not allowed in config.xml", ErrorCodes::UNACCEPTABLE_URL); + throw Exception("URL \"" + host + ":" + port + "\" is not allowed in configuration file, see ", ErrorCodes::UNACCEPTABLE_URL); } void RemoteHostFilter::setValuesFromConfig(const Poco::Util::AbstractConfiguration & config) From 9138d8965d5d1f5c25ee7ab4559b593b82aaf68e Mon Sep 17 00:00:00 2001 From: andrc1901 <92211164+andrc1901@users.noreply.github.com> Date: Tue, 26 Oct 2021 20:46:44 +0300 Subject: [PATCH 116/396] Update docs/en/sql-reference/functions/splitting-merging-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/en/sql-reference/functions/splitting-merging-functions.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/functions/splitting-merging-functions.md b/docs/en/sql-reference/functions/splitting-merging-functions.md index 0323834c5aa..0c91820b244 100644 --- a/docs/en/sql-reference/functions/splitting-merging-functions.md +++ b/docs/en/sql-reference/functions/splitting-merging-functions.md @@ -324,6 +324,8 @@ Type: [Array](../data-types/array.md). **Example** +Query: + ``` sql SELECT tokens('test1,;\\ test2,;\\ test3,;\\ test4') AS tokens; ``` From e45c2241a9340acfcb71e19545d12719fb4ea8d1 Mon Sep 17 00:00:00 2001 From: andrc1901 <92211164+andrc1901@users.noreply.github.com> Date: Tue, 26 Oct 2021 20:47:10 +0300 Subject: [PATCH 117/396] Update docs/en/sql-reference/functions/splitting-merging-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/en/sql-reference/functions/splitting-merging-functions.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/functions/splitting-merging-functions.md b/docs/en/sql-reference/functions/splitting-merging-functions.md index 0c91820b244..93bf2746c3c 100644 --- a/docs/en/sql-reference/functions/splitting-merging-functions.md +++ b/docs/en/sql-reference/functions/splitting-merging-functions.md @@ -330,6 +330,8 @@ Query: SELECT tokens('test1,;\\ test2,;\\ test3,;\\ test4') AS tokens; ``` +Result: + ``` text ┌─tokens────────────────────────────┐ │ ['test1','test2','test3','test4'] │ From c569a1b37e9fe3a47d3b4a5da34af429d7037c60 Mon Sep 17 00:00:00 2001 From: andrc1901 <92211164+andrc1901@users.noreply.github.com> Date: Tue, 26 Oct 2021 20:47:22 +0300 Subject: [PATCH 118/396] Update docs/ru/sql-reference/functions/splitting-merging-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/functions/splitting-merging-functions.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/ru/sql-reference/functions/splitting-merging-functions.md b/docs/ru/sql-reference/functions/splitting-merging-functions.md index df2f778d6b6..7e7a80def2e 100644 --- a/docs/ru/sql-reference/functions/splitting-merging-functions.md +++ b/docs/ru/sql-reference/functions/splitting-merging-functions.md @@ -290,6 +290,8 @@ SELECT ngrams('ClickHouse', 3); SELECT tokens('test1,;\\ test2,;\\ test3,;\\ test4') AS tokens; ``` +Результат: + ``` text ┌─tokens────────────────────────────┐ │ ['test1','test2','test3','test4'] │ From a4d0bbe399f8d81ab27b7860145d64e46469c3c3 Mon Sep 17 00:00:00 2001 From: andrc1901 <92211164+andrc1901@users.noreply.github.com> Date: Tue, 26 Oct 2021 20:47:32 +0300 Subject: [PATCH 119/396] Update docs/ru/sql-reference/functions/splitting-merging-functions.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/sql-reference/functions/splitting-merging-functions.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/ru/sql-reference/functions/splitting-merging-functions.md b/docs/ru/sql-reference/functions/splitting-merging-functions.md index 7e7a80def2e..595ad22ac46 100644 --- a/docs/ru/sql-reference/functions/splitting-merging-functions.md +++ b/docs/ru/sql-reference/functions/splitting-merging-functions.md @@ -286,6 +286,8 @@ SELECT ngrams('ClickHouse', 3); **Пример** +Запрос: + ``` sql SELECT tokens('test1,;\\ test2,;\\ test3,;\\ test4') AS tokens; ``` From 8d0f4e0d7077a412b5a6fb6a370e41c2d94fed2f Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Tue, 26 Oct 2021 21:20:24 +0300 Subject: [PATCH 120/396] Translate to Russian MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Выполнил перевод на русский язык. --- docs/en/operations/system-tables/columns.md | 10 +- .../system-tables/information_schema.md | 21 +- docs/en/operations/system-tables/tables.md | 2 +- docs/ru/operations/system-tables/columns.md | 31 ++- .../system-tables/information_schema.md | 210 ++++++++++++++++++ docs/ru/operations/system-tables/tables.md | 23 ++ 6 files changed, 270 insertions(+), 27 deletions(-) create mode 100644 docs/ru/operations/system-tables/information_schema.md diff --git a/docs/en/operations/system-tables/columns.md b/docs/en/operations/system-tables/columns.md index 087e6dea8fb..d9f1ea18196 100644 --- a/docs/en/operations/system-tables/columns.md +++ b/docs/en/operations/system-tables/columns.md @@ -24,11 +24,11 @@ Columns: - `is_in_primary_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the primary key expression. - `is_in_sampling_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the sampling key expression. - `compression_codec` ([String](../../sql-reference/data-types/string.md)) — Compression codec name. -- `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. -- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. Otherwise, the NULL value is returned. -- `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. Otherwise, the NULL value is returned. -- `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. Otherwise, the NULL value is returned. -- `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Subtype code for `DateTime` and `ISO` interval data types. For other data types, the NULL value is returned. +- `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. Otherwise, the `NULL` value is returned. +- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. Otherwise, the `NULL` value is returned. +- `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. Otherwise, the `NULL` value is returned. +- `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. Otherwise, the `NULL` value is returned. +- `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Subtype code for `DateTime` and `ISO` interval data types. For other data types, the `NULL` value is returned. **Example** diff --git a/docs/en/operations/system-tables/information_schema.md b/docs/en/operations/system-tables/information_schema.md index 5aa20515643..20926f512cb 100644 --- a/docs/en/operations/system-tables/information_schema.md +++ b/docs/en/operations/system-tables/information_schema.md @@ -1,6 +1,6 @@ # information_schema {#information-schema} -`INFORMATION_SCHEMA` (`information_schema`) is a system database that contains views. Using these views, you can get information about the metadata of database objects. +`INFORMATION_SCHEMA` (`information_schema`) is a system database that contains views. Using these views, you can get information about the metadata of database objects. These views read data from the columns of the [system.columns](../../operations/system-tables/columns.md), [system.databases](../../operations/system-tables/databases.md) and [system.tables](../../operations/system-tables/tables.md) system tables. The structure and composition of system tables may change in different versions of the product, but the support of the `information_schema` makes it possible to change the structure of system tables without changing the method of access to metadata. Metadata requests do not depend on the DBMS used. @@ -26,7 +26,7 @@ SHOW TABLES FROM INFORMATION_SCHEMA; ## COLUMNS {#columns} -Allows to get a list of table columns read using this view. +Contains columns read from the `system.columns` system table and columns that are not supported in ClickHouse or do not make sense (always `NULL`), but must be by the standard. Columns: @@ -94,13 +94,13 @@ domain_name: ᴺᵁᴸᴸ ## SCHEMATA {#schemata} -Allows to get a list of schemas read using this view. +Contains columns read from the `system.databases` system table and columns that are not supported in ClickHouse or do not make sense (always `NULL`), but must be by the standard. Columns: - `catalog_name` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the table is located. - `schema_name` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the schema is located. -- `schema_owner` ([String](../../sql-reference/data-types/string.md)) — Schema owner name. +- `schema_owner` ([String](../../sql-reference/data-types/string.md)) — Schema owner name, always `'default'`. - `default_character_set_catalog` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. - `default_character_set_schema` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. - `default_character_set_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. @@ -130,7 +130,7 @@ sql_path: ᴺᵁᴸᴸ ## TABLES {#tables} -Allows to get a list of tables read using this view. +Contains columns read from the `system.tables` system table. Columns: @@ -165,7 +165,7 @@ table_type: BASE TABLE ## VIEWS {#views} -Allows to get a list of views read using this view. +Contains columns read from the `system.tables` system table, when the table engine [View](../../engines/table-engines/special/view.md) is used. Columns: @@ -175,7 +175,7 @@ Columns: - `view_definition` ([String](../../sql-reference/data-types/string.md)) — `SELECT` query for view. - `check_option` ([String](../../sql-reference/data-types/string.md)) — `NONE`, no checking. - `is_updatable` ([Enum8](../../sql-reference/data-types/enum.md)) — `NO`, the view is not updated. -- `is_insertable_into` ([Enum8](../../sql-reference/data-types/enum.md)) — Shows whether the created view is materialized. Possible values: +- `is_insertable_into` ([Enum8](../../sql-reference/data-types/enum.md)) — Shows whether the created view is [materialized](../../sql-reference/statements/create/view/#materialized). Possible values: - `NO` — The created view is not materialized. - `YES` — The created view is materialized. - `is_trigger_updatable` ([Enum8](../../sql-reference/data-types/enum.md)) — `NO`, the trigger is not updated. @@ -208,10 +208,3 @@ is_trigger_updatable: NO is_trigger_deletable: NO is_trigger_insertable_into: NO ``` - -**See Also** - -- [View](../../sql-reference/statements/create/view.md). -- [system.tables](../../operations/system-tables/tables.md). -- [system.columns](../../operations/system-tables/columns.md). -- [system.query_views_log](../../operations/system-tables/query_views_log.md). diff --git a/docs/en/operations/system-tables/tables.md b/docs/en/operations/system-tables/tables.md index 9bacd468b87..0ccf69bc048 100644 --- a/docs/en/operations/system-tables/tables.md +++ b/docs/en/operations/system-tables/tables.md @@ -58,7 +58,7 @@ Columns: - `comment` ([String](../../sql-reference/data-types/string.md)) - The comment for the table. -- `has_own_data` ([UInt8](../../sql-reference/data-types/int-uint.md)) - Flag that indicates whether the table itself stores some data on disk or only accesses some other source. +- `has_own_data` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the table itself stores some data on disk or only accesses some other source. The `system.tables` table is used in `SHOW TABLES` query implementation. diff --git a/docs/ru/operations/system-tables/columns.md b/docs/ru/operations/system-tables/columns.md index a896360b3f9..5b526372820 100644 --- a/docs/ru/operations/system-tables/columns.md +++ b/docs/ru/operations/system-tables/columns.md @@ -24,6 +24,11 @@ Cтолбцы: - `is_in_primary_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — флаг, показывающий включение столбца в первичный ключ. - `is_in_sampling_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — флаг, показывающий включение столбца в ключ выборки. - `compression_codec` ([String](../../sql-reference/data-types/string.md)) — имя кодека сжатия. +- `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Максимальная длина в байтах для двоичных данных, символьных данных или текстовых данных и изображений. Иначе возвращается значение `NULL`. +- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Точность приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. Иначе возвращается значение `NULL`. +- `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Основание системы счисления точности приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. Иначе возвращается значение `NULL`. +- `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Масштаб приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. Иначе возвращается значение `NULL`. +- `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Код подтипа для типов данных интервала `DateTime` и `ISO`. Для других типов данных возвращается значение `NULL`. **Пример** @@ -34,10 +39,11 @@ SELECT * FROM system.columns LIMIT 2 FORMAT Vertical; ```text Row 1: ────── -database: system -table: aggregate_function_combinators -name: name +database: INFORMATION_SCHEMA +table: COLUMNS +name: table_catalog type: String +position: 1 default_kind: default_expression: data_compressed_bytes: 0 @@ -49,13 +55,19 @@ is_in_sorting_key: 0 is_in_primary_key: 0 is_in_sampling_key: 0 compression_codec: +character_octet_length: ᴺᵁᴸᴸ +numeric_precision: ᴺᵁᴸᴸ +numeric_precision_radix: ᴺᵁᴸᴸ +numeric_scale: ᴺᵁᴸᴸ +datetime_precision: ᴺᵁᴸᴸ Row 2: ────── -database: system -table: aggregate_function_combinators -name: is_internal -type: UInt8 +database: INFORMATION_SCHEMA +table: COLUMNS +name: table_schema +type: String +position: 2 default_kind: default_expression: data_compressed_bytes: 0 @@ -67,4 +79,9 @@ is_in_sorting_key: 0 is_in_primary_key: 0 is_in_sampling_key: 0 compression_codec: +character_octet_length: ᴺᵁᴸᴸ +numeric_precision: ᴺᵁᴸᴸ +numeric_precision_radix: ᴺᵁᴸᴸ +numeric_scale: ᴺᵁᴸᴸ +datetime_precision: ᴺᵁᴸᴸ ``` diff --git a/docs/ru/operations/system-tables/information_schema.md b/docs/ru/operations/system-tables/information_schema.md new file mode 100644 index 00000000000..dad7f99e2bd --- /dev/null +++ b/docs/ru/operations/system-tables/information_schema.md @@ -0,0 +1,210 @@ +# information_schema {#information-schema} + +`INFORMATION_SCHEMA` (`information_schema`) — это системная база данных, содержащая представления. Используя эти представления, вы можете получить информацию о метаданных объектов базы данных. Эти представления считывают данные из столбцов системных таблиц [system.columns](../../operations/system-tables/columns.md), [system.databases](../../operations/system-tables/databases.md) и [system.tables](../../operations/system-tables/tables.md). + +Структура и состав системных таблиц могут меняться в разных версиях продукта, но поддержка `information_schema` позволяет изменять структуру системных таблиц без изменения способа доступа к метаданным. Запросы метаданных не зависят от используемой СУБД. + +``` sql +SHOW TABLES FROM INFORMATION_SCHEMA; +``` + +``` text +┌─name─────┐ +│ COLUMNS │ +│ SCHEMATA │ +│ TABLES │ +│ VIEWS │ +└──────────┘ +``` + +`INFORMATION_SCHEMA` содержит следующие представления: + +- [COLUMNS](#columns) +- [SCHEMATA](#schemata) +- [TABLES](#tables) +- [VIEWS](#views) + +## COLUMNS {#columns} + +Содержит столбцы, которые считываются из системной таблицы `system.columns`, и столбцы, которые не поддерживаются в ClickHouse или не имеют смысла (всегда имеют значение `NULL`), но должны быть по стандарту. + +Столбцы: + +- `table_catalog` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится таблица. +- `table_schema` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится схема. +- `table_name` ([String](../../sql-reference/data-types/string.md)) — имя таблицы. +- `column_name` ([String](../../sql-reference/data-types/string.md)) — имя столбца. +- `ordinal_position` ([UInt64](../../sql-reference/data-types/int-uint.md)) — порядковый номер столбца в таблице (нумерация начинается с 1). +- `column_default` ([String](../../sql-reference/data-types/string.md)) — выражение для значения по умолчанию или пустая строка. +- `is_nullable` ([UInt8](../../sql-reference/data-types/int-uint.md)) — флаг, показывающий является ли столбец типа `Nullable`. +- `data_type` ([String](../../sql-reference/data-types/string.md)) — тип столбца. +- `character_maximum_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Максимальная длина в байтах для двоичных данных, символьных данных или текстовых данных и изображений. Иначе возвращается значение `NULL`. +- `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Максимальная длина в байтах для двоичных данных, символьных данных или текстовых данных и изображений. Иначе возвращается значение `NULL`. +- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Точность приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. Иначе возвращается значение `NULL`. +- `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Основание системы счисления точности приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. Иначе возвращается значение `NULL`. +- `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Масштаб приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. Иначе возвращается значение `NULL`. +- `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Код подтипа для типов данных интервала `DateTime` и `ISO`. Для других типов данных возвращается значение `NULL`. +- `character_set_catalog` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. +- `character_set_schema` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. +- `character_set_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. +- `collation_catalog` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. +- `collation_schema` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. +- `collation_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. +- `domain_catalog` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. +- `domain_schema` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. +- `domain_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. + +**Пример** + +Запрос: + +``` sql +SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE (table_schema=currentDatabase() OR table_schema='') AND table_name NOT LIKE '%inner%' LIMIT 1 FORMAT Vertical; +``` + +Результат: + +``` text +Row 1: +────── +table_catalog: default +table_schema: default +table_name: describe_example +column_name: id +ordinal_position: 1 +column_default: +is_nullable: 0 +data_type: UInt64 +character_maximum_length: ᴺᵁᴸᴸ +character_octet_length: ᴺᵁᴸᴸ +numeric_precision: 64 +numeric_precision_radix: 2 +numeric_scale: 0 +datetime_precision: ᴺᵁᴸᴸ +character_set_catalog: ᴺᵁᴸᴸ +character_set_schema: ᴺᵁᴸᴸ +character_set_name: ᴺᵁᴸᴸ +collation_catalog: ᴺᵁᴸᴸ +collation_schema: ᴺᵁᴸᴸ +collation_name: ᴺᵁᴸᴸ +domain_catalog: ᴺᵁᴸᴸ +domain_schema: ᴺᵁᴸᴸ +domain_name: ᴺᵁᴸᴸ +``` + +## SCHEMATA {#schemata} + +Содержит столбцы, которые считываются из системной таблицы `system.databases`, и столбцы, которые не поддерживаются в ClickHouse или не имеют смысла (всегда имеют значение `NULL`), но должны быть по стандарту. + +Столбцы: + +- `catalog_name` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится таблица. +- `schema_name` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится схема. +- `schema_owner` ([String](../../sql-reference/data-types/string.md)) — имя владельца схемы, всегда `'default'`. +- `default_character_set_catalog` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. +- `default_character_set_schema` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. +- `default_character_set_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. +- `sql_path` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. + +**Пример** + +Запрос: + +``` sql +SELECT * FROM information_schema.schemata WHERE schema_name ILIKE 'information_schema' LIMIT 1 FORMAT Vertical; +``` + +Результат: + +``` text +Row 1: +────── +catalog_name: INFORMATION_SCHEMA +schema_name: INFORMATION_SCHEMA +schema_owner: default +default_character_set_catalog: ᴺᵁᴸᴸ +default_character_set_schema: ᴺᵁᴸᴸ +default_character_set_name: ᴺᵁᴸᴸ +sql_path: ᴺᵁᴸᴸ +``` + +## TABLES {#tables} + +Содержит столбцы, которые считываются из системной таблицы `system.tables`. + +Столбцы: + +- `table_catalog` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится таблица. +- `table_schema` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится схема. +- `table_name` ([String](../../sql-reference/data-types/string.md)) — имя таблицы. +- `table_type` ([Enum8](../../sql-reference/data-types/enum.md)) — тип таблицы. Возможные значения: + - `BASE TABLE` + - `VIEW` + - `FOREIGN TABLE` + - `LOCAL TEMPORARY` + - `SYSTEM VIEW` + +**Пример** + +Запрос: + +``` sql +SELECT * FROM INFORMATION_SCHEMA.TABLES WHERE (table_schema = currentDatabase() OR table_schema = '') AND table_name NOT LIKE '%inner%' LIMIT 1 FORMAT Vertical; +``` + +Результат: + +``` text +Row 1: +────── +table_catalog: default +table_schema: default +table_name: describe_example +table_type: BASE TABLE +``` + +## VIEWS {#views} + +Содержит столбцы, которые считываются из системной таблицы `system.tables`, если использован движок [View](../../engines/table-engines/special/view.md)). + +Столбцы: + +- `table_catalog` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится таблица. +- `table_schema` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится схема. +- `table_name` ([String](../../sql-reference/data-types/string.md)) — имя таблицы. +- `view_definition` ([String](../../sql-reference/data-types/string.md)) — `SELECT` запрос для представления. +- `check_option` ([String](../../sql-reference/data-types/string.md)) — `NONE`, нет проверки. +- `is_updatable` ([Enum8](../../sql-reference/data-types/enum.md)) — `NO`, представление не обновляется. +- `is_insertable_into` ([Enum8](../../sql-reference/data-types/enum.md)) — показывает является ли представление [материализованным](../../sql-reference/statements/create/view/#materialized). Возможные значения: + - `NO` — создано обычное представление. + - `YES` — создано материализованное представление. +- `is_trigger_updatable` ([Enum8](../../sql-reference/data-types/enum.md)) — `NO`, триггер не обновляется. +- `is_trigger_deletable` ([Enum8](../../sql-reference/data-types/enum.md)) — `NO`, триггер не удаляется. +- `is_trigger_insertable_into` ([Enum8](../../sql-reference/data-types/enum.md)) — `NO`, данные не вставляются в триггер. + +**Пример** + +Запрос: + +``` sql +CREATE VIEW v (n Nullable(Int32), f Float64) AS SELECT n, f FROM t; +CREATE MATERIALIZED VIEW mv ENGINE = Null AS SELECT * FROM system.one; +SELECT * FROM information_schema.views WHERE table_schema = currentDatabase() LIMIT 1 FORMAT Vertical; +``` + +Результат: + +``` text +Row 1: +────── +table_catalog: default +table_schema: default +table_name: mv +view_definition: SELECT * FROM system.one +check_option: NONE +is_updatable: NO +is_insertable_into: YES +is_trigger_updatable: NO +is_trigger_deletable: NO +is_trigger_insertable_into: NO +``` diff --git a/docs/ru/operations/system-tables/tables.md b/docs/ru/operations/system-tables/tables.md index 03ad174780f..bf47051442e 100644 --- a/docs/ru/operations/system-tables/tables.md +++ b/docs/ru/operations/system-tables/tables.md @@ -9,20 +9,37 @@ Столбцы: - `database` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится таблица. + - `name` ([String](../../sql-reference/data-types/string.md)) — имя таблицы. + - `engine` ([String](../../sql-reference/data-types/string.md)) — движок таблицы (без параметров). + - `is_temporary` ([UInt8](../../sql-reference/data-types/int-uint.md)) — флаг, указывающий на то, временная это таблица или нет. + - `data_path` ([String](../../sql-reference/data-types/string.md)) — путь к данным таблицы в файловой системе. + - `metadata_path` ([String](../../sql-reference/data-types/string.md)) — путь к табличным метаданным в файловой системе. + - `metadata_modification_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — время последней модификации табличных метаданных. + - `dependencies_database` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — зависимости базы данных. + - `dependencies_table` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — табличные зависимости (таблицы [MaterializedView](../../engines/table-engines/special/materializedview.md), созданные на базе текущей таблицы). + - `create_table_query` ([String](../../sql-reference/data-types/string.md)) — запрос, при помощи которого создавалась таблица. + - `engine_full` ([String](../../sql-reference/data-types/string.md)) — параметры табличного движка. + +- `as_select` ([String](../../sql-reference/data-types/string.md)) - `SELECT` запрос для представления. + - `partition_key` ([String](../../sql-reference/data-types/string.md)) — ключ партиционирования таблицы. + - `sorting_key` ([String](../../sql-reference/data-types/string.md)) — ключ сортировки таблицы. + - `primary_key` ([String](../../sql-reference/data-types/string.md)) - первичный ключ таблицы. + - `sampling_key` ([String](../../sql-reference/data-types/string.md)) — ключ сэмплирования таблицы. + - `storage_policy` ([String](../../sql-reference/data-types/string.md)) - политика хранения данных: - [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) @@ -41,6 +58,8 @@ - `comment` ([String](../../sql-reference/data-types/string.md)) — комментарий к таблице. +- `has_own_data` ([UInt8](../../sql-reference/data-types/int-uint.md)) — флаг, показывающий хранит ли таблица сама какие-то данные на диске или только обращается к какому-то другому источнику. + Таблица `system.tables` используется при выполнении запроса `SHOW TABLES`. **Пример** @@ -64,6 +83,7 @@ dependencies_database: [] dependencies_table: [] create_table_query: CREATE TABLE base.t1 (`n` UInt64) ENGINE = MergeTree ORDER BY n SETTINGS index_granularity = 8192 engine_full: MergeTree ORDER BY n SETTINGS index_granularity = 8192 +as_select: SELECT database AS table_catalog partition_key: sorting_key: n primary_key: n @@ -74,6 +94,7 @@ total_bytes: 99 lifetime_rows: ᴺᵁᴸᴸ lifetime_bytes: ᴺᵁᴸᴸ comment: +has_own_data: 0 Row 2: ────── @@ -89,6 +110,7 @@ dependencies_database: [] dependencies_table: [] create_table_query: CREATE TABLE default.`53r93yleapyears` (`id` Int8, `febdays` Int8) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192 engine_full: MergeTree ORDER BY id SETTINGS index_granularity = 8192 +as_select: SELECT name AS catalog_name partition_key: sorting_key: id primary_key: id @@ -99,4 +121,5 @@ total_bytes: 155 lifetime_rows: ᴺᵁᴸᴸ lifetime_bytes: ᴺᵁᴸᴸ comment: +has_own_data: 0 ``` From 46e41a10e847709374962e71a3d5eaad5702fb10 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Tue, 26 Oct 2021 22:16:24 +0300 Subject: [PATCH 121/396] Fix typo MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил опечатку. --- docs/en/operations/system-tables/information_schema.md | 2 +- docs/ru/operations/system-tables/information_schema.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/system-tables/information_schema.md b/docs/en/operations/system-tables/information_schema.md index 20926f512cb..dfdb9df5dfd 100644 --- a/docs/en/operations/system-tables/information_schema.md +++ b/docs/en/operations/system-tables/information_schema.md @@ -1,4 +1,4 @@ -# information_schema {#information-schema} +# INFORMATION_SCHEMA {#information-schema} `INFORMATION_SCHEMA` (`information_schema`) is a system database that contains views. Using these views, you can get information about the metadata of database objects. These views read data from the columns of the [system.columns](../../operations/system-tables/columns.md), [system.databases](../../operations/system-tables/databases.md) and [system.tables](../../operations/system-tables/tables.md) system tables. diff --git a/docs/ru/operations/system-tables/information_schema.md b/docs/ru/operations/system-tables/information_schema.md index dad7f99e2bd..1e8e69a002e 100644 --- a/docs/ru/operations/system-tables/information_schema.md +++ b/docs/ru/operations/system-tables/information_schema.md @@ -1,4 +1,4 @@ -# information_schema {#information-schema} +# INFORMATION_SCHEMA {#information-schema} `INFORMATION_SCHEMA` (`information_schema`) — это системная база данных, содержащая представления. Используя эти представления, вы можете получить информацию о метаданных объектов базы данных. Эти представления считывают данные из столбцов системных таблиц [system.columns](../../operations/system-tables/columns.md), [system.databases](../../operations/system-tables/databases.md) и [system.tables](../../operations/system-tables/tables.md). @@ -165,7 +165,7 @@ table_type: BASE TABLE ## VIEWS {#views} -Содержит столбцы, которые считываются из системной таблицы `system.tables`, если использован движок [View](../../engines/table-engines/special/view.md)). +Содержит столбцы, которые считываются из системной таблицы `system.tables`, если использован движок [View](../../engines/table-engines/special/view.md). Столбцы: From bc9ee5e33f284ab592abefea66b5961c5f14317f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 23 Oct 2021 22:28:14 +0300 Subject: [PATCH 122/396] Add tests for compatibility of LogFamily. --- ..._complex_structs_data_file_dumps.reference | 322 ++++++++++++++++++ ..._family_complex_structs_data_file_dumps.sh | 41 +++ ...02047_log_family_data_file_dumps.reference | 160 +++++++++ .../02047_log_family_data_file_dumps.sh | 41 +++ 4 files changed, 564 insertions(+) create mode 100644 tests/queries/0_stateless/02047_log_family_complex_structs_data_file_dumps.reference create mode 100755 tests/queries/0_stateless/02047_log_family_complex_structs_data_file_dumps.sh create mode 100644 tests/queries/0_stateless/02047_log_family_data_file_dumps.reference create mode 100755 tests/queries/0_stateless/02047_log_family_data_file_dumps.sh diff --git a/tests/queries/0_stateless/02047_log_family_complex_structs_data_file_dumps.reference b/tests/queries/0_stateless/02047_log_family_complex_structs_data_file_dumps.reference new file mode 100644 index 00000000000..312809828db --- /dev/null +++ b/tests/queries/0_stateless/02047_log_family_complex_structs_data_file_dumps.reference @@ -0,0 +1,322 @@ +Log: +empty: +1 element: +[[3,5,7]] ['a1','a2'] [4.1,4.2] +__marks.mrk size=96 +00000000 01 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00 |................| +* +00000060 +sizes.json size=226 +00000000 7b 22 63 6c 69 63 6b 68 6f 75 73 65 22 3a 7b 22 |{"clickhouse":{"| +00000010 5f 5f 6d 61 72 6b 73 25 32 45 6d 72 6b 22 3a 7b |__marks%2Emrk":{| +00000020 22 73 69 7a 65 22 3a 22 39 36 22 7d 2c 22 78 25 |"size":"96"},"x%| +00000030 32 45 62 69 6e 22 3a 7b 22 73 69 7a 65 22 3a 22 |2Ebin":{"size":"| +00000040 33 38 22 7d 2c 22 78 25 32 45 73 69 7a 65 30 25 |38"},"x%2Esize0%| +00000050 32 45 62 69 6e 22 3a 7b 22 73 69 7a 65 22 3a 22 |2Ebin":{"size":"| +00000060 33 34 22 7d 2c 22 78 25 32 45 73 69 7a 65 31 25 |34"},"x%2Esize1%| +00000070 32 45 62 69 6e 22 3a 7b 22 73 69 7a 65 22 3a 22 |2Ebin":{"size":"| +00000080 33 34 22 7d 2c 22 79 25 32 35 32 45 77 25 32 45 |34"},"y%252Ew%2E| +00000090 62 69 6e 22 3a 7b 22 73 69 7a 65 22 3a 22 33 34 |bin":{"size":"34| +000000a0 22 7d 2c 22 79 25 32 35 32 45 7a 25 32 45 62 69 |"},"y%252Ez%2Ebi| +000000b0 6e 22 3a 7b 22 73 69 7a 65 22 3a 22 33 32 22 7d |n":{"size":"32"}| +000000c0 2c 22 79 25 32 45 73 69 7a 65 30 25 32 45 62 69 |,"y%2Esize0%2Ebi| +000000d0 6e 22 3a 7b 22 73 69 7a 65 22 3a 22 33 34 22 7d |n":{"size":"34"}| +000000e0 7d 7d |}}| +000000e2 +x.bin size=38 +00000000 91 54 4b 94 65 b3 4b ec 03 da 1a 40 e6 25 75 67 |.TK.e.K....@.%ug| +00000010 82 16 00 00 00 0c 00 00 00 c0 03 00 00 00 05 00 |................| +00000020 00 00 07 00 00 00 |......| +00000026 +x.size0.bin size=34 +00000000 48 54 99 92 92 db 6a 74 c5 da 23 ed 7e b0 4b 3e |HT....jt..#.~.K>| +00000010 82 12 00 00 00 08 00 00 00 80 01 00 00 00 00 00 |................| +00000020 00 00 |..| +00000022 +x.size1.bin size=34 +00000000 4f 48 a7 2c 99 02 fb 38 ec 48 b5 b1 ed 95 03 a5 |OH.,...8.H......| +00000010 82 12 00 00 00 08 00 00 00 80 03 00 00 00 00 00 |................| +00000020 00 00 |..| +00000022 +y%2Ew.bin size=34 +00000000 a8 87 9a f0 64 ca 12 e6 ca ac 68 b9 81 be ec c8 |....d.....h.....| +00000010 82 12 00 00 00 08 00 00 00 80 33 33 83 40 66 66 |..........33.@ff| +00000020 86 40 |.@| +00000022 +y%2Ez.bin size=32 +00000000 84 ac b0 60 39 d3 a5 77 24 cc 33 07 db 27 f1 b2 |...`9..w$.3..'..| +00000010 82 10 00 00 00 06 00 00 00 60 02 61 31 02 61 32 |.........`.a1.a2| +00000020 +y.size0.bin size=34 +00000000 5e 56 3c 70 ce d8 15 fe 0a 1d ef 3b 4a 1c 99 4f |^V| +00000010 82 12 00 00 00 08 00 00 00 80 01 00 00 00 00 00 |................| +00000020 00 00 1a 9d 14 3f 60 b2 ab 75 b2 00 12 3a 64 37 |.....?`..u...:d7| +00000030 86 5b 82 16 00 00 00 10 00 00 00 13 00 01 00 80 |.[..............| +00000040 01 00 00 00 00 00 00 00 |........| +00000048 +x.size1.bin size=68 +00000000 4f 48 a7 2c 99 02 fb 38 ec 48 b5 b1 ed 95 03 a5 |OH.,...8.H......| +00000010 82 12 00 00 00 08 00 00 00 80 03 00 00 00 00 00 |................| +00000020 00 00 48 54 99 92 92 db 6a 74 c5 da 23 ed 7e b0 |..HT....jt..#.~.| +00000030 4b 3e 82 12 00 00 00 08 00 00 00 80 01 00 00 00 |K>..............| +00000040 00 00 00 00 |....| +00000044 +y%2Ew.bin size=64 +00000000 a8 87 9a f0 64 ca 12 e6 ca ac 68 b9 81 be ec c8 |....d.....h.....| +00000010 82 12 00 00 00 08 00 00 00 80 33 33 83 40 66 66 |..........33.@ff| +00000020 86 40 4d 8f 35 86 dc 39 5f 7c 7e 38 f1 9b 5e d7 |.@M.5..9_|~8..^.| +00000030 ba 6e 82 0e 00 00 00 04 00 00 00 40 9a 99 89 40 |.n.........@...@| +00000040 +y%2Ez.bin size=61 +00000000 84 ac b0 60 39 d3 a5 77 24 cc 33 07 db 27 f1 b2 |...`9..w$.3..'..| +00000010 82 10 00 00 00 06 00 00 00 60 02 61 31 02 61 32 |.........`.a1.a2| +00000020 09 39 35 e0 9b 95 cb c9 15 41 27 69 d9 d4 cc 89 |.95......A'i....| +00000030 82 0d 00 00 00 03 00 00 00 30 02 61 33 |.........0.a3| +0000003d +y.size0.bin size=72 +00000000 5e 56 3c 70 ce d8 15 fe 0a 1d ef 3b 4a 1c 99 4f |^V| +00000010 82 12 00 00 00 08 00 00 00 80 01 00 00 00 00 00 |................| +00000020 00 00 |..| +00000022 +x.size1.bin size=34 +00000000 4f 48 a7 2c 99 02 fb 38 ec 48 b5 b1 ed 95 03 a5 |OH.,...8.H......| +00000010 82 12 00 00 00 08 00 00 00 80 03 00 00 00 00 00 |................| +00000020 00 00 |..| +00000022 +y%2Ew.bin size=34 +00000000 a8 87 9a f0 64 ca 12 e6 ca ac 68 b9 81 be ec c8 |....d.....h.....| +00000010 82 12 00 00 00 08 00 00 00 80 33 33 83 40 66 66 |..........33.@ff| +00000020 86 40 |.@| +00000022 +y%2Ez.bin size=32 +00000000 84 ac b0 60 39 d3 a5 77 24 cc 33 07 db 27 f1 b2 |...`9..w$.3..'..| +00000010 82 10 00 00 00 06 00 00 00 60 02 61 31 02 61 32 |.........`.a1.a2| +00000020 +y.size0.bin size=34 +00000000 5e 56 3c 70 ce d8 15 fe 0a 1d ef 3b 4a 1c 99 4f |^V| +00000010 82 12 00 00 00 08 00 00 00 80 01 00 00 00 00 00 |................| +00000020 00 00 1a 9d 14 3f 60 b2 ab 75 b2 00 12 3a 64 37 |.....?`..u...:d7| +00000030 86 5b 82 16 00 00 00 10 00 00 00 13 00 01 00 80 |.[..............| +00000040 01 00 00 00 00 00 00 00 |........| +00000048 +x.size1.bin size=68 +00000000 4f 48 a7 2c 99 02 fb 38 ec 48 b5 b1 ed 95 03 a5 |OH.,...8.H......| +00000010 82 12 00 00 00 08 00 00 00 80 03 00 00 00 00 00 |................| +00000020 00 00 48 54 99 92 92 db 6a 74 c5 da 23 ed 7e b0 |..HT....jt..#.~.| +00000030 4b 3e 82 12 00 00 00 08 00 00 00 80 01 00 00 00 |K>..............| +00000040 00 00 00 00 |....| +00000044 +y%2Ew.bin size=64 +00000000 a8 87 9a f0 64 ca 12 e6 ca ac 68 b9 81 be ec c8 |....d.....h.....| +00000010 82 12 00 00 00 08 00 00 00 80 33 33 83 40 66 66 |..........33.@ff| +00000020 86 40 4d 8f 35 86 dc 39 5f 7c 7e 38 f1 9b 5e d7 |.@M.5..9_|~8..^.| +00000030 ba 6e 82 0e 00 00 00 04 00 00 00 40 9a 99 89 40 |.n.........@...@| +00000040 +y%2Ez.bin size=61 +00000000 84 ac b0 60 39 d3 a5 77 24 cc 33 07 db 27 f1 b2 |...`9..w$.3..'..| +00000010 82 10 00 00 00 06 00 00 00 60 02 61 31 02 61 32 |.........`.a1.a2| +00000020 09 39 35 e0 9b 95 cb c9 15 41 27 69 d9 d4 cc 89 |.95......A'i....| +00000030 82 0d 00 00 00 03 00 00 00 30 02 61 33 |.........0.a3| +0000003d +y.size0.bin size=72 +00000000 5e 56 3c 70 ce d8 15 fe 0a 1d ef 3b 4a 1c 99 4f |^V/dev/null + + echo "1 element:" + $CLICKHOUSE_CLIENT --query="INSERT INTO tbl VALUES ([[3,5,7]], ['a1', 'a2'], [4.1, 4.2])" + $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") + echo "$name size=$file_size" + hexdump -C $file_path + done + + echo "3 elements:" + $CLICKHOUSE_CLIENT --query="INSERT INTO tbl VALUES ([], [], []), ([[10]], ['a3'], [4.3])" + $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") + echo "$name size=$file_size" + hexdump -C $file_path + done + + echo +done diff --git a/tests/queries/0_stateless/02047_log_family_data_file_dumps.reference b/tests/queries/0_stateless/02047_log_family_data_file_dumps.reference new file mode 100644 index 00000000000..bdb91497eb0 --- /dev/null +++ b/tests/queries/0_stateless/02047_log_family_data_file_dumps.reference @@ -0,0 +1,160 @@ +Log: +empty: +1 element: +1 a +__marks.mrk size=32 +00000000 01 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00 |................| +* +00000020 +sizes.json size=94 +00000000 7b 22 63 6c 69 63 6b 68 6f 75 73 65 22 3a 7b 22 |{"clickhouse":{"| +00000010 5f 5f 6d 61 72 6b 73 25 32 45 6d 72 6b 22 3a 7b |__marks%2Emrk":{| +00000020 22 73 69 7a 65 22 3a 22 33 32 22 7d 2c 22 78 25 |"size":"32"},"x%| +00000030 32 45 62 69 6e 22 3a 7b 22 73 69 7a 65 22 3a 22 |2Ebin":{"size":"| +00000040 33 30 22 7d 2c 22 79 25 32 45 62 69 6e 22 3a 7b |30"},"y%2Ebin":{| +00000050 22 73 69 7a 65 22 3a 22 32 38 22 7d 7d 7d |"size":"28"}}}| +0000005e +x.bin size=30 +00000000 39 9f 28 b3 25 cf c7 13 95 53 21 42 ef ac ef ec |9.(.%....S!B....| +00000010 82 0e 00 00 00 04 00 00 00 40 01 00 00 00 |.........@....| +0000001e +y.bin size=28 +00000000 7d 24 2d b7 e2 da 37 8a db fd 99 47 48 4b 9e c1 |}$-...7....GHK..| +00000010 82 0c 00 00 00 02 00 00 00 20 01 61 |......... .a| +0000001c +3 elements: +1 a +22 bc +333 def +__marks.mrk size=64 +00000000 01 00 00 00 00 00 00 00 00 00 00 00 00 00 00 00 |................| +* +00000020 03 00 00 00 00 00 00 00 1e 00 00 00 00 00 00 00 |................| +00000030 03 00 00 00 00 00 00 00 1c 00 00 00 00 00 00 00 |................| +00000040 +sizes.json size=94 +00000000 7b 22 63 6c 69 63 6b 68 6f 75 73 65 22 3a 7b 22 |{"clickhouse":{"| +00000010 5f 5f 6d 61 72 6b 73 25 32 45 6d 72 6b 22 3a 7b |__marks%2Emrk":{| +00000020 22 73 69 7a 65 22 3a 22 36 34 22 7d 2c 22 78 25 |"size":"64"},"x%| +00000030 32 45 62 69 6e 22 3a 7b 22 73 69 7a 65 22 3a 22 |2Ebin":{"size":"| +00000040 36 34 22 7d 2c 22 79 25 32 45 62 69 6e 22 3a 7b |64"},"y%2Ebin":{| +00000050 22 73 69 7a 65 22 3a 22 36 31 22 7d 7d 7d |"size":"61"}}}| +0000005e +x.bin size=64 +00000000 39 9f 28 b3 25 cf c7 13 95 53 21 42 ef ac ef ec |9.(.%....S!B....| +00000010 82 0e 00 00 00 04 00 00 00 40 01 00 00 00 f3 5b |.........@.....[| +00000020 0d 60 9c a7 ec c2 76 d7 e7 c3 a5 1e d9 22 82 12 |.`....v......"..| +00000030 00 00 00 08 00 00 00 80 16 00 00 00 4d 01 00 00 |............M...| +00000040 +y.bin size=61 +00000000 7d 24 2d b7 e2 da 37 8a db fd 99 47 48 4b 9e c1 |}$-...7....GHK..| +00000010 82 0c 00 00 00 02 00 00 00 20 01 61 4a 41 a1 bf |......... .aJA..| +00000020 6e 15 00 05 d3 41 ed 9b 02 46 62 1b 82 11 00 00 |n....A...Fb.....| +00000030 00 07 00 00 00 70 02 62 63 03 64 65 66 |.....p.bc.def| +0000003d + +TinyLog: +empty: +1 element: +1 a +sizes.json size=64 +00000000 7b 22 63 6c 69 63 6b 68 6f 75 73 65 22 3a 7b 22 |{"clickhouse":{"| +00000010 78 25 32 45 62 69 6e 22 3a 7b 22 73 69 7a 65 22 |x%2Ebin":{"size"| +00000020 3a 22 33 30 22 7d 2c 22 79 25 32 45 62 69 6e 22 |:"30"},"y%2Ebin"| +00000030 3a 7b 22 73 69 7a 65 22 3a 22 32 38 22 7d 7d 7d |:{"size":"28"}}}| +00000040 +x.bin size=30 +00000000 39 9f 28 b3 25 cf c7 13 95 53 21 42 ef ac ef ec |9.(.%....S!B....| +00000010 82 0e 00 00 00 04 00 00 00 40 01 00 00 00 |.........@....| +0000001e +y.bin size=28 +00000000 7d 24 2d b7 e2 da 37 8a db fd 99 47 48 4b 9e c1 |}$-...7....GHK..| +00000010 82 0c 00 00 00 02 00 00 00 20 01 61 |......... .a| +0000001c +3 elements: +1 a +22 bc +333 def +sizes.json size=64 +00000000 7b 22 63 6c 69 63 6b 68 6f 75 73 65 22 3a 7b 22 |{"clickhouse":{"| +00000010 78 25 32 45 62 69 6e 22 3a 7b 22 73 69 7a 65 22 |x%2Ebin":{"size"| +00000020 3a 22 36 34 22 7d 2c 22 79 25 32 45 62 69 6e 22 |:"64"},"y%2Ebin"| +00000030 3a 7b 22 73 69 7a 65 22 3a 22 36 31 22 7d 7d 7d |:{"size":"61"}}}| +00000040 +x.bin size=64 +00000000 39 9f 28 b3 25 cf c7 13 95 53 21 42 ef ac ef ec |9.(.%....S!B....| +00000010 82 0e 00 00 00 04 00 00 00 40 01 00 00 00 f3 5b |.........@.....[| +00000020 0d 60 9c a7 ec c2 76 d7 e7 c3 a5 1e d9 22 82 12 |.`....v......"..| +00000030 00 00 00 08 00 00 00 80 16 00 00 00 4d 01 00 00 |............M...| +00000040 +y.bin size=61 +00000000 7d 24 2d b7 e2 da 37 8a db fd 99 47 48 4b 9e c1 |}$-...7....GHK..| +00000010 82 0c 00 00 00 02 00 00 00 20 01 61 4a 41 a1 bf |......... .aJA..| +00000020 6e 15 00 05 d3 41 ed 9b 02 46 62 1b 82 11 00 00 |n....A...Fb.....| +00000030 00 07 00 00 00 70 02 62 63 03 64 65 66 |.....p.bc.def| +0000003d + +StripeLog: +empty: +1 element: +1 a +data.bin size=104 +00000000 0c d3 31 cc 9f f3 94 97 56 a2 ee 76 f7 4c 35 0f |..1.....V..v.L5.| +00000010 82 0c 00 00 00 02 00 00 00 20 02 01 d3 b5 6c 48 |......... ....lH| +00000020 05 89 3b 01 ad 3a 99 3a 40 10 f0 fa 82 17 00 00 |..;..:.:@.......| +00000030 00 0d 00 00 00 d0 01 78 06 55 49 6e 74 33 32 01 |.......x.UInt32.| +00000040 00 00 00 07 12 7f b8 90 bb cb c1 5b 93 d1 61 7e |...........[..a~| +00000050 27 86 3e 82 15 00 00 00 0b 00 00 00 b0 01 79 06 |'.>...........y.| +00000060 53 74 72 69 6e 67 01 61 |String.a| +00000068 +index.mrk size=60 +00000000 e0 18 ce 21 0a cc 9f ac d8 51 d4 27 00 4d 7e 75 |...!.....Q.'.M~u| +00000010 82 2c 00 00 00 34 00 00 00 da 02 01 01 78 06 55 |.,...4.......x.U| +00000020 49 6e 74 33 32 1c 00 01 00 a6 01 79 06 53 74 72 |Int32......y.Str| +00000030 69 6e 67 43 18 00 50 00 00 00 00 00 |ingC..P.....| +0000003c +sizes.json size=72 +00000000 7b 22 63 6c 69 63 6b 68 6f 75 73 65 22 3a 7b 22 |{"clickhouse":{"| +00000010 64 61 74 61 25 32 45 62 69 6e 22 3a 7b 22 73 69 |data%2Ebin":{"si| +00000020 7a 65 22 3a 22 31 30 34 22 7d 2c 22 69 6e 64 65 |ze":"104"},"inde| +00000030 78 25 32 45 6d 72 6b 22 3a 7b 22 73 69 7a 65 22 |x%2Emrk":{"size"| +00000040 3a 22 36 30 22 7d 7d 7d |:"60"}}}| +00000048 +3 elements: +1 a +22 bc +333 def +data.bin size=219 +00000000 0c d3 31 cc 9f f3 94 97 56 a2 ee 76 f7 4c 35 0f |..1.....V..v.L5.| +00000010 82 0c 00 00 00 02 00 00 00 20 02 01 d3 b5 6c 48 |......... ....lH| +00000020 05 89 3b 01 ad 3a 99 3a 40 10 f0 fa 82 17 00 00 |..;..:.:@.......| +00000030 00 0d 00 00 00 d0 01 78 06 55 49 6e 74 33 32 01 |.......x.UInt32.| +00000040 00 00 00 07 12 7f b8 90 bb cb c1 5b 93 d1 61 7e |...........[..a~| +00000050 27 86 3e 82 15 00 00 00 0b 00 00 00 b0 01 79 06 |'.>...........y.| +00000060 53 74 72 69 6e 67 01 61 55 84 52 9f 32 10 4d 9e |String.aU.R.2.M.| +00000070 eb 58 e8 2a 3a 8c e8 df 82 0c 00 00 00 02 00 00 |.X.*:...........| +00000080 00 20 02 02 5f c2 08 eb 23 e5 a3 7a e2 22 e7 fc |. .._...#..z."..| +00000090 23 57 79 fd 82 1c 00 00 00 11 00 00 00 f0 02 01 |#Wy.............| +000000a0 78 06 55 49 6e 74 33 32 16 00 00 00 4d 01 00 00 |x.UInt32....M...| +000000b0 87 a2 af 21 6c 93 79 77 88 8c 5f ae e9 41 df 5a |...!l.yw.._..A.Z| +000000c0 82 1b 00 00 00 10 00 00 00 f0 01 01 79 06 53 74 |............y.St| +000000d0 72 69 6e 67 02 62 63 03 64 65 66 |ring.bc.def| +000000db +index.mrk size=120 +00000000 e0 18 ce 21 0a cc 9f ac d8 51 d4 27 00 4d 7e 75 |...!.....Q.'.M~u| +00000010 82 2c 00 00 00 34 00 00 00 da 02 01 01 78 06 55 |.,...4.......x.U| +00000020 49 6e 74 33 32 1c 00 01 00 a6 01 79 06 53 74 72 |Int32......y.Str| +00000030 69 6e 67 43 18 00 50 00 00 00 00 00 82 27 58 46 |ingC..P......'XF| +00000040 ad 45 bc 2f 66 02 78 84 e1 42 dd 03 82 2c 00 00 |.E./f.x..B...,..| +00000050 00 34 00 00 00 da 02 02 01 78 06 55 49 6e 74 33 |.4.......x.UInt3| +00000060 32 84 00 01 00 a6 01 79 06 53 74 72 69 6e 67 b0 |2......y.String.| +00000070 18 00 50 00 00 00 00 00 |..P.....| +00000078 +sizes.json size=73 +00000000 7b 22 63 6c 69 63 6b 68 6f 75 73 65 22 3a 7b 22 |{"clickhouse":{"| +00000010 64 61 74 61 25 32 45 62 69 6e 22 3a 7b 22 73 69 |data%2Ebin":{"si| +00000020 7a 65 22 3a 22 32 31 39 22 7d 2c 22 69 6e 64 65 |ze":"219"},"inde| +00000030 78 25 32 45 6d 72 6b 22 3a 7b 22 73 69 7a 65 22 |x%2Emrk":{"size"| +00000040 3a 22 31 32 30 22 7d 7d 7d |:"120"}}}| +00000049 + diff --git a/tests/queries/0_stateless/02047_log_family_data_file_dumps.sh b/tests/queries/0_stateless/02047_log_family_data_file_dumps.sh new file mode 100755 index 00000000000..1edfd200fab --- /dev/null +++ b/tests/queries/0_stateless/02047_log_family_data_file_dumps.sh @@ -0,0 +1,41 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +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" + 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") + echo "$name size=$file_size" + hexdump -C $file_path + 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") + echo "$name size=$file_size" + hexdump -C $file_path + done + + echo +done From cebdee890597791deca691f8e3352064c2183d87 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 09:14:10 +0800 Subject: [PATCH 123/396] Modify the content --- docs/zh/operations/system-tables/crash-log.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/operations/system-tables/crash-log.md b/docs/zh/operations/system-tables/crash-log.md index 54ba9c570bd..d8752d74c11 100644 --- a/docs/zh/operations/system-tables/crash-log.md +++ b/docs/zh/operations/system-tables/crash-log.md @@ -16,7 +16,7 @@ - `revision` ([UInt32](../../sql-reference/data-types/int-uint.md)) — ClickHouse 服务器订正版本. - `build_id` ([String](../../sql-reference/data-types/string.md)) — 编译器生成的 BuildID. -**Example** +**案例** 查询: From 6a633d86cc906357e782380fa2b9f45e5e77d2b0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 09:24:52 +0800 Subject: [PATCH 124/396] Delete current-roles.md --- docs/zh/operations/system-tables/current-roles.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/current-roles.md diff --git a/docs/zh/operations/system-tables/current-roles.md b/docs/zh/operations/system-tables/current-roles.md deleted file mode 120000 index b9f1f9d6e9d..00000000000 --- a/docs/zh/operations/system-tables/current-roles.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/current-roles.md \ No newline at end of file From 23640513d7d58462031e75364d07997e3dd2b041 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 09:32:25 +0800 Subject: [PATCH 125/396] Create current-roles.md --- docs/zh/operations/system-tables/current-roles.md | 11 +++++++++++ 1 file changed, 11 insertions(+) create mode 100644 docs/zh/operations/system-tables/current-roles.md diff --git a/docs/zh/operations/system-tables/current-roles.md b/docs/zh/operations/system-tables/current-roles.md new file mode 100644 index 00000000000..8b9ddec2e0b --- /dev/null +++ b/docs/zh/operations/system-tables/current-roles.md @@ -0,0 +1,11 @@ +# system.current_roles {#system_tables-current_roles} + +包含当前用户的激活角色. `SET ROLE` 修改该表的内容. + +列信息: + + - `role_name` ([String](../../sql-reference/data-types/string.md))) — 角色名称. + - `with_admin_option` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 显示 `current_role` 是否是具有 `ADMIN OPTION` 权限的角色的标志. + - `is_default` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 显示 `current_role` 是否为默认角色的标志. + + [原始文章](https://clickhouse.com/docs/en/operations/system-tables/current-roles) From 492ba037ec2dd48ba1e877aef80da61542b8c8b4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 09:33:01 +0800 Subject: [PATCH 126/396] Update crash-log.md --- docs/zh/operations/system-tables/crash-log.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/operations/system-tables/crash-log.md b/docs/zh/operations/system-tables/crash-log.md index d8752d74c11..bd5c97937de 100644 --- a/docs/zh/operations/system-tables/crash-log.md +++ b/docs/zh/operations/system-tables/crash-log.md @@ -45,4 +45,4 @@ build_id: **另请参阅** - [trace_log](../../operations/system-tables/trace_log.md) 系统表 -[Original article](https://clickhouse.com/docs/en/operations/system-tables/crash-log) +[原始文章](https://clickhouse.com/docs/en/operations/system-tables/crash-log) From 386b097009a48d142500ee62746028b29e1e3d3b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 09:33:53 +0800 Subject: [PATCH 127/396] Delete data_skipping_indices.md --- docs/zh/operations/system-tables/data_skipping_indices.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/data_skipping_indices.md diff --git a/docs/zh/operations/system-tables/data_skipping_indices.md b/docs/zh/operations/system-tables/data_skipping_indices.md deleted file mode 120000 index 2820987bf69..00000000000 --- a/docs/zh/operations/system-tables/data_skipping_indices.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/data_skipping_indices.md \ No newline at end of file From c9e5bdf436f4b1400246b5cecf11293c52b768d5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 09:38:59 +0800 Subject: [PATCH 128/396] Create data_skipping_indices.md --- .../system-tables/data_skipping_indices.md | 47 +++++++++++++++++++ 1 file changed, 47 insertions(+) create mode 100644 docs/zh/operations/system-tables/data_skipping_indices.md diff --git a/docs/zh/operations/system-tables/data_skipping_indices.md b/docs/zh/operations/system-tables/data_skipping_indices.md new file mode 100644 index 00000000000..3278a7f830f --- /dev/null +++ b/docs/zh/operations/system-tables/data_skipping_indices.md @@ -0,0 +1,47 @@ +# system.data_skipping_indices {#system-data-skipping-indices} + +包含有关所有表中现有数据跳过索引的信息. + +列信息: + +- `database` ([String](../../sql-reference/data-types/string.md)) — 数据库名称. +- `table` ([String](../../sql-reference/data-types/string.md)) — 数据表名称. +- `name` ([String](../../sql-reference/data-types/string.md)) — 索引名称. +- `type` ([String](../../sql-reference/data-types/string.md)) — 索引类型. +- `expr` ([String](../../sql-reference/data-types/string.md)) — 索引计算表达式. +- `granularity` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 块中颗粒的数量. +- `data_compressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 压缩数据的大小, 以字节为单位. +- `data_uncompressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 解压缩数据的大小, 以字节为单位. +- `marks_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 标记的大小, 以字节为单位. + +**示例** + +```sql +SELECT * FROM system.data_skipping_indices LIMIT 2 FORMAT Vertical; +``` + +```text +Row 1: +────── +database: default +table: user_actions +name: clicks_idx +type: minmax +expr: clicks +granularity: 1 +data_compressed_bytes: 58 +data_uncompressed_bytes: 6 +marks: 48 + +Row 2: +────── +database: default +table: users +name: contacts_null_idx +type: minmax +expr: assumeNotNull(contacts_null) +granularity: 1 +data_compressed_bytes: 58 +data_uncompressed_bytes: 6 +marks: 48 +``` From ba86f0e3f499ad276545e16672fd195f60728e5a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 09:40:34 +0800 Subject: [PATCH 129/396] Delete distributed_ddl_queue.md --- docs/zh/operations/system-tables/distributed_ddl_queue.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/distributed_ddl_queue.md diff --git a/docs/zh/operations/system-tables/distributed_ddl_queue.md b/docs/zh/operations/system-tables/distributed_ddl_queue.md deleted file mode 120000 index a9cadc74af2..00000000000 --- a/docs/zh/operations/system-tables/distributed_ddl_queue.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/distributed_ddl_queue.md \ No newline at end of file From 76ad1e777b221d5a2dcd0f351cf83dc81bccf876 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 09:48:35 +0800 Subject: [PATCH 130/396] Create distributed_ddl_queue.md --- .../system-tables/distributed_ddl_queue.md | 64 +++++++++++++++++++ 1 file changed, 64 insertions(+) create mode 100644 docs/zh/operations/system-tables/distributed_ddl_queue.md diff --git a/docs/zh/operations/system-tables/distributed_ddl_queue.md b/docs/zh/operations/system-tables/distributed_ddl_queue.md new file mode 100644 index 00000000000..5237673d5d0 --- /dev/null +++ b/docs/zh/operations/system-tables/distributed_ddl_queue.md @@ -0,0 +1,64 @@ +# system.distributed_ddl_queue {#system_tables-distributed_ddl_queue} + +包含有关在集群上执行的[分布式ddl查询(集群环境)](../../sql-reference/distributed-ddl.md)的信息. + +列信息: + +- `entry` ([String](../../sql-reference/data-types/string.md)) — 查询ID. +- `host_name` ([String](../../sql-reference/data-types/string.md)) — 主机名称. +- `host_address` ([String](../../sql-reference/data-types/string.md)) — 主机名解析到的IP地址. +- `port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — 主机端口. +- `status` ([Enum8](../../sql-reference/data-types/enum.md)) — 查询状态. +- `cluster` ([String](../../sql-reference/data-types/string.md)) — 群集名称. +- `query` ([String](../../sql-reference/data-types/string.md)) — 执行查询. +- `initiator` ([String](../../sql-reference/data-types/string.md)) — 执行查询的节点. +- `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — 查询开始时间. +- `query_finish_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — 查询结束时间. +- `query_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 查询执行时间(毫秒). +- `exception_code` ([Enum8](../../sql-reference/data-types/enum.md)) — 来自于[ZooKeeper](../../operations/tips.md#zookeeper)的异常代码. + +**示例** + +``` sql +SELECT * +FROM system.distributed_ddl_queue +WHERE cluster = 'test_cluster' +LIMIT 2 +FORMAT Vertical + +Query id: f544e72a-6641-43f1-836b-24baa1c9632a + +Row 1: +────── +entry: query-0000000000 +host_name: clickhouse01 +host_address: 172.23.0.11 +port: 9000 +status: Finished +cluster: test_cluster +query: CREATE DATABASE test_db UUID '4a82697e-c85e-4e5b-a01e-a36f2a758456' ON CLUSTER test_cluster +initiator: clickhouse01:9000 +query_start_time: 2020-12-30 13:07:51 +query_finish_time: 2020-12-30 13:07:51 +query_duration_ms: 6 +exception_code: ZOK + +Row 2: +────── +entry: query-0000000000 +host_name: clickhouse02 +host_address: 172.23.0.12 +port: 9000 +status: Finished +cluster: test_cluster +query: CREATE DATABASE test_db UUID '4a82697e-c85e-4e5b-a01e-a36f2a758456' ON CLUSTER test_cluster +initiator: clickhouse01:9000 +query_start_time: 2020-12-30 13:07:51 +query_finish_time: 2020-12-30 13:07:51 +query_duration_ms: 6 +exception_code: ZOK + +2 rows in set. Elapsed: 0.025 sec. +``` + +[原始文章](https://clickhouse.com/docs/en/operations/system_tables/distributed_ddl_queuedistributed_ddl_queue.md) From 64434fb10b2cc10bf1dbc0d66cb204d3aafb4eb1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 09:49:18 +0800 Subject: [PATCH 131/396] Delete distribution_queue.md --- docs/zh/operations/system-tables/distribution_queue.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/distribution_queue.md diff --git a/docs/zh/operations/system-tables/distribution_queue.md b/docs/zh/operations/system-tables/distribution_queue.md deleted file mode 120000 index 3831b85000c..00000000000 --- a/docs/zh/operations/system-tables/distribution_queue.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/distribution_queue.md \ No newline at end of file From 7ca924b2f61a968cef01d56252e6d77667551aaf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 09:54:20 +0800 Subject: [PATCH 132/396] Create distribution_queue.md --- .../system-tables/distribution_queue.md | 50 +++++++++++++++++++ 1 file changed, 50 insertions(+) create mode 100644 docs/zh/operations/system-tables/distribution_queue.md diff --git a/docs/zh/operations/system-tables/distribution_queue.md b/docs/zh/operations/system-tables/distribution_queue.md new file mode 100644 index 00000000000..2317e0677a7 --- /dev/null +++ b/docs/zh/operations/system-tables/distribution_queue.md @@ -0,0 +1,50 @@ +# system.distribution_queue {#system_tables-distribution_queue} + +包含关于队列中要发送到分片的本地文件的信息. 这些本地文件包含通过以异步模式将新数据插入到Distributed表中而创建的新部分. + +列信息: + +- `database` ([String](../../sql-reference/data-types/string.md)) — 数据库名称. + +- `table` ([String](../../sql-reference/data-types/string.md)) — 表名称. + +- `data_path` ([String](../../sql-reference/data-types/string.md)) — 存放本地文件的文件夹的路径. + +- `is_blocked` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag表示是否阻止向服务器发送本地文件. + +- `error_count` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 错误总数. + +- `data_files` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 文件夹中的本地文件数. + +- `data_compressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 本地文件中压缩数据的大小, 以字节为单位. + +- `broken_data_files` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 被标记为损坏的文件数量(由于错误). + +- `broken_data_compressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 破碎文件中压缩数据的大小, 以字节为单位. + +- `last_exception` ([String](../../sql-reference/data-types/string.md)) — 关于最近发生的错误的文本信息(如果有的话). + +**示例** + +``` sql +SELECT * FROM system.distribution_queue LIMIT 1 FORMAT Vertical; +``` + +``` text +Row 1: +────── +database: default +table: dist +data_path: ./store/268/268bc070-3aad-4b1a-9cf2-4987580161af/default@127%2E0%2E0%2E2:9000/ +is_blocked: 1 +error_count: 0 +data_files: 1 +data_compressed_bytes: 499 +last_exception: +``` + +**另请参阅** + +- [分布式表引擎](../../engines/table-engines/special/distributed.md) + +[原始文章](https://clickhouse.com/docs/en/operations/system_tables/distribution_queue) From b7f63332191e098118fee3166e7b709983c38157 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 09:55:32 +0800 Subject: [PATCH 133/396] Delete enabled-roles.md --- docs/zh/operations/system-tables/enabled-roles.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/enabled-roles.md diff --git a/docs/zh/operations/system-tables/enabled-roles.md b/docs/zh/operations/system-tables/enabled-roles.md deleted file mode 120000 index 04ffee25343..00000000000 --- a/docs/zh/operations/system-tables/enabled-roles.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/enabled-roles.md \ No newline at end of file From 1d4227aab6e5c8961c4bae03a98bfd5de9dc72c2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 09:57:37 +0800 Subject: [PATCH 134/396] Create enabled-roles.md --- docs/zh/operations/system-tables/enabled-roles.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) create mode 100644 docs/zh/operations/system-tables/enabled-roles.md diff --git a/docs/zh/operations/system-tables/enabled-roles.md b/docs/zh/operations/system-tables/enabled-roles.md new file mode 100644 index 00000000000..f02c8ec1e65 --- /dev/null +++ b/docs/zh/operations/system-tables/enabled-roles.md @@ -0,0 +1,12 @@ +# system.enabled_roles {#system_tables-enabled_roles} + +包含当前所有活动角色, 包括当前用户的当前角色和当前角色的已授予角色. + +列信息: + +- `role_name` ([String](../../sql-reference/data-types/string.md))) — 角色名称. +- `with_admin_option` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 显示 `enabled_role` 是否为具有 `ADMIN OPTION` 权限的角色的标志. +- `is_current` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 显示 `enabled_role` 是否是当前用户的当前角色的标志. +- `is_default` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 显示 `enabled_role` 是否为默认角色的标志. + +[原始文章](https://clickhouse.com/docs/en/operations/system-tables/enabled-roles) From 4b56e76c0bcbfb258131b34440293d217a1987c8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 09:58:08 +0800 Subject: [PATCH 135/396] Delete errors.md --- docs/zh/operations/system-tables/errors.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/errors.md diff --git a/docs/zh/operations/system-tables/errors.md b/docs/zh/operations/system-tables/errors.md deleted file mode 120000 index 2bb8c1dff0d..00000000000 --- a/docs/zh/operations/system-tables/errors.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/errors.md \ No newline at end of file From 6f4bc11a04a5a21a44f182496e0703a8fa5d12f7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 10:00:59 +0800 Subject: [PATCH 136/396] Create errors.md --- docs/zh/operations/system-tables/errors.md | 35 ++++++++++++++++++++++ 1 file changed, 35 insertions(+) create mode 100644 docs/zh/operations/system-tables/errors.md diff --git a/docs/zh/operations/system-tables/errors.md b/docs/zh/operations/system-tables/errors.md new file mode 100644 index 00000000000..4d7baafb865 --- /dev/null +++ b/docs/zh/operations/system-tables/errors.md @@ -0,0 +1,35 @@ +# system.errors {#system_tables-errors} + +包含错误代码和它们被触发的次数. + +列信息: + +- `name` ([String](../../sql-reference/data-types/string.md)) — 错误名称 (`errorCodeToName`). +- `code` ([Int32](../../sql-reference/data-types/int-uint.md)) — 错误码. +- `value` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 发生此错误的次数. +- `last_error_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — 上一次错误发生的时间. +- `last_error_message` ([String](../../sql-reference/data-types/string.md)) — 最后一个错误的消息. +- `last_error_trace` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — 一个[堆栈跟踪](https://en.wikipedia.org/wiki/Stack_trace), 它表示存储被调用方法的物理地址列表. +- `remote` ([UInt8](../../sql-reference/data-types/int-uint.md)) — 远程异常(即在一个分布式查询期间接收的). + +**示例** + +``` sql +SELECT name, code, value +FROM system.errors +WHERE value > 0 +ORDER BY code ASC +LIMIT 1 + +┌─name─────────────┬─code─┬─value─┐ +│ CANNOT_OPEN_FILE │ 76 │ 1 │ +└──────────────────┴──────┴───────┘ +``` + +``` sql +WITH arrayMap(x -> demangle(addressToSymbol(x)), last_error_trace) AS all +SELECT name, arrayStringConcat(all, '\n') AS res +FROM system.errors +LIMIT 1 +SETTINGS allow_introspection_functions=1\G +``` From 289dcb60c6d5601709ab11bb70049dceacc77634 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 10:01:32 +0800 Subject: [PATCH 137/396] Delete grants.md --- docs/zh/operations/system-tables/grants.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/grants.md diff --git a/docs/zh/operations/system-tables/grants.md b/docs/zh/operations/system-tables/grants.md deleted file mode 120000 index ec7f50b118e..00000000000 --- a/docs/zh/operations/system-tables/grants.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/grants.md \ No newline at end of file From 96e560bcfbd401f1ac90885bd1514aeaeb6cf872 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 10:06:30 +0800 Subject: [PATCH 138/396] Create grants.md --- docs/zh/operations/system-tables/grants.md | 24 ++++++++++++++++++++++ 1 file changed, 24 insertions(+) create mode 100644 docs/zh/operations/system-tables/grants.md diff --git a/docs/zh/operations/system-tables/grants.md b/docs/zh/operations/system-tables/grants.md new file mode 100644 index 00000000000..8795f5ebf19 --- /dev/null +++ b/docs/zh/operations/system-tables/grants.md @@ -0,0 +1,24 @@ +# system.grants {#system_tables-grants} + +授予ClickHouse用户帐户的权限. + +列信息: +- `user_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — 用户名称. + +- `role_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — 分配给用户帐号的角色. + +- `access_type` ([Enum8](../../sql-reference/data-types/enum.md)) — ClickHouse用户帐号的接入参数. + +- `database` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — 数据库名称. + +- `table` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — 表名称. + +- `column` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — 被授予访问权限的列的名称. + +- `is_partial_revoke` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 逻辑值. 它显示了某些特权是否被取消. 可能的值: +- `0` — 该行描述了部分撤销. +- `1` — 这一行描述了一个授权. + +- `grant_option` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 被授予`WITH GRANT OPTION` 权限, 参见 [GRANT](../../sql-reference/statements/grant.md#grant-privigele-syntax). + +[原始文章](https://clickhouse.com/docs/en/operations/system-tables/grants) From 18b843be7725b97761c49027c80e3257f41fdc87 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 10:07:55 +0800 Subject: [PATCH 139/396] Delete licenses.md --- docs/zh/operations/system-tables/licenses.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/licenses.md diff --git a/docs/zh/operations/system-tables/licenses.md b/docs/zh/operations/system-tables/licenses.md deleted file mode 120000 index a84b4bbf3e3..00000000000 --- a/docs/zh/operations/system-tables/licenses.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/licenses.md \ No newline at end of file From 6a6f7ca6226f0a1b9ef79a00807c614272659249 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 10:10:33 +0800 Subject: [PATCH 140/396] Create licenses.md --- docs/zh/operations/system-tables/licenses.md | 39 ++++++++++++++++++++ 1 file changed, 39 insertions(+) create mode 100644 docs/zh/operations/system-tables/licenses.md diff --git a/docs/zh/operations/system-tables/licenses.md b/docs/zh/operations/system-tables/licenses.md new file mode 100644 index 00000000000..7e59ba865f5 --- /dev/null +++ b/docs/zh/operations/system-tables/licenses.md @@ -0,0 +1,39 @@ +# system.licenses {#system-tables_system.licenses} + +包含位于 ClickHouse 源的 [contrib](https://github.com/ClickHouse/ClickHouse/tree/master/contrib) 目录中的第三方库的许可证. + +列信息: + +- `library_name` ([String](../../sql-reference/data-types/string.md)) — 库的名称, 它是与之连接的许可证. +- `license_type` ([String](../../sql-reference/data-types/string.md)) — 许可类型-例如Apache, MIT. +- `license_path` ([String](../../sql-reference/data-types/string.md)) — 带有许可文本的文件的路径. +- `license_text` ([String](../../sql-reference/data-types/string.md)) — 许可协议文本. + +**示例** + +``` sql +SELECT library_name, license_type, license_path FROM system.licenses LIMIT 15 +``` + +``` text +┌─library_name───────┬─license_type─┬─license_path────────────────────────┐ +│ FastMemcpy │ MIT │ /contrib/FastMemcpy/LICENSE │ +│ arrow │ Apache │ /contrib/arrow/LICENSE.txt │ +│ avro │ Apache │ /contrib/avro/LICENSE.txt │ +│ aws-c-common │ Apache │ /contrib/aws-c-common/LICENSE │ +│ aws-c-event-stream │ Apache │ /contrib/aws-c-event-stream/LICENSE │ +│ aws-checksums │ Apache │ /contrib/aws-checksums/LICENSE │ +│ aws │ Apache │ /contrib/aws/LICENSE.txt │ +│ base64 │ BSD 2-clause │ /contrib/base64/LICENSE │ +│ boost │ Boost │ /contrib/boost/LICENSE_1_0.txt │ +│ brotli │ MIT │ /contrib/brotli/LICENSE │ +│ capnproto │ MIT │ /contrib/capnproto/LICENSE │ +│ cassandra │ Apache │ /contrib/cassandra/LICENSE.txt │ +│ cctz │ Apache │ /contrib/cctz/LICENSE.txt │ +│ cityhash102 │ MIT │ /contrib/cityhash102/COPYING │ +│ cppkafka │ BSD 2-clause │ /contrib/cppkafka/LICENSE │ +└────────────────────┴──────────────┴─────────────────────────────────────┘ + +``` + +[原始文章](https://clickhouse.com/docs/en/operations/system-tables/licenses) From 8c7528b0d4aeefd01c79a6fc4589f0eb483b80c1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 10:11:19 +0800 Subject: [PATCH 141/396] Delete opentelemetry_span_log.md --- docs/zh/operations/system-tables/opentelemetry_span_log.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/opentelemetry_span_log.md diff --git a/docs/zh/operations/system-tables/opentelemetry_span_log.md b/docs/zh/operations/system-tables/opentelemetry_span_log.md deleted file mode 120000 index 59aab8cd11d..00000000000 --- a/docs/zh/operations/system-tables/opentelemetry_span_log.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/opentelemetry_span_log.md \ No newline at end of file From 4d2dcb6d78cf937ca4ececa96336c2468237b6a0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 10:28:46 +0800 Subject: [PATCH 142/396] Create opentelemetry_span_log.md --- .../system-tables/opentelemetry_span_log.md | 53 +++++++++++++++++++ 1 file changed, 53 insertions(+) create mode 100644 docs/zh/operations/system-tables/opentelemetry_span_log.md diff --git a/docs/zh/operations/system-tables/opentelemetry_span_log.md b/docs/zh/operations/system-tables/opentelemetry_span_log.md new file mode 100644 index 00000000000..6eceb3da889 --- /dev/null +++ b/docs/zh/operations/system-tables/opentelemetry_span_log.md @@ -0,0 +1,53 @@ +# system.opentelemetry_span_log {#system_tables-opentelemetry_span_log} + +包含已执行查询的[跟踪范围](https://opentracing.io/docs/overview/spans/)的信息. + +列信息: + +- `trace_id` ([UUID](../../sql-reference/data-types/uuid.md)) — 执行的查询的跟踪ID. + +- `span_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — `跟踪 跨度` ID. + +- `parent_span_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 父级`跟踪 跨度` ID. + +- `operation_name` ([String](../../sql-reference/data-types/string.md)) — 操作的名称. + +- `start_time_us` ([UInt64](../../sql-reference/data-types/int-uint.md)) — `跟踪 跨度` 开始时间 (微秒). + +- `finish_time_us` ([UInt64](../../sql-reference/data-types/int-uint.md)) — `跟踪 跨度 结束时间 (微秒). + +- `finish_date` ([Date](../../sql-reference/data-types/date.md)) — `跟踪 跨度` 完成日期. + +- `attribute.names` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — [属性](https://opentelemetry.io/docs/go/instrumentation/#attributes) 名称取决于 `跟踪 跨度`. 它们是根据[OpenTelemetry](https://opentelemetry.io/)标准中的建议填写的. + +- `attribute.values` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — 属性值取决于 `跟踪 跨度`. 它们是根据 `OpenTelemetry` 标准中的建议填写的. + +**示例** + +查询: + +``` sql +SELECT * FROM system.opentelemetry_span_log LIMIT 1 FORMAT Vertical; +``` + +结果: + +``` text +Row 1: +────── +trace_id: cdab0847-0d62-61d5-4d38-dd65b19a1914 +span_id: 701487461015578150 +parent_span_id: 2991972114672045096 +operation_name: DB::Block DB::InterpreterSelectQuery::getSampleBlockImpl() +start_time_us: 1612374594529090 +finish_time_us: 1612374594529108 +finish_date: 2021-02-03 +attribute.names: [] +attribute.values: [] +``` + +**另请参阅** + +- [OpenTelemetry](../../operations/opentelemetry.md) + +[原始文章](https://clickhouse.com/docs/en/operations/system_tables/opentelemetry_span_log) From 9031e42d5c282cfd56208c02400634ef49ddc87d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 10:40:15 +0800 Subject: [PATCH 143/396] Delete parts_columns.md --- docs/zh/operations/system-tables/parts_columns.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/parts_columns.md diff --git a/docs/zh/operations/system-tables/parts_columns.md b/docs/zh/operations/system-tables/parts_columns.md deleted file mode 120000 index f2c7d4dde34..00000000000 --- a/docs/zh/operations/system-tables/parts_columns.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/parts_columns.md \ No newline at end of file From 69459549c42e6e6bb824d612633954f841cc4ad7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 11:01:38 +0800 Subject: [PATCH 144/396] Create parts_columns.md --- .../operations/system-tables/parts_columns.md | 148 ++++++++++++++++++ 1 file changed, 148 insertions(+) create mode 100644 docs/zh/operations/system-tables/parts_columns.md diff --git a/docs/zh/operations/system-tables/parts_columns.md b/docs/zh/operations/system-tables/parts_columns.md new file mode 100644 index 00000000000..cd8b2fe9526 --- /dev/null +++ b/docs/zh/operations/system-tables/parts_columns.md @@ -0,0 +1,148 @@ +# system.parts_columns {#system_tables-parts_columns} + +包含关于[MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md)表的部分和列的信息. + +每一行描述一个数据部分. + +列信息: + +- `partition` ([String](../../sql-reference/data-types/string.md)) — 分区的名称. 要了解什么是分区, 请参阅[ALTER](../../sql-reference/statements/alter/index.md#query_language_queries_alter)查询的描述. + + 格式: + + - `YYYYMM` 按月自动分区. + - `any_string` 当手动分区. + +- `name` ([String](../../sql-reference/data-types/string.md)) — 数据部分的名称. + +- `part_type` ([String](../../sql-reference/data-types/string.md)) — 数据部分存储格式. + + 可能的值: + + - `Wide` — 每一列存储在文件系统中的一个单独的文件中. + - `Compact` — 所有列都存储在文件系统中的一个文件中. + + 数据存储格式由[MergeTree](../../engines/table-engines/mergetree-family/mergetree.md)表的 `min_bytes_for_wide_part` 和 `min_rows_for_wide_part` 设置控制. + +- `active` ([UInt8](../../sql-reference/data-types/int-uint.md)) — 数据部分是否处于活动状态的标志. 如果数据部分是活动的, 则在表中使用它. 否则, 它被删除. 合并后仍保留非活动数据部分. + +- `marks` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 标记数. 要获得数据部分中的大约行数, 请将“标记”乘以索引粒度(通常为8192)(此提示不适用于自适应粒度). + +- `rows` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 行数. + +- `bytes_on_disk` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 所有数据部分文件的总大小(以字节为单位). + +- `data_compressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 在数据部分中压缩数据的总大小. 不包括所有辅助文件(例如,带有标记的文件). + +- `data_uncompressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 数据部分中未压缩数据的总大小. 不包括所有辅助文件(例如,带有标记的文件). + +- `marks_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 带标记的文件的大小. + +- `modification_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — 包含数据部分的目录被修改的时间. 这通常对应于数据部分创建的时间. + +- `remove_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — 数据部分变为非活动状态的时间. + +- `refcount` ([UInt32](../../sql-reference/data-types/int-uint.md)) — 使用数据部分的位置数. 大于2的值表示该数据部分用于查询或合并. + +- `min_date` ([Date](../../sql-reference/data-types/date.md)) — 数据部分中日期键的最小值. + +- `max_date` ([Date](../../sql-reference/data-types/date.md)) — 数据部分中日期键的最大值. + +- `partition_id` ([String](../../sql-reference/data-types/string.md)) — 分区ID. + +- `min_block_number` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 合并后组成当前部分的数据部分最小值. + +- `max_block_number` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 合并后组成当前部分的数据部分最大值. + +- `level` ([UInt32](../../sql-reference/data-types/int-uint.md)) — 合并树的深度. 0表示当前部分是通过插入而不是合并其他部分创建的. + +- `data_version` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 用于确定应该对数据部分应用哪些突变的编号(版本高于 `data_version` 的突变). + +- `primary_key_bytes_in_memory` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 主键值使用的内存量(以字节为单位). + +- `primary_key_bytes_in_memory_allocated` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 为主键值保留的内存量(以字节为单位). + +- `database` ([String](../../sql-reference/data-types/string.md)) — 数据库名称. + +- `table` ([String](../../sql-reference/data-types/string.md)) — 表名称. + +- `engine` ([String](../../sql-reference/data-types/string.md)) — 不带参数的表引擎的名称. + +- `disk_name` ([String](../../sql-reference/data-types/string.md)) — 存储数据部分的磁盘名称. + +- `path` ([String](../../sql-reference/data-types/string.md)) — 数据部件文件文件夹的绝对路径. + +- `column` ([String](../../sql-reference/data-types/string.md)) — 列名称. + +- `type` ([String](../../sql-reference/data-types/string.md)) — 列类型. + +- `column_position` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 表中以1开头的一列的序号位置. + +- `default_kind` ([String](../../sql-reference/data-types/string.md)) — 默认值的表达式类型 (`DEFAULT`, `MATERIALIZED`, `ALIAS`), 如果未定义则为空字符串. + +- `default_expression` ([String](../../sql-reference/data-types/string.md)) — 表达式的默认值, 如果未定义则为空字符串. + +- `column_bytes_on_disk` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 列的总大小(以字节为单位). + +- `column_data_compressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 列中压缩数据的总大小,以字节为单位. + +- `column_data_uncompressed_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 列中解压缩数据的总大小,以字节为单位. + +- `column_marks_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 带标记的列的大小,以字节为单位. + +- `bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) — `bytes_on_disk` 别名. + +- `marks_size` ([UInt64](../../sql-reference/data-types/int-uint.md)) — `marks_bytes` 别名. + +**示例** + +``` sql +SELECT * FROM system.parts_columns LIMIT 1 FORMAT Vertical; +``` + +``` text +Row 1: +────── +partition: tuple() +name: all_1_2_1 +part_type: Wide +active: 1 +marks: 2 +rows: 2 +bytes_on_disk: 155 +data_compressed_bytes: 56 +data_uncompressed_bytes: 4 +marks_bytes: 96 +modification_time: 2020-09-23 10:13:36 +remove_time: 2106-02-07 06:28:15 +refcount: 1 +min_date: 1970-01-01 +max_date: 1970-01-01 +partition_id: all +min_block_number: 1 +max_block_number: 2 +level: 1 +data_version: 1 +primary_key_bytes_in_memory: 2 +primary_key_bytes_in_memory_allocated: 64 +database: default +table: 53r93yleapyears +engine: MergeTree +disk_name: default +path: /var/lib/clickhouse/data/default/53r93yleapyears/all_1_2_1/ +column: id +type: Int8 +column_position: 1 +default_kind: +default_expression: +column_bytes_on_disk: 76 +column_data_compressed_bytes: 28 +column_data_uncompressed_bytes: 2 +column_marks_bytes: 48 +``` + +**另请参阅** + +- [MergeTree family](../../engines/table-engines/mergetree-family/mergetree.md) + +[原始文章](https://clickhouse.com/docs/en/operations/system_tables/parts_columns) From 4da05888d5f7db2b6f76dca79e32c852a3716f06 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 13:07:07 +0800 Subject: [PATCH 145/396] Delete query_views_log.md --- docs/zh/operations/system-tables/query_views_log.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/query_views_log.md diff --git a/docs/zh/operations/system-tables/query_views_log.md b/docs/zh/operations/system-tables/query_views_log.md deleted file mode 120000 index f606e4108ca..00000000000 --- a/docs/zh/operations/system-tables/query_views_log.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/query_views_log.md \ No newline at end of file From 35672e7f6c3ede6fc67fc5043aa92cb4251c2f3e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 13:28:24 +0800 Subject: [PATCH 146/396] Create query_views_log.md --- .../system-tables/query_views_log.md | 86 +++++++++++++++++++ 1 file changed, 86 insertions(+) create mode 100644 docs/zh/operations/system-tables/query_views_log.md diff --git a/docs/zh/operations/system-tables/query_views_log.md b/docs/zh/operations/system-tables/query_views_log.md new file mode 100644 index 00000000000..57946e5104a --- /dev/null +++ b/docs/zh/operations/system-tables/query_views_log.md @@ -0,0 +1,86 @@ +# system.query_views_log {#system_tables-query_views_log} + +包含有关运行查询时执行的从属视图的信息,例如视图类型或执行时间. + +开始记录: + +1. 在 [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log) 部分配置参数. +2. 设置 [log_query_views](../../operations/settings/settings.md#settings-log-query-views) 为 1. + +数据的刷新周期是在[query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log)服务器设置部分的 `flush_interval_milliseconds` 参数中设置的. 要强制刷新,请使用[SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs)查询. + +ClickHouse不会自动从表中删除数据. 详见 [Introduction](../../operations/system-tables/index.md#system-tables-introduction). + +您可以使用[log_queries_probability](../../operations/settings/settings.md#log-queries-probability)设置来减少在 `query_views_log` 表中注册的查询数量. + +列信息: + +- `event_date` ([Date](../../sql-reference/data-types/date.md)) — 视图的最后一个事件发生的日期. +- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — 视图完成执行的日期和时间. +- `event_time_microseconds` ([DateTime](../../sql-reference/data-types/datetime.md)) — 视图以微秒精度完成执行的日期和时间. +- `view_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 视图执行的持续时间(各阶段之和), 以毫秒为单位. +- `initial_query_id` ([String](../../sql-reference/data-types/string.md)) — 初始查询的ID (用于分布式查询执行). +- `view_name` ([String](../../sql-reference/data-types/string.md)) — 视图名称. +- `view_uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — 视图的UUID. +- `view_type` ([Enum8](../../sql-reference/data-types/enum.md)) — 视图类型. 值: + - `'Default' = 1` — [Default views](../../sql-reference/statements/create/view.md#normal). 不应该出现在日志中. + - `'Materialized' = 2` — [Materialized views](../../sql-reference/statements/create/view.md#materialized). + - `'Live' = 3` — [Live views](../../sql-reference/statements/create/view.md#live-view). +- `view_query` ([String](../../sql-reference/data-types/string.md)) — 视图执行的查询. +- `view_target` ([String](../../sql-reference/data-types/string.md)) — 视图目标表的名称. +- `read_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 读行数. +- `read_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 读字节数. +- `written_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 写入行数. +- `written_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 写入字节数. +- `peak_memory_usage` ([Int64](../../sql-reference/data-types/int-uint.md)) — 在此视图上下文中, 已分配内存和已释放内存之间的最大差值. +- `ProfileEvents` ([Map(String, UInt64)](../../sql-reference/data-types/array.md)) — ProfileEvents度量不同的指标. 它们的描述可以在表 [system.events](../../operations/system-tables/events.md#system_tables-events) 中找到. +- `status` ([Enum8](../../sql-reference/data-types/enum.md)) — 视图状态. 值: + - `'QueryStart' = 1` — 成功启动视图执行. 不应该出现. + - `'QueryFinish' = 2` — 视图执行成功结束. + - `'ExceptionBeforeStart' = 3` — 视图执行开始前的异常. + - `'ExceptionWhileProcessing' = 4` — 视图执行期间的异常. +- `exception_code` ([Int32](../../sql-reference/data-types/int-uint.md)) — 异常代码. +- `exception` ([String](../../sql-reference/data-types/string.md)) — 异常报文. +- `stack_trace` ([String](../../sql-reference/data-types/string.md)) — [堆栈跟踪](https://en.wikipedia.org/wiki/Stack_trace). 如果查询成功完成, 则为空字符串. + +**示例** + +查询: + +``` sql +SELECT * FROM system.query_views_log LIMIT 1 \G; +``` + +结果: + +``` text +Row 1: +────── +event_date: 2021-06-22 +event_time: 2021-06-22 13:23:07 +event_time_microseconds: 2021-06-22 13:23:07.738221 +view_duration_ms: 0 +initial_query_id: c3a1ac02-9cad-479b-af54-9e9c0a7afd70 +view_name: default.matview_inner +view_uuid: 00000000-0000-0000-0000-000000000000 +view_type: Materialized +view_query: SELECT * FROM default.table_b +view_target: default.`.inner.matview_inner` +read_rows: 4 +read_bytes: 64 +written_rows: 2 +written_bytes: 32 +peak_memory_usage: 4196188 +ProfileEvents: {'FileOpen':2,'WriteBufferFromFileDescriptorWrite':2,'WriteBufferFromFileDescriptorWriteBytes':187,'IOBufferAllocs':3,'IOBufferAllocBytes':3145773,'FunctionExecute':3,'DiskWriteElapsedMicroseconds':13,'InsertedRows':2,'InsertedBytes':16,'SelectedRows':4,'SelectedBytes':48,'ContextLock':16,'RWLockAcquiredReadLocks':1,'RealTimeMicroseconds':698,'SoftPageFaults':4,'OSReadChars':463} +status: QueryFinish +exception_code: 0 +exception: +stack_trace: +``` + +**另请参阅** + +- [system.query_log](../../operations/system-tables/query_log.md#system_tables-query_log) — 包含查询执行的常用信息的 `query_log`系统表的描述. +- [system.query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log) — 包含关于每个查询执行线程的信息. + +[原始文章](https://clickhouse.com/docs/en/operations/system_tables/query_thread_log) From fb6aaae7f07d5262fef0c93153f7601481ad3245 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 13:29:55 +0800 Subject: [PATCH 147/396] Delete quota_limits.md --- docs/zh/operations/system-tables/quota_limits.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/quota_limits.md diff --git a/docs/zh/operations/system-tables/quota_limits.md b/docs/zh/operations/system-tables/quota_limits.md deleted file mode 120000 index 3fecf881c92..00000000000 --- a/docs/zh/operations/system-tables/quota_limits.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/quota_limits.md \ No newline at end of file From 3770db6c35c1aaa964c7b49d62511638021c8774 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 13:40:35 +0800 Subject: [PATCH 148/396] Create quota_limits.md --- .../operations/system-tables/quota_limits.md | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) create mode 100644 docs/zh/operations/system-tables/quota_limits.md diff --git a/docs/zh/operations/system-tables/quota_limits.md b/docs/zh/operations/system-tables/quota_limits.md new file mode 100644 index 00000000000..67a2a01eb71 --- /dev/null +++ b/docs/zh/operations/system-tables/quota_limits.md @@ -0,0 +1,21 @@ +# system.quota_limits {#system_tables-quota_limits} + +包含关于所有配额的所有间隔的最大值的信息. 任何行数或0行都可以对应一个配额. + +列信息: +- `quota_name` ([String](../../sql-reference/data-types/string.md)) — 配额名称. +- `duration` ([UInt32](../../sql-reference/data-types/int-uint.md)) — 计算资源消耗的时间间隔长度,单位为秒. +- `is_randomized_interval` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 逻辑值. 它显示了间隔是否是随机的. 如果间隔不是随机的, 它总是在同一时间开始. 例如, 1 分钟的间隔总是从整数分钟开始(即它可以从 11:20:00 开始, 但它永远不会从 11:20:01 开始), 一天的间隔总是从 UTC 午夜开始. 如果间隔是随机的, 则第一个间隔在随机时间开始, 随后的间隔一个接一个开始. 值: +- `0` — 区间不是随机的. +- `1` — 区间是随机的. +- `max_queries` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 最大查询数. +- `max_query_selects` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — select 最大查询数. +- `max_query_inserts` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — insert 最大查询数. +- `max_errors` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 最大错误数. +- `max_result_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 最大结果行数. +- `max_result_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 用于存储查询结果的最大RAM容量(以字节为单位). +- `max_read_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 从参与查询的所有表和表函数中读取的最大行数. +- `max_read_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 从参与查询的所有表和表函数中读取的最大字节数. +- `max_execution_time` ([Nullable](../../sql-reference/data-types/nullable.md)([Float64](../../sql-reference/data-types/float.md))) — 查询执行时间的最大值, 单位为秒. + +[原始文章](https://clickhouse.com/docs/en/operations/system-tables/quota_limits) From 4da75a83fc3f6572877e2db6406129b23b85e873 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 13:41:11 +0800 Subject: [PATCH 149/396] Delete quota_usage.md --- docs/zh/operations/system-tables/quota_usage.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/quota_usage.md diff --git a/docs/zh/operations/system-tables/quota_usage.md b/docs/zh/operations/system-tables/quota_usage.md deleted file mode 120000 index c79f1a75033..00000000000 --- a/docs/zh/operations/system-tables/quota_usage.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/quota_usage.md \ No newline at end of file From ced053d51e18004b05c15a54786fd3636dc6ce62 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 13:52:59 +0800 Subject: [PATCH 150/396] Create quota_usage.md --- .../operations/system-tables/quota_usage.md | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) create mode 100644 docs/zh/operations/system-tables/quota_usage.md diff --git a/docs/zh/operations/system-tables/quota_usage.md b/docs/zh/operations/system-tables/quota_usage.md new file mode 100644 index 00000000000..020d14aee5e --- /dev/null +++ b/docs/zh/operations/system-tables/quota_usage.md @@ -0,0 +1,32 @@ +# system.quota_usage {#system_tables-quota_usage} + +当前用户的配额使用情况: 使用了多少, 还剩多少. + +列信息: +- `quota_name` ([String](../../sql-reference/data-types/string.md)) — 配额名称. +- `quota_key`([String](../../sql-reference/data-types/string.md)) — 配额数值. 比如, if keys = \[`ip address`\], `quota_key` 可能有一个值 ‘192.168.1.1’. +- `start_time`([Nullable](../../sql-reference/data-types/nullable.md)([DateTime](../../sql-reference/data-types/datetime.md))) — 计算资源消耗的开始时间. +- `end_time`([Nullable](../../sql-reference/data-types/nullable.md)([DateTime](../../sql-reference/data-types/datetime.md))) — 计算资源消耗的结束时间. +- `duration` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 计算资源消耗的时间间隔长度, 单位为秒. +- `queries` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 在此间隔内的请求总数. +- `query_selects` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 在此间隔内查询请求的总数. +- `query_inserts` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 在此间隔内插入请求的总数. +- `max_queries` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 最大请求数. +- `errors` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 抛出异常的查询数. +- `max_errors` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 最大错误数. +- `result_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 结果给出的总行数. +- `max_result_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 最大结果行数. +- `result_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 用于存储查询结果的RAM容量(以字节为单位). +- `max_result_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 用于存储查询结果的最大RAM容量,以字节为单位. +- `read_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The total number of source rows read from tables for running the query on all remote servers. +- `max_read_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 参与查询的所有表和表函数中读取的最大行数. +- `read_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 参与查询的所有表和表函数中读取的总字节数. +- `max_read_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 所有表和表函数中读取的最大字节数. +- `execution_time` ([Nullable](../../sql-reference/data-types/nullable.md)([Float64](../../sql-reference/data-types/float.md))) — 总查询执行时间, 以秒为单位(挂墙时间). +- `max_execution_time` ([Nullable](../../sql-reference/data-types/nullable.md)([Float64](../../sql-reference/data-types/float.md))) — 查询最大执行时间. + +## 另请参阅 {#see-also} + +- [查看配额信息](../../sql-reference/statements/show.md#show-quota-statement) + +[原始文章](https://clickhouse.com/docs/en/operations/system-tables/quota_usage) From 05594b4ff7f683c9b95a68bdd196a7ae423ca6e0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 13:53:42 +0800 Subject: [PATCH 151/396] Delete quotas.md --- docs/zh/operations/system-tables/quotas.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/quotas.md diff --git a/docs/zh/operations/system-tables/quotas.md b/docs/zh/operations/system-tables/quotas.md deleted file mode 120000 index b6a26bf77f3..00000000000 --- a/docs/zh/operations/system-tables/quotas.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/quotas.md \ No newline at end of file From 6e8db7715954661ae377d5a5e60e22a4df40b937 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 14:09:01 +0800 Subject: [PATCH 152/396] Create quotas.md --- docs/zh/operations/system-tables/quotas.md | 28 ++++++++++++++++++++++ 1 file changed, 28 insertions(+) create mode 100644 docs/zh/operations/system-tables/quotas.md diff --git a/docs/zh/operations/system-tables/quotas.md b/docs/zh/operations/system-tables/quotas.md new file mode 100644 index 00000000000..b25c213bdd1 --- /dev/null +++ b/docs/zh/operations/system-tables/quotas.md @@ -0,0 +1,28 @@ +# system.quotas {#system_tables-quotas} + +包含 [quotas](../../operations/system-tables/quotas.md) 信息. + +列信息: +- `name` ([String](../../sql-reference/data-types/string.md)) — 配额名称. +- `id` ([UUID](../../sql-reference/data-types/uuid.md)) — 配额 ID. +- `storage`([String](../../sql-reference/data-types/string.md)) — 存储配额. 可能的值:“users.xml”(如果在 users.xml 文件中配置了配额), “disk”(如果由 SQL 查询配置的配额). +- `keys` ([Array](../../sql-reference/data-types/array.md)([Enum8](../../sql-reference/data-types/enum.md))) — Key指定配额应该如何共享. 如果两个连接使用相同的配额和键,则它们共享相同数量的资源. 值: + - `[]` — 所有用户共享相同的配额. + - `['user_name']` — 相同用户名的连接共享相同的配额. + - `['ip_address']` — 来自同一IP的连接共享相同的配额. + - `['client_key']` — 具有相同密钥的连接共享相同配额. 密钥必须由客户端显式提供. 使用[clickhouse-client](../../interfaces/cli.md)时, 在 `--quota_key` 参数中传递一个key值, 或者在客户端配置文件中使用 `quota_key` 参数. 使用 HTTP 接口时, 使用 `X-ClickHouse-Quota` 报头. + - `['user_name', 'client_key']` — 具有相同 `client_key` 的连接共享相同的配额. 如果客户端没有提供密钥, 配额将跟踪 `user_name`. + - `['client_key', 'ip_address']` — 具有相同 `client_key` 的连接共享相同的配额. 如果客户端没有提供密钥, 配额将跟踪 `ip_address`. +- `durations` ([Array](../../sql-reference/data-types/array.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 时间间隔以秒为单位. +- `apply_to_all` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 逻辑值. 它显示了配额应用于哪些用户. 值: + - `0` — 配额应用于 `apply_to_list` 中指定的用户. + - `1` — 配额适用于除 `apply_to_except` 中列出的用户之外的所有用户. +- `apply_to_list` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — 需要应用配额的用户名/[角色](../../operations/access-rights.md#role-management) 列表. +- `apply_to_except` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — 不需要应用配额的用户名/角色列表. + +## 另请参阅 {#see-also} + +- [查看配额信息](../../sql-reference/statements/show.md#show-quotas-statement) + +[原始文章](https://clickhouse.com/docs/en/operations/system-tables/quotas) + From ab8b9330206ed5f13c50bcc7666153a53f5c849e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 14:09:33 +0800 Subject: [PATCH 153/396] Delete quotas_usage.md --- docs/zh/operations/system-tables/quotas_usage.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/quotas_usage.md diff --git a/docs/zh/operations/system-tables/quotas_usage.md b/docs/zh/operations/system-tables/quotas_usage.md deleted file mode 120000 index ba204a4d9c0..00000000000 --- a/docs/zh/operations/system-tables/quotas_usage.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/quotas_usage.md \ No newline at end of file From 66ec6545bf3c2dcea82b7bf5cfdbd6e245eee8ca Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 14:19:11 +0800 Subject: [PATCH 154/396] Create quotas_usage.md --- .../operations/system-tables/quotas_usage.md | 35 +++++++++++++++++++ 1 file changed, 35 insertions(+) create mode 100644 docs/zh/operations/system-tables/quotas_usage.md diff --git a/docs/zh/operations/system-tables/quotas_usage.md b/docs/zh/operations/system-tables/quotas_usage.md new file mode 100644 index 00000000000..f554539ecfe --- /dev/null +++ b/docs/zh/operations/system-tables/quotas_usage.md @@ -0,0 +1,35 @@ +# system.quotas_usage {#system_tables-quotas_usage} + +所有用户配额使用情况. + +列信息: +- `quota_name` ([String](../../sql-reference/data-types/string.md)) — 配额名称. +- `quota_key` ([String](../../sql-reference/data-types/string.md)) — 配额key值. +- `is_current` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 当前用户配额使用情况. +- `start_time` ([Nullable](../../sql-reference/data-types/nullable.md)([DateTime](../../sql-reference/data-types/datetime.md)))) — 计算资源消耗的开始时间. +- `end_time` ([Nullable](../../sql-reference/data-types/nullable.md)([DateTime](../../sql-reference/data-types/datetime.md)))) — 计算资源消耗的结束时间. +- `duration` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt32](../../sql-reference/data-types/int-uint.md))) — 计算资源消耗的时间间隔长度,单位为秒. +- `queries` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 在此间隔内的请求总数. +- `max_queries` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 最大请求数. +- `query_selects` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 此间隔内查询请求的总数. +- `max_query_selects` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 查询请求的最大数量. +- `query_inserts` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 此间隔内插入请求的总数. +- `max_query_inserts` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 最大插入请求数. +- `errors` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 抛出异常的查询数. +- `max_errors` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 最大误差数. +- `result_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 结果给出的总行数. +- `max_result_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 从表中读取的最大源行数. +- `result_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 用于存储查询结果的RAM容量(以字节为单位). +- `max_result_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 用于存储查询结果的最大RAM容量, 以字节为单位. +- `read_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md)))) — 为在所有远程服务器上运行查询而从表中读取的源行总数. +- `max_read_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 参与查询的所有表和表函数中读取的最大行数. +- `read_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 参与查询的所有表和表函数中读取的总字节数. +- `max_read_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — 所有表和表函数中读取的最大字节数. +- `execution_time` ([Nullable](../../sql-reference/data-types/nullable.md)([Float64](../../sql-reference/data-types/float.md))) — 总查询执行时间, 以秒为单位(挂墙时间). +- `max_execution_time` ([Nullable](../../sql-reference/data-types/nullable.md)([Float64](../../sql-reference/data-types/float.md))) — 查询最大执行时间. + +## 另请参阅 {#see-also} + +- [查看配额信息](../../sql-reference/statements/show.md#show-quota-statement) + +[原始文章](https://clickhouse.com/docs/en/operations/system-tables/quotas_usage) From 7d97db65f61f07f2097afcf7ebb31694e7e604c5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 14:20:26 +0800 Subject: [PATCH 155/396] Delete replicated_fetches.md --- docs/zh/operations/system-tables/replicated_fetches.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/replicated_fetches.md diff --git a/docs/zh/operations/system-tables/replicated_fetches.md b/docs/zh/operations/system-tables/replicated_fetches.md deleted file mode 120000 index bf4547ebd07..00000000000 --- a/docs/zh/operations/system-tables/replicated_fetches.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/replicated_fetches.md \ No newline at end of file From e51288653f0a294031fe0aa534fa447bd6f209f7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 14:28:42 +0800 Subject: [PATCH 156/396] Create replicated_fetches.md --- .../system-tables/replicated_fetches.md | 70 +++++++++++++++++++ 1 file changed, 70 insertions(+) create mode 100644 docs/zh/operations/system-tables/replicated_fetches.md diff --git a/docs/zh/operations/system-tables/replicated_fetches.md b/docs/zh/operations/system-tables/replicated_fetches.md new file mode 100644 index 00000000000..30846c49bb1 --- /dev/null +++ b/docs/zh/operations/system-tables/replicated_fetches.md @@ -0,0 +1,70 @@ +# system.replicated_fetches {#system_tables-replicated_fetches} + +包含当前正在运行的后台提取的信息. + +列信息: + +- `database` ([String](../../sql-reference/data-types/string.md)) — 数据库名称. + +- `table` ([String](../../sql-reference/data-types/string.md)) — 表名称. + +- `elapsed` ([Float64](../../sql-reference/data-types/float.md)) — 显示当前正在运行的后台提取开始以来经过的时间(以秒为单位). + +- `progress` ([Float64](../../sql-reference/data-types/float.md)) — 完成工作的百分比从0到1. + +- `result_part_name` ([String](../../sql-reference/data-types/string.md)) — 显示当前正在运行的后台提取的结果而形成的部分的名称. + +- `result_part_path` ([String](../../sql-reference/data-types/string.md)) — 显示当前正在运行的后台提取的结果而形成的部分的绝对路径. + +- `partition_id` ([String](../../sql-reference/data-types/string.md)) — 分区 ID. + +- `total_size_bytes_compressed` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 结果部分中压缩数据的总大小(以字节为单位). + +- `bytes_read_compressed` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 从结果部分读取的压缩字节数. + +- `source_replica_path` ([String](../../sql-reference/data-types/string.md)) — 源副本的绝对路径. + +- `source_replica_hostname` ([String](../../sql-reference/data-types/string.md)) — 源副本的主机名称. + +- `source_replica_port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — 源副本的端口号. + +- `interserver_scheme` ([String](../../sql-reference/data-types/string.md)) — Name of the interserver scheme. + +- `URI` ([String](../../sql-reference/data-types/string.md)) — 统一资源标识符. + +- `to_detached` ([UInt8](../../sql-reference/data-types/int-uint.md)) — 该标志指示是否正在使用 `TO DETACHED` 表达式执行当前正在运行的后台提取. + +- `thread_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 线程标识符. + +**示例** + +``` sql +SELECT * FROM system.replicated_fetches LIMIT 1 FORMAT Vertical; +``` + +``` text +Row 1: +────── +database: default +table: t +elapsed: 7.243039876 +progress: 0.41832135995612835 +result_part_name: all_0_0_0 +result_part_path: /var/lib/clickhouse/store/700/70080a04-b2de-4adf-9fa5-9ea210e81766/all_0_0_0/ +partition_id: all +total_size_bytes_compressed: 1052783726 +bytes_read_compressed: 440401920 +source_replica_path: /clickhouse/test/t/replicas/1 +source_replica_hostname: node1 +source_replica_port: 9009 +interserver_scheme: http +URI: http://node1:9009/?endpoint=DataPartsExchange%3A%2Fclickhouse%2Ftest%2Ft%2Freplicas%2F1&part=all_0_0_0&client_protocol_version=4&compress=false +to_detached: 0 +thread_id: 54 +``` + +**另请参阅** + +- [Managing ReplicatedMergeTree Tables](../../sql-reference/statements/system/#query-language-system-replicated) + +[原始文章](https://clickhouse.com/docs/en/operations/system_tables/replicated_fetches) From 87f83ecf7c531f7a48cc6067acd7e3a7929397ea Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 14:29:09 +0800 Subject: [PATCH 157/396] Delete replication_queue.md --- docs/zh/operations/system-tables/replication_queue.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/replication_queue.md diff --git a/docs/zh/operations/system-tables/replication_queue.md b/docs/zh/operations/system-tables/replication_queue.md deleted file mode 120000 index cdc452594e6..00000000000 --- a/docs/zh/operations/system-tables/replication_queue.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/replication_queue.md \ No newline at end of file From c2cf915391a67dff25c903da73e116703120d890 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 14:49:22 +0800 Subject: [PATCH 158/396] Create replication_queue.md --- .../system-tables/replication_queue.md | 91 +++++++++++++++++++ 1 file changed, 91 insertions(+) create mode 100644 docs/zh/operations/system-tables/replication_queue.md diff --git a/docs/zh/operations/system-tables/replication_queue.md b/docs/zh/operations/system-tables/replication_queue.md new file mode 100644 index 00000000000..0c9e19f76d0 --- /dev/null +++ b/docs/zh/operations/system-tables/replication_queue.md @@ -0,0 +1,91 @@ +# system.replication_queue {#system_tables-replication_queue} + +包含用于 `ReplicatedMergeTree` 系列表的复制队列中存储在ZooKeeper中的任务的信息. + +列信息: + +- `database` ([String](../../sql-reference/data-types/string.md)) — 数据库名称. + +- `table` ([String](../../sql-reference/data-types/string.md)) — 表名称. + +- `replica_name` ([String](../../sql-reference/data-types/string.md)) — ZooKeeper中的副本名称. 同一张表的不同副本具有不同的名称. + +- `position` ([UInt32](../../sql-reference/data-types/int-uint.md)) — 任务在队列中的位置. + +- `node_name` ([String](../../sql-reference/data-types/string.md)) — ZooKeeper中的节点名称. + +- `type` ([String](../../sql-reference/data-types/string.md)) — 队列中任务的类型, 其中之一: + + - `GET_PART` — 从另一个副本获取分片. + - `ATTACH_PART` — 附加的部分, 可能来自我们自己的副本(如果在 `detached` 文件夹中找到). 您可以将其视为具有一些优化的 `GET_PART` , 因为它们几乎相同. + - `MERGE_PARTS` — 合并分片. + - `DROP_RANGE` — 删除指定分区内指定编号范围内的分片. + - `CLEAR_COLUMN` — 注意:已弃用. 从指定分区删除特定列. + - `CLEAR_INDEX` — 注意:已弃用. 从指定分区删除特定索引. + - `REPLACE_RANGE` — 丢弃一定范围的零件并用新零件替换它们. + - `MUTATE_PART` — 对分片应用一个或多个突变. + - `ALTER_METADATA` — 根据全局 /metadata 和 /columns 路径应用alter修改. + +- `create_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — 提交任务执行的日期和时间. + +- `required_quorum` ([UInt32](../../sql-reference/data-types/int-uint.md)) — 等待任务完成并确认完成的副本数. 此列仅与 `GET_PARTS` 任务相关. + +- `source_replica` ([String](../../sql-reference/data-types/string.md)) — 源副本的名称. + +- `new_part_name` ([String](../../sql-reference/data-types/string.md)) — 新分片的名称. + +- `parts_to_merge` ([Array](../../sql-reference/data-types/array.md) ([String](../../sql-reference/data-types/string.md))) — 要合并或更新的分片名称. + +- `is_detach` ([UInt8](../../sql-reference/data-types/int-uint.md)) — 该标志指示 `DETACH_PARTS` 任务是否在队列中. + +- `is_currently_executing` ([UInt8](../../sql-reference/data-types/int-uint.md)) — 该标志指示当前是否正在执行特定任务. + +- `num_tries` ([UInt32](../../sql-reference/data-types/int-uint.md)) — 尝试完成任务失败的次数. + +- `last_exception` ([String](../../sql-reference/data-types/string.md)) — 发生的最后一个错误的短信(如果有). + +- `last_attempt_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — 上次尝试任务的日期和时间. + +- `num_postponed` ([UInt32](../../sql-reference/data-types/int-uint.md)) — 延期任务数. + +- `postpone_reason` ([String](../../sql-reference/data-types/string.md)) — 任务延期的原因. + +- `last_postpone_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — 上次推迟任务的日期和时间. + +- `merge_type` ([String](../../sql-reference/data-types/string.md)) — 当前合并的类型. 如果是突变则为空. + +**示例** + +``` sql +SELECT * FROM system.replication_queue LIMIT 1 FORMAT Vertical; +``` + +``` text +Row 1: +────── +database: merge +table: visits_v2 +replica_name: mtgiga001-1t.metrika.yandex.net +position: 15 +node_name: queue-0009325559 +type: MERGE_PARTS +create_time: 2020-12-07 14:04:21 +required_quorum: 0 +source_replica: mtgiga001-1t.metrika.yandex.net +new_part_name: 20201130_121373_121384_2 +parts_to_merge: ['20201130_121373_121378_1','20201130_121379_121379_0','20201130_121380_121380_0','20201130_121381_121381_0','20201130_121382_121382_0','20201130_121383_121383_0','20201130_121384_121384_0'] +is_detach: 0 +is_currently_executing: 0 +num_tries: 36 +last_exception: Code: 226, e.displayText() = DB::Exception: Marks file '/opt/clickhouse/data/merge/visits_v2/tmp_fetch_20201130_121373_121384_2/CounterID.mrk' does not exist (version 20.8.7.15 (official build)) +last_attempt_time: 2020-12-08 17:35:54 +num_postponed: 0 +postpone_reason: +last_postpone_time: 1970-01-01 03:00:00 +``` + +**另请参阅** + +- [管理 ReplicatedMergeTree 表](../../sql-reference/statements/system.md#query-language-system-replicated) + +[原始文章](https://clickhouse.com/docs/en/operations/system_tables/replication_queue) From cde159a72917329753593afb1aaf00517af2814c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 14:49:57 +0800 Subject: [PATCH 159/396] Update replicated_fetches.md --- docs/zh/operations/system-tables/replicated_fetches.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/operations/system-tables/replicated_fetches.md b/docs/zh/operations/system-tables/replicated_fetches.md index 30846c49bb1..fededbaf28d 100644 --- a/docs/zh/operations/system-tables/replicated_fetches.md +++ b/docs/zh/operations/system-tables/replicated_fetches.md @@ -65,6 +65,6 @@ thread_id: 54 **另请参阅** -- [Managing ReplicatedMergeTree Tables](../../sql-reference/statements/system/#query-language-system-replicated) +- [管理 ReplicatedMergeTree 表](../../sql-reference/statements/system/#query-language-system-replicated) [原始文章](https://clickhouse.com/docs/en/operations/system_tables/replicated_fetches) From 20772fde046bd34b5b21d38fe26fd29e69f5edce Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 26 Oct 2021 15:22:13 +0300 Subject: [PATCH 160/396] For storage --- src/Storages/HDFS/StorageHDFS.cpp | 18 +++++++++++----- src/Storages/HDFS/StorageHDFS.h | 4 +++- src/Storages/PartitionedSink.h | 2 ++ src/Storages/StorageS3.cpp | 18 ++++++++++++---- src/Storages/StorageS3.h | 4 +++- src/Storages/StorageURL.cpp | 24 +++++++++++++++------ src/Storages/StorageURL.h | 7 ++++-- tests/integration/test_storage_hdfs/test.py | 10 +++++++++ tests/integration/test_storage_s3/test.py | 7 ++++++ 9 files changed, 75 insertions(+), 19 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index c5ad0fcc30c..b46668a233b 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include #include @@ -26,7 +27,6 @@ #include #include #include -#include #include #include @@ -61,8 +61,10 @@ StorageHDFS::StorageHDFS( const ConstraintsDescription & constraints_, const String & comment, ContextPtr context_, - const String & compression_method_ = "") + const String & compression_method_ = "", + ASTPtr partition_by_) : IStorage(table_id_), WithContext(context_), uri(uri_), format_name(format_name_), compression_method(compression_method_) + , partition_by(partition_by_) { context_->getRemoteHostFilter().checkURL(Poco::URI(uri)); @@ -370,12 +372,13 @@ SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataP { bool has_wildcards = uri.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; const auto * insert_query = dynamic_cast(query.get()); - bool is_partitioned_implementation = insert_query && insert_query->partition_by && has_wildcards; + auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; + bool is_partitioned_implementation = partition_by_ast && has_wildcards; if (is_partitioned_implementation) { return std::make_shared( - insert_query->partition_by, + partition_by_ast, uri, format_name, metadata_snapshot->getSampleBlock(), @@ -432,10 +435,15 @@ void registerStorageHDFS(StorageFactory & factory) compression_method = engine_args[2]->as().value.safeGet(); } else compression_method = "auto"; + ASTPtr partition_by; + if (args.storage_def->partition_by) + partition_by = args.storage_def->partition_by->clone(); + return StorageHDFS::create( - url, args.table_id, format_name, args.columns, args.constraints, args.comment, args.getContext(), compression_method); + url, args.table_id, format_name, args.columns, args.constraints, args.comment, args.getContext(), compression_method, partition_by); }, { + .supports_sort_order = true, // for partition by .source_access_type = AccessType::HDFS, }); } diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index e2ce1f1dcf8..bee8b7843d0 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -49,12 +49,14 @@ protected: const ConstraintsDescription & constraints_, const String & comment, ContextPtr context_, - const String & compression_method_); + const String & compression_method_, + ASTPtr partition_by = nullptr); private: const String uri; String format_name; String compression_method; + ASTPtr partition_by; Poco::Logger * log = &Poco::Logger::get("StorageHDFS"); }; diff --git a/src/Storages/PartitionedSink.h b/src/Storages/PartitionedSink.h index 53654b089ce..bc59a603fac 100644 --- a/src/Storages/PartitionedSink.h +++ b/src/Storages/PartitionedSink.h @@ -1,3 +1,5 @@ +#pragma once + #include #include #include diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index a17a8512fc2..0b7574621a4 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -464,7 +464,8 @@ StorageS3::StorageS3( ContextPtr context_, std::optional format_settings_, const String & compression_method_, - bool distributed_processing_) + bool distributed_processing_, + ASTPtr partition_by_) : IStorage(table_id_) , client_auth{uri_, access_key_id_, secret_access_key_, max_connections_, {}, {}} /// Client and settings will be updated later , format_name(format_name_) @@ -475,6 +476,7 @@ StorageS3::StorageS3( , name(uri_.storage_name) , distributed_processing(distributed_processing_) , format_settings(format_settings_) + , partition_by(partition_by_) { context_->getGlobalContext()->getRemoteHostFilter().checkURL(uri_.uri); StorageInMemoryMetadata storage_metadata; @@ -559,12 +561,13 @@ SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr bool has_wildcards = client_auth.uri.bucket.find(PARTITION_ID_WILDCARD) != String::npos || client_auth.uri.key.find(PARTITION_ID_WILDCARD) != String::npos; auto insert_query = std::dynamic_pointer_cast(query); - bool is_partitioned_implementation = insert_query && insert_query->partition_by && has_wildcards; + auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; + bool is_partitioned_implementation = partition_by_ast && has_wildcards; if (is_partitioned_implementation) { return std::make_shared( - insert_query->partition_by, + partition_by_ast, format_name, sample_block, local_context, @@ -746,6 +749,10 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) auto max_single_part_upload_size = args.getLocalContext()->getSettingsRef().s3_max_single_part_upload_size; auto max_connections = args.getLocalContext()->getSettingsRef().s3_max_connections; + ASTPtr partition_by; + if (args.storage_def->partition_by) + partition_by = args.storage_def->partition_by->clone(); + return StorageS3::create( s3_uri, configuration.access_key_id, @@ -761,10 +768,13 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) args.comment, args.getContext(), format_settings, - configuration.compression_method); + configuration.compression_method, + /* distributed_processing_ */false, + partition_by); }, { .supports_settings = true, + .supports_sort_order = true, // for partition by .source_access_type = AccessType::S3, }); } diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 89bbdda87f9..8ce287ff681 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -118,7 +118,8 @@ public: ContextPtr context_, std::optional format_settings_, const String & compression_method_ = "", - bool distributed_processing_ = false); + bool distributed_processing_ = false, + ASTPtr partition_by_ = nullptr); String getName() const override { @@ -169,6 +170,7 @@ private: String name; const bool distributed_processing; std::optional format_settings; + ASTPtr partition_by; static void updateClientAndAuthSettings(ContextPtr, ClientAuthentication &); }; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 46cc13b450d..328da6949be 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -49,7 +49,8 @@ IStorageURLBase::IStorageURLBase( const String & comment, const String & compression_method_, const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_, - const String & method_) + const String & method_, + ASTPtr partition_by_) : IStorage(table_id_) , uri(uri_) , compression_method(compression_method_) @@ -57,6 +58,7 @@ IStorageURLBase::IStorageURLBase( , format_settings(format_settings_) , headers(headers_) , method(method_) + , partition_by(partition_by_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); @@ -413,12 +415,13 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad bool has_wildcards = uri.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; const auto * insert_query = dynamic_cast(query.get()); - bool is_partitioned_implementation = insert_query && insert_query->partition_by && has_wildcards; + auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; + bool is_partitioned_implementation = partition_by_ast && has_wildcards; if (is_partitioned_implementation) { return std::make_shared( - insert_query->partition_by, + partition_by_ast, uri, format_name, format_settings, metadata_snapshot->getSampleBlock(), context, ConnectionTimeouts::getHTTPTimeouts(context), @@ -444,8 +447,10 @@ StorageURL::StorageURL( ContextPtr context_, const String & compression_method_, const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_, - const String & method_) - : IStorageURLBase(uri_, context_, table_id_, format_name_, format_settings_, columns_, constraints_, comment, compression_method_, headers_, method_) + const String & method_, + ASTPtr partition_by_) + : IStorageURLBase(uri_, context_, table_id_, format_name_, format_settings_, + columns_, constraints_, comment, compression_method_, headers_, method_, partition_by_) { context_->getRemoteHostFilter().checkURL(Poco::URI(uri)); } @@ -570,6 +575,10 @@ void registerStorageURL(StorageFactory & factory) headers.emplace_back(std::make_pair(header, value_literal)); } + ASTPtr partition_by; + if (args.storage_def->partition_by) + partition_by = args.storage_def->partition_by->clone(); + return StorageURL::create( configuration.url, args.table_id, @@ -580,10 +589,13 @@ void registerStorageURL(StorageFactory & factory) args.comment, args.getContext(), configuration.compression_method, - headers, configuration.method); + headers, + configuration.method, + partition_by); }, { .supports_settings = true, + .supports_sort_order = true, // for partition by .source_access_type = AccessType::URL, }); } diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index 42110cafd75..fcc0c231778 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -53,7 +53,8 @@ protected: const String & comment, const String & compression_method_, const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_ = {}, - const String & method_ = ""); + const String & method_ = "", + ASTPtr partition_by = nullptr); String uri; String compression_method; @@ -65,6 +66,7 @@ protected: std::optional format_settings; ReadWriteBufferFromHTTP::HTTPHeaderEntries headers; String method; /// For insert can choose Put instead of default Post. + ASTPtr partition_by; virtual std::string getReadMethod() const; @@ -127,7 +129,8 @@ public: ContextPtr context_, const String & compression_method_, const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_ = {}, - const String & method_ = ""); + const String & method_ = "", + ASTPtr partition_by_ = nullptr); String getName() const override { diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 22fb22cf60c..995f704262e 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -274,6 +274,16 @@ def test_partition_by(started_cluster): result = node1.query(f"select * from hdfs('hdfs://hdfs1:9000/test_3', 'TSV', '{table_format}')") assert(result.strip() == "1\t2\t3") + file_name = "test2_{_partition_id}" + node1.query(f"create table p(column1 UInt32, column2 UInt32, column3 UInt32) engine = HDFS('hdfs://hdfs1:9000/{file_name}', 'TSV') partition by column3") + node1.query(f"insert into p values {values}") + result = node1.query(f"select * from hdfs('hdfs://hdfs1:9000/test2_1', 'TSV', '{table_format}')") + assert(result.strip() == "3\t2\t1") + result = node1.query(f"select * from hdfs('hdfs://hdfs1:9000/test2_2', 'TSV', '{table_format}')") + assert(result.strip() == "1\t3\t2") + result = node1.query(f"select * from hdfs('hdfs://hdfs1:9000/test2_3', 'TSV', '{table_format}')") + assert(result.strip() == "1\t2\t3") + if __name__ == '__main__': cluster.start() diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 2f49b462d19..8889edec17c 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -163,6 +163,13 @@ def test_partition_by(started_cluster): assert "3,2,1\n" == get_s3_file_content(started_cluster, bucket, "test_1.csv") assert "78,43,45\n" == get_s3_file_content(started_cluster, bucket, "test_45.csv") + filename = "test2_{_partition_id}.csv" + instance.query(f"create table p ({table_format}) engine=S3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/{filename}', 'CSV') partition by column3") + instance.query(f"insert into p values {values}") + assert "1,2,3\n" == get_s3_file_content(started_cluster, bucket, "test2_3.csv") + assert "3,2,1\n" == get_s3_file_content(started_cluster, bucket, "test2_1.csv") + assert "78,43,45\n" == get_s3_file_content(started_cluster, bucket, "test2_45.csv") + def test_partition_by_string_column(started_cluster): bucket = started_cluster.minio_bucket From 8f4572ab8102aa44c2d2f970786047c7135e34df Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 15:09:22 +0800 Subject: [PATCH 161/396] Delete users.md --- docs/zh/operations/system-tables/users.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/users.md diff --git a/docs/zh/operations/system-tables/users.md b/docs/zh/operations/system-tables/users.md deleted file mode 120000 index 540c5ac6620..00000000000 --- a/docs/zh/operations/system-tables/users.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/users.md \ No newline at end of file From 217bc18a0c079ee155e215e34a76d5b723badbdc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 15:15:34 +0800 Subject: [PATCH 162/396] Create users.md --- docs/zh/operations/system-tables/users.md | 34 +++++++++++++++++++++++ 1 file changed, 34 insertions(+) create mode 100644 docs/zh/operations/system-tables/users.md diff --git a/docs/zh/operations/system-tables/users.md b/docs/zh/operations/system-tables/users.md new file mode 100644 index 00000000000..521c54de801 --- /dev/null +++ b/docs/zh/operations/system-tables/users.md @@ -0,0 +1,34 @@ +# system.users {#system_tables-users} + +包含服务器上配置的[用户账号](../../operations/access-rights.md#user-account-management)的列表. + +列信息: +- `name` ([String](../../sql-reference/data-types/string.md)) — 用户名称. + +- `id` ([UUID](../../sql-reference/data-types/uuid.md)) — 用户 ID. + +- `storage` ([String](../../sql-reference/data-types/string.md)) — 用户存储路径. 在 `access_control_path` 参数中配置. + +- `auth_type` ([Enum8](../../sql-reference/data-types/enum.md)('no_password' = 0,'plaintext_password' = 1, 'sha256_password' = 2, 'double_sha1_password' = 3)) — 显示认证类型. 有多种用户识别方式: 无密码, 纯文本密码, [SHA256](https://ru.wikipedia.org/wiki/SHA-2)-encoded password or with [double SHA-1](https://ru.wikipedia.org/wiki/SHA-1)-编码的密码. + +- `auth_params` ([String](../../sql-reference/data-types/string.md)) — JSON 格式的身份验证参数取决于`auth_type`. + +- `host_ip` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — 许连接到 ClickHouse 服务器的主机的 IP 地址. + +- `host_names` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — 允许连接到 ClickHouse 服务器的主机名称. + +- `host_names_regexp` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — 允许连接到 ClickHouse 服务器的主机名的正则表达式. + +- `host_names_like` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — 使用 LIKE 谓词设置允许连接到 ClickHouse 服务器的主机名称. + +- `default_roles_all` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 显示默认情况下为用户设置的所有授予的角色. + +- `default_roles_list` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — 默认提供的授权角色列表. + +- `default_roles_except` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — 除了列出的角色之外所有授予的角色都设置为默认值. + +## 另请参阅 {#see-also} + +- [查看用户信息](../../sql-reference/statements/show.md#show-users-statement) + +[原始文章](https://clickhouse.com/docs/en/operations/system-tables/users) From c2d9a14887052307fc74b81af8cdef154f6cd948 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 15:19:18 +0800 Subject: [PATCH 163/396] Delete time_zones.md --- docs/zh/operations/system-tables/time_zones.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/time_zones.md diff --git a/docs/zh/operations/system-tables/time_zones.md b/docs/zh/operations/system-tables/time_zones.md deleted file mode 120000 index d7b0f07d326..00000000000 --- a/docs/zh/operations/system-tables/time_zones.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/time_zones.md \ No newline at end of file From fc02711b88e8e0d91bb1d9a42d64df6ca48ee25a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 15:20:12 +0800 Subject: [PATCH 164/396] Create time_zones.md --- .../zh/operations/system-tables/time_zones.md | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) create mode 100644 docs/zh/operations/system-tables/time_zones.md diff --git a/docs/zh/operations/system-tables/time_zones.md b/docs/zh/operations/system-tables/time_zones.md new file mode 100644 index 00000000000..d01725edf5b --- /dev/null +++ b/docs/zh/operations/system-tables/time_zones.md @@ -0,0 +1,30 @@ +# system.time_zones {#system-time_zones} + +包含 ClickHouse 服务器支持的时区列表. 此时区列表可能因 ClickHouse 的版本而异 + +列信息: + +- `time_zone` (String) — List of supported time zones. + +**示例** + +``` sql +SELECT * FROM system.time_zones LIMIT 10 +``` + +``` text +┌─time_zone──────────┐ +│ Africa/Abidjan │ +│ Africa/Accra │ +│ Africa/Addis_Ababa │ +│ Africa/Algiers │ +│ Africa/Asmara │ +│ Africa/Asmera │ +│ Africa/Bamako │ +│ Africa/Bangui │ +│ Africa/Banjul │ +│ Africa/Bissau │ +└────────────────────┘ +``` + +[原始文章](https://clickhouse.com/docs/en/operations/system-tables/time_zones) From 9edfcdb4bbfc2a387393451b02142e775c07387a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 15:20:40 +0800 Subject: [PATCH 165/396] Delete stack_trace.md --- docs/zh/operations/system-tables/stack_trace.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/stack_trace.md diff --git a/docs/zh/operations/system-tables/stack_trace.md b/docs/zh/operations/system-tables/stack_trace.md deleted file mode 120000 index 8dea20028f1..00000000000 --- a/docs/zh/operations/system-tables/stack_trace.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/stack_trace.md \ No newline at end of file From 074a46c1d15e07d8262cdf00985dbfbb81763040 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 15:29:50 +0800 Subject: [PATCH 166/396] Create stack_trace.md --- .../operations/system-tables/stack_trace.md | 91 +++++++++++++++++++ 1 file changed, 91 insertions(+) create mode 100644 docs/zh/operations/system-tables/stack_trace.md diff --git a/docs/zh/operations/system-tables/stack_trace.md b/docs/zh/operations/system-tables/stack_trace.md new file mode 100644 index 00000000000..8e46efd17ab --- /dev/null +++ b/docs/zh/operations/system-tables/stack_trace.md @@ -0,0 +1,91 @@ +# system.stack_trace {#system-tables_stack_trace} + +包含所有服务器线程的堆栈跟踪. 允许开发人员对服务器状态进行自省. + +要分析堆栈帧, 请使用 `addressToLine`, `addressToSymbol` and `demangle` [内省函数](../../sql-reference/functions/introspection.md). + +列信息: + +- `thread_name` ([String](../../sql-reference/data-types/string.md)) — 线程名称. +- `thread_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 线程标识符. +- `query_id` ([String](../../sql-reference/data-types/string.md)) — 用于获取从[query_log](../system-tables/query_log.md) 系统表运行的查询的详细信息查询标识符. +- `trace` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — [堆栈跟踪](https://en.wikipedia.org/wiki/Stack_trace) 表示存储调用方法的物理地址列表. + +**示例** + +启用内省功能: + +``` sql +SET allow_introspection_functions = 1; +``` + +从 ClickHouse 目标文件中获取符号: + +``` sql +WITH arrayMap(x -> demangle(addressToSymbol(x)), trace) AS all SELECT thread_name, thread_id, query_id, arrayStringConcat(all, '\n') AS res FROM system.stack_trace LIMIT 1 \G; +``` + +``` text +Row 1: +────── +thread_name: clickhouse-serv + +thread_id: 686 +query_id: 1a11f70b-626d-47c1-b948-f9c7b206395d +res: sigqueue +DB::StorageSystemStackTrace::fillData(std::__1::vector::mutable_ptr, std::__1::allocator::mutable_ptr > >&, DB::Context const&, DB::SelectQueryInfo const&) const +DB::IStorageSystemOneBlock::read(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&, DB::SelectQueryInfo const&, DB::Context const&, DB::QueryProcessingStage::Enum, unsigned long, unsigned int) +DB::InterpreterSelectQuery::executeFetchColumns(DB::QueryProcessingStage::Enum, DB::QueryPipeline&, std::__1::shared_ptr const&, std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&) +DB::InterpreterSelectQuery::executeImpl(DB::QueryPipeline&, std::__1::shared_ptr const&, std::__1::optional) +DB::InterpreterSelectQuery::execute() +DB::InterpreterSelectWithUnionQuery::execute() +DB::executeQueryImpl(char const*, char const*, DB::Context&, bool, DB::QueryProcessingStage::Enum, bool, DB::ReadBuffer*) +DB::executeQuery(std::__1::basic_string, std::__1::allocator > const&, DB::Context&, bool, DB::QueryProcessingStage::Enum, bool) +DB::TCPHandler::runImpl() +DB::TCPHandler::run() +Poco::Net::TCPServerConnection::start() +Poco::Net::TCPServerDispatcher::run() +Poco::PooledThread::run() +Poco::ThreadImpl::runnableEntry(void*) +start_thread +__clone +``` + +在 ClickHouse 源代码中获取文件名和行号: + +``` sql +WITH arrayMap(x -> addressToLine(x), trace) AS all, arrayFilter(x -> x LIKE '%/dbms/%', all) AS dbms SELECT thread_name, thread_id, query_id, arrayStringConcat(notEmpty(dbms) ? dbms : all, '\n') AS res FROM system.stack_trace LIMIT 1 \G; +``` + +``` text +Row 1: +────── +thread_name: clickhouse-serv + +thread_id: 686 +query_id: cad353e7-1c29-4b2e-949f-93e597ab7a54 +res: /lib/x86_64-linux-gnu/libc-2.27.so +/build/obj-x86_64-linux-gnu/../src/Storages/System/StorageSystemStackTrace.cpp:182 +/build/obj-x86_64-linux-gnu/../contrib/libcxx/include/vector:656 +/build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterSelectQuery.cpp:1338 +/build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterSelectQuery.cpp:751 +/build/obj-x86_64-linux-gnu/../contrib/libcxx/include/optional:224 +/build/obj-x86_64-linux-gnu/../src/Interpreters/InterpreterSelectWithUnionQuery.cpp:192 +/build/obj-x86_64-linux-gnu/../src/Interpreters/executeQuery.cpp:384 +/build/obj-x86_64-linux-gnu/../src/Interpreters/executeQuery.cpp:643 +/build/obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:251 +/build/obj-x86_64-linux-gnu/../src/Server/TCPHandler.cpp:1197 +/build/obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerConnection.cpp:57 +/build/obj-x86_64-linux-gnu/../contrib/libcxx/include/atomic:856 +/build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/include/Poco/Mutex_POSIX.h:59 +/build/obj-x86_64-linux-gnu/../contrib/poco/Foundation/include/Poco/AutoPtr.h:223 +/lib/x86_64-linux-gnu/libpthread-2.27.so +/lib/x86_64-linux-gnu/libc-2.27.so +``` + +**另请参阅** + +- [Introspection Functions](../../sql-reference/functions/introspection.md) — 哪些内省功能是可用的以及如何使用它们. +- [system.trace_log](../system-tables/trace_log.md) — 包含由抽样查询分析器收集的堆栈跟踪. +- [arrayMap](../../sql-reference/functions/array-functions.md#array-map) — `arrayMap` 函数的描述和使用示例 +- [arrayFilter](../../sql-reference/functions/array-functions.md#array-filter) — `arrayFilter` 函数的描述和使用示例. From 4301f5c2a78845dcdee2842d03b020e45c1a12c4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 15:30:22 +0800 Subject: [PATCH 167/396] Delete settings_profiles.md --- docs/zh/operations/system-tables/settings_profiles.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/settings_profiles.md diff --git a/docs/zh/operations/system-tables/settings_profiles.md b/docs/zh/operations/system-tables/settings_profiles.md deleted file mode 120000 index 1c559bf2445..00000000000 --- a/docs/zh/operations/system-tables/settings_profiles.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/settings_profiles.md \ No newline at end of file From 110e41c1ce9199ff6b29323df5dfccd9af225c31 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 15:37:30 +0800 Subject: [PATCH 168/396] Create settings_profiles.md --- .../system-tables/settings_profiles.md | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) create mode 100644 docs/zh/operations/system-tables/settings_profiles.md diff --git a/docs/zh/operations/system-tables/settings_profiles.md b/docs/zh/operations/system-tables/settings_profiles.md new file mode 100644 index 00000000000..46102f9c2a6 --- /dev/null +++ b/docs/zh/operations/system-tables/settings_profiles.md @@ -0,0 +1,24 @@ +# system.settings_profiles {#system_tables-settings_profiles} + +包含 Setting 配置文件中指定的属性. + +列信息: +- `name` ([String](../../sql-reference/data-types/string.md)) — Setting 配置文件 name. + +- `id` ([UUID](../../sql-reference/data-types/uuid.md)) — Setting 配置文件 ID. + +- `storage` ([String](../../sql-reference/data-types/string.md)) — Setting 配置文件的存储路径. 在`access_control_path`参数中配置. + +- `num_elements` ([UInt64](../../sql-reference/data-types/int-uint.md)) — `system.settings_profile_elements` 表中此配置文件的元素数. + +- `apply_to_all` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 为所有角色和/或用户设置的 Setting 配置文件. + +- `apply_to_list` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — 应用 Setting 配置文件的角色和/或用户列表. + +- `apply_to_except` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Setting 配置文件适用于除所列角色和/或用户之外的所有角色和/或用户. + +## 另请参阅 {#see-also} + +- [查看配置文件信息](../../sql-reference/statements/show.md#show-profiles-statement) + +[原始文章](https://clickhouse.com/docs/en/operations/system-tables/settings_profiles) From e6fbf2f7a85efb4141e6d36cdc4eb471a960934f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 15:37:50 +0800 Subject: [PATCH 169/396] Delete settings_profile_elements.md --- docs/zh/operations/system-tables/settings_profile_elements.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/settings_profile_elements.md diff --git a/docs/zh/operations/system-tables/settings_profile_elements.md b/docs/zh/operations/system-tables/settings_profile_elements.md deleted file mode 120000 index 3b0e70ead49..00000000000 --- a/docs/zh/operations/system-tables/settings_profile_elements.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/settings_profile_elements.md \ No newline at end of file From 92b04c74f4dc7ff4c354633086d3a7b7ae5f8b8e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 15:45:01 +0800 Subject: [PATCH 170/396] Create settings_profile_elements.md --- .../settings_profile_elements.md | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) create mode 100644 docs/zh/operations/system-tables/settings_profile_elements.md diff --git a/docs/zh/operations/system-tables/settings_profile_elements.md b/docs/zh/operations/system-tables/settings_profile_elements.md new file mode 100644 index 00000000000..2f5eb5b3044 --- /dev/null +++ b/docs/zh/operations/system-tables/settings_profile_elements.md @@ -0,0 +1,30 @@ +# system.settings_profile_elements {#system_tables-settings_profile_elements} + +描述settings配置文件的内容: + +- 约束. +- setting适用的角色和用户. +- 父级 setting 配置文件. + +列信息: +- `profile_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — Setting 配置文件名称. + +- `user_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — 用户名称. + +- `role_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — 角色名称. + +- `index` ([UInt64](../../sql-reference/data-types/int-uint.md)) — settings 配置文件元素的顺序编号. + +- `setting_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — Setting 名称. + +- `value` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — Setting 值. + +- `min` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — setting 最小值. 未设置则赋 `NULL`. + +- `max` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — setting 最大值. 未设置则赋 `NULL`. + +- `readonly` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges))) — 只允许读查询的配置文件. + +- `inherit_profile` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — 此setting配置文件的父配置文件. 未设置则赋 `NULL`. 设置则将从其父配置文件继承所有设置的值和约束(`min`、`max`、`readonly`). + +[原始文章](https://clickhouse.com/docs/en/operations/system-tables/settings_profile_elements) From 6a2550e1f37dc5ef5304c296186513ca661bbe4a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 15:45:29 +0800 Subject: [PATCH 171/396] Delete role-grants.md --- docs/zh/operations/system-tables/role-grants.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/role-grants.md diff --git a/docs/zh/operations/system-tables/role-grants.md b/docs/zh/operations/system-tables/role-grants.md deleted file mode 120000 index 6a25ffa31ce..00000000000 --- a/docs/zh/operations/system-tables/role-grants.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/role-grants.md \ No newline at end of file From 0a3cb819a6b52853a96c24ecf1d70a24434b7d74 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 15:50:10 +0800 Subject: [PATCH 172/396] Create role-grants.md --- .../operations/system-tables/role-grants.md | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) create mode 100644 docs/zh/operations/system-tables/role-grants.md diff --git a/docs/zh/operations/system-tables/role-grants.md b/docs/zh/operations/system-tables/role-grants.md new file mode 100644 index 00000000000..8957c73df73 --- /dev/null +++ b/docs/zh/operations/system-tables/role-grants.md @@ -0,0 +1,21 @@ +#system.role_grants {#system_tables-role_grants} + +包含用户和角色的角色授予. 向该表添加项, 请使用`GRANT role TO user`. + +列信息: + +- `user_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — 用户名称. + +- `role_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — 角色名称. + +- `granted_role_name` ([String](../../sql-reference/data-types/string.md)) — 授予 `role_name` 角色的角色名称. 要将一个角色授予另一个角色, 请使用`GRANT role1 TO role2`. + +- `granted_role_is_default` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 显示 `granted_role` 是否为默认角色的标志. 参考值: + - 1 — `granted_role` is a default role. + - 0 — `granted_role` is not a default role. + +- `with_admin_option` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 显示 `granted_role` 是否是具有 [ADMIN OPTION](../../sql-reference/statements/grant.md#admin-option-privilege) 特权的角色的标志. 参考值: + - 1 — 该角色具有 `ADMIN OPTION` 权限. + - 0 — 该角色不具有 `ADMIN OPTION` 权限. + +[原始文章](https://clickhouse.com/docs/en/operations/system-tables/role-grants) From e297c23348430645816846aea4b227e8ea12a322 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 15:50:36 +0800 Subject: [PATCH 173/396] Delete roles.md --- docs/zh/operations/system-tables/roles.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/roles.md diff --git a/docs/zh/operations/system-tables/roles.md b/docs/zh/operations/system-tables/roles.md deleted file mode 120000 index 391bc980a48..00000000000 --- a/docs/zh/operations/system-tables/roles.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/roles.md \ No newline at end of file From 1daf9a96117e95c1b3c5f1de63f709a26d35ea7a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 15:52:32 +0800 Subject: [PATCH 174/396] Create roles.md --- docs/zh/operations/system-tables/roles.md | 15 +++++++++++++++ 1 file changed, 15 insertions(+) create mode 100644 docs/zh/operations/system-tables/roles.md diff --git a/docs/zh/operations/system-tables/roles.md b/docs/zh/operations/system-tables/roles.md new file mode 100644 index 00000000000..c3537b978e4 --- /dev/null +++ b/docs/zh/operations/system-tables/roles.md @@ -0,0 +1,15 @@ +# system.roles {#system_tables-roles} + +包含有关已配置的 [角色](../../operations/access-rights.md#role-management) 信息. + +列信息: + +- `name` ([String](../../sql-reference/data-types/string.md)) — 角色名称. +- `id` ([UUID](../../sql-reference/data-types/uuid.md)) — 角色 ID. +- `storage` ([String](../../sql-reference/data-types/string.md)) — 角色存储的路径. 在 `access_control_path` 参数中配置. + +## 另请参阅 {#see-also} + +- [查看角色信息](../../sql-reference/statements/show.md#show-roles-statement) + +[原始文章](https://clickhouse.com/docs/en/operations/system-tables/roles) From c2bd68cfc99c82e8c90a9ee4b952f4837b8a9a6c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 15:53:01 +0800 Subject: [PATCH 175/396] Delete row_policies.md --- docs/zh/operations/system-tables/row_policies.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/row_policies.md diff --git a/docs/zh/operations/system-tables/row_policies.md b/docs/zh/operations/system-tables/row_policies.md deleted file mode 120000 index b194161cf1d..00000000000 --- a/docs/zh/operations/system-tables/row_policies.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/row_policies.md \ No newline at end of file From 30886230e9d5f0f99475b0da40be225acd0736bb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 15:59:11 +0800 Subject: [PATCH 176/396] Create row_policies.md --- .../operations/system-tables/row_policies.md | 34 +++++++++++++++++++ 1 file changed, 34 insertions(+) create mode 100644 docs/zh/operations/system-tables/row_policies.md diff --git a/docs/zh/operations/system-tables/row_policies.md b/docs/zh/operations/system-tables/row_policies.md new file mode 100644 index 00000000000..4eaf291cc68 --- /dev/null +++ b/docs/zh/operations/system-tables/row_policies.md @@ -0,0 +1,34 @@ +# system.row_policies {#system_tables-row_policies} + +包含一个特定表的过滤器, 以及应该使用此行策略的角色和/或用户列表. + +列信息: +- `name` ([String](../../sql-reference/data-types/string.md)) — 行策略的名称. + +- `short_name` ([String](../../sql-reference/data-types/string.md)) — 行策略的短名称. 行策略的名称是复合的,例如:myfilter ON mydb.mytable. 这里 "myfilter ON mydb.mytable" 是行策略的名称, "myfilter" 是它的简称. + +- `database` ([String](../../sql-reference/data-types/string.md)) — 数据库名称. + +- `table` ([String](../../sql-reference/data-types/string.md)) — 表名称. + +- `id` ([UUID](../../sql-reference/data-types/uuid.md)) — 行策略 ID. + +- `storage` ([String](../../sql-reference/data-types/string.md)) — 存储行策略的目录名. + +- `select_filter` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — 用于过滤行的条件. + +- `is_restrictive` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 显示行策略是否限制对行的访问, 参考 [CREATE ROW POLICY](../../sql-reference/statements/create/row-policy.md#create-row-policy-as). 值: +- `0` — 行策略使用 `AS PERMISSIVE` 子句定义. +- `1` — 行策略使用 `AS RESTRICTIVE` 子句定义. + +- `apply_to_all` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — 显示为所有角色和/或用户设置的行策略. + +- `apply_to_list` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — 应用行策略的角色和/或用户列表. + +- `apply_to_except` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — 行策略应用于除列出的角色和/或用户之外的所有角色和/或用户. + +## 另请参阅 {#see-also} + +- [SHOW POLICIES](../../sql-reference/statements/show.md#show-policies-statement) + +[原始文章](https://clickhouse.com/docs/en/operations/system-tables/row_policies) From c2bfa811eda6feff550ed71206218e5be3ca8b9e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 15:59:31 +0800 Subject: [PATCH 177/396] Delete zookeeper_log.md --- docs/zh/operations/system-tables/zookeeper_log.md | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/operations/system-tables/zookeeper_log.md diff --git a/docs/zh/operations/system-tables/zookeeper_log.md b/docs/zh/operations/system-tables/zookeeper_log.md deleted file mode 120000 index c7db82e978a..00000000000 --- a/docs/zh/operations/system-tables/zookeeper_log.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/operations/system-tables/zookeeper_log.md \ No newline at end of file From 9a70475f2ba6da97c275cecbaed1a6ef850a5580 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9C=A8=E6=9C=A8=E5=A4=95120?= Date: Wed, 27 Oct 2021 16:14:02 +0800 Subject: [PATCH 178/396] Create zookeeper_log.md --- .../operations/system-tables/zookeeper_log.md | 129 ++++++++++++++++++ 1 file changed, 129 insertions(+) create mode 100644 docs/zh/operations/system-tables/zookeeper_log.md diff --git a/docs/zh/operations/system-tables/zookeeper_log.md b/docs/zh/operations/system-tables/zookeeper_log.md new file mode 100644 index 00000000000..678e2b8a93b --- /dev/null +++ b/docs/zh/operations/system-tables/zookeeper_log.md @@ -0,0 +1,129 @@ +# system.zookeeper_log {#system-zookeeper_log} + +此表包含有关对 ZooKeeper 服务器的请求及其响应的参数的信息. + +对于请求,只填充有请求参数的列,其余列填充默认值 (`0` or `NULL`). 当响应到达时,来自响应的数据被添加到其他列. + +带有请求参数的列: + +- `type` ([Enum](../../sql-reference/data-types/enum.md)) — ZooKeeper 客户端中的事件类型. 可以具有以下值之一: + - `Request` — 请求已发送. + - `Response` — 已收到回复. + - `Finalize` — 连接丢失, 未收到响应. +- `event_date` ([Date](../../sql-reference/data-types/date.md)) — 事件发生的日期. +- `event_time` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — 事件发生的日期和时间. +- `address` ([IPv6](../../sql-reference/data-types/domains/ipv6.md)) — 用于发出请求的 ZooKeeper 服务器的 IP 地址. +- `port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — 用于发出请求的 ZooKeeper 服务器的端口. +- `session_id` ([Int64](../../sql-reference/data-types/int-uint.md)) — ZooKeeper 服务器为每个连接设置的会话 ID. +- `xid` ([Int32](../../sql-reference/data-types/int-uint.md)) — 会话中请求的 ID. 这通常是一个连续的请求编号. 请求行和配对的 `response`/`finalize` 行相同. +- `has_watch` ([UInt8](../../sql-reference/data-types/int-uint.md)) — 请求是否设置了 [watch](https://zookeeper.apache.org/doc/r3.3.3/zookeeperProgrammers.html#ch_zkWatches) . +- `op_num` ([Enum](../../sql-reference/data-types/enum.md)) — 请求或响应的类型. +- `path` ([String](../../sql-reference/data-types/string.md)) — 请求中指定的 ZooKeeper 节点的路径, 如果请求不需要指定路径, 则为空字符串. +- `data` ([String](../../sql-reference/data-types/string.md)) — 写入 ZooKeeper 节点的数据(对于 `SET` 和 `CREATE` 请求 - 请求想要写入的内容,对于 `GET` 请求的响应 - 读取的内容)或空字符串. +- `is_ephemeral` ([UInt8](../../sql-reference/data-types/int-uint.md)) — ZooKeeper 节点是否被创建为 [ephemeral](https://zookeeper.apache.org/doc/r3.3.3/zookeeperProgrammers.html#Ephemeral+Nodes). +- `is_sequential` ([UInt8](../../sql-reference/data-types/int-uint.md)) — ZooKeeper 节点是否被创建为 [sequential](https://zookeeper.apache.org/doc/r3.3.3/zookeeperProgrammers.html#Sequence+Nodes+--+Unique+Naming). +- `version` ([Nullable(Int32)](../../sql-reference/data-types/nullable.md)) — 请求执行时期望的 ZooKeeper 节点的版本. 这支持`CHECK`、`SET`、`REMOVE`请求(如果请求不检查版本, 则为相关的`-1`或不支持版本检查的其他请求的`NULL`). +- `requests_size` ([UInt32](../../sql-reference/data-types/int-uint.md)) —多请求中包含的请求数(这是一个特殊的请求,由几个连续的普通请求组成, 并以原子方式执行). 多请求中包含的所有请求都将具有相同的 `xid`. +- `request_idx` ([UInt32](../../sql-reference/data-types/int-uint.md)) — 包含在多请求中的请求数(对于多请求 — `0`,然后从 `1` 开始). + +带有请求响应参数的列: + +- `zxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — ZooKeeper 事务 ID. ZooKeeper 服务器响应成功执行的请求而发出的序列号(`0` 表示请求没有执行/返回错误/客户端不知道请求是否被执行). +- `error` ([Nullable(Enum)](../../sql-reference/data-types/nullable.md)) — 错误代码. 可以有很多值, 这里只是其中的一些: + - `ZOK` — 请求被安全执行. + - `ZCONNECTIONLOSS` — 连接丢失. + - `ZOPERATIONTIMEOUT` — 请求执行超时已过期. + - `ZSESSIONEXPIRED` — 会话已过期. + - `NULL` — 请求完成. +- `watch_type` ([Nullable(Enum)](../../sql-reference/data-types/nullable.md)) — `watch` 事件的类型(对于带有 `op_num` = `Watch` 的响应), 对于其余响应:`NULL`. +- `watch_state` ([Nullable(Enum)](../../sql-reference/data-types/nullable.md)) — `watch` 事件的状态(对于带有 `op_num` = `Watch` 的响应), 对于其余响应:`NULL`. +- `path_created` ([String](../../sql-reference/data-types/string.md)) — 创建的 ZooKeeper 节点的路径(用于响应 `CREATE` 请求),如果节点被创建为 `sequential`, 则可能与 `path` 不同. +- `stat_czxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — 导致创建此 ZooKeeper 节点的更改的 `zxid`. +- `stat_mzxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — 最后一次修改该ZooKeeper节点的 `zxid`. +- `stat_pzxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — 最后一次修改该ZooKeeper节点的子节点的事务ID +- `stat_version` ([Int32](../../sql-reference/data-types/int-uint.md)) — 此 ZooKeeper 节点的数据更改次数. +- `stat_cversion` ([Int32](../../sql-reference/data-types/int-uint.md)) — 此 ZooKeeper 节点的子节点的更改次数. +- `stat_dataLength` ([Int32](../../sql-reference/data-types/int-uint.md)) — 这个 ZooKeeper 节点的数据字段的长度. +- `stat_numChildren` ([Int32](../../sql-reference/data-types/int-uint.md)) — 此 ZooKeeper 节点的子节点数. +- `children` ([Array(String)](../../sql-reference/data-types/array.md)) — ZooKeeper 子节点列表(用于响应 `LIST` 请求). + +**示例** + +查询: + +``` sql +SELECT * FROM system.zookeeper_log WHERE (session_id = '106662742089334927') AND (xid = '10858') FORMAT Vertical; +``` + +结果: + +``` text +Row 1: +────── +type: Request +event_date: 2021-08-09 +event_time: 2021-08-09 21:38:30.291792 +address: :: +port: 2181 +session_id: 106662742089334927 +xid: 10858 +has_watch: 1 +op_num: List +path: /clickhouse/task_queue/ddl +data: +is_ephemeral: 0 +is_sequential: 0 +version: ᴺᵁᴸᴸ +requests_size: 0 +request_idx: 0 +zxid: 0 +error: ᴺᵁᴸᴸ +watch_type: ᴺᵁᴸᴸ +watch_state: ᴺᵁᴸᴸ +path_created: +stat_czxid: 0 +stat_mzxid: 0 +stat_pzxid: 0 +stat_version: 0 +stat_cversion: 0 +stat_dataLength: 0 +stat_numChildren: 0 +children: [] + +Row 2: +────── +type: Response +event_date: 2021-08-09 +event_time: 2021-08-09 21:38:30.292086 +address: :: +port: 2181 +session_id: 106662742089334927 +xid: 10858 +has_watch: 1 +op_num: List +path: /clickhouse/task_queue/ddl +data: +is_ephemeral: 0 +is_sequential: 0 +version: ᴺᵁᴸᴸ +requests_size: 0 +request_idx: 0 +zxid: 16926267 +error: ZOK +watch_type: ᴺᵁᴸᴸ +watch_state: ᴺᵁᴸᴸ +path_created: +stat_czxid: 16925469 +stat_mzxid: 16925469 +stat_pzxid: 16926179 +stat_version: 0 +stat_cversion: 7 +stat_dataLength: 0 +stat_numChildren: 7 +children: ['query-0000000006','query-0000000005','query-0000000004','query-0000000003','query-0000000002','query-0000000001','query-0000000000'] +``` + +**另请参阅** + +- [ZooKeeper](../../operations/tips.md#zookeeper) +- [ZooKeeper 指南](https://zookeeper.apache.org/doc/r3.3.3/zookeeperProgrammers.html) From d1ef96a5efec518626baae177a8c8185d2b87b29 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 27 Oct 2021 17:29:15 +0300 Subject: [PATCH 179/396] Add test, avoid unnecessary allocations, use PeekableReadBuffer only in corner case --- docs/en/interfaces/formats.md | 5 +- docs/en/operations/settings/settings.md | 16 +-- src/Core/Settings.h | 1 - .../Serializations/SerializationNullable.cpp | 61 +++++++-- src/Formats/FormatFactory.cpp | 1 - src/Formats/FormatSettings.h | 1 - src/IO/PeekableReadBuffer.cpp | 4 +- src/IO/PeekableReadBuffer.h | 7 +- ...v_csv_custom_null_representation.reference | 55 ++++++++ ...2103_tsv_csv_custom_null_representation.sh | 125 ++++++++++++++++++ ...104_json_strings_nullable_string.reference | 2 + .../02104_json_strings_nullable_string.sh | 17 +++ 12 files changed, 268 insertions(+), 27 deletions(-) create mode 100644 tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.reference create mode 100755 tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.sh create mode 100644 tests/queries/0_stateless/02104_json_strings_nullable_string.reference create mode 100755 tests/queries/0_stateless/02104_json_strings_nullable_string.sh diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index d0e5c44b4f7..57459152129 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -124,7 +124,8 @@ Only a small set of symbols are escaped. You can easily stumble onto a string va Arrays are written as a list of comma-separated values in square brackets. Number items in the array are formatted as normally. `Date` and `DateTime` types are written in single quotes. Strings are written in single quotes with the same escaping rules as above. -[NULL](../sql-reference/syntax.md) is formatted as `\N`. +[NULL](../sql-reference/syntax.md) is formatted according to setting [format_tsv_null_representation](../operations/settings/settings.md#settings-format_tsv_null_representation) (default value is `\N`). + Each element of [Nested](../sql-reference/data-types/nested-data-structures/nested.md) structures is represented as array. @@ -380,7 +381,7 @@ Empty unquoted input values are replaced with default values for the respective [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) is enabled. -`NULL` is formatted as `\N` or `NULL` or an empty unquoted string (see settings [input_format_csv_unquoted_null_literal_as_null](../operations/settings/settings.md#settings-input_format_csv_unquoted_null_literal_as_null) and [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields)). +`NULL` is formatted according to setting [format_csv_null_representation](../operations/settings/settings.md#settings-format_csv_null_representation) (default value is `\N`). The CSV format supports the output of totals and extremes the same way as `TabSeparated`. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2b088bf45bf..b10d649a952 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1512,10 +1512,6 @@ When `output_format_json_quote_denormals = 1`, the query returns: The character is interpreted as a delimiter in the CSV data. By default, the delimiter is `,`. -## input_format_csv_unquoted_null_literal_as_null {#settings-input_format_csv_unquoted_null_literal_as_null} - -For CSV input format enables or disables parsing of unquoted `NULL` as literal (synonym for `\N`). - ## input_format_csv_enum_as_number {#settings-input_format_csv_enum_as_number} Enables or disables parsing enum values as enum ids for CSV input format. @@ -2859,9 +2855,9 @@ Possible values: Default value: `1`. -## output_format_csv_null_representation {#output_format_csv_null_representation} +## format_csv_null_representation {#format_csv_null_representation} -Defines the representation of `NULL` for [CSV](../../interfaces/formats.md#csv) output format. User can set any string as a value, for example, `My NULL`. +Defines the representation of `NULL` for [CSV](../../interfaces/formats.md#csv) output and input formats. User can set any string as a value, for example, `My NULL`. Default value: `\N`. @@ -2884,7 +2880,7 @@ Result Query ```sql -SET output_format_csv_null_representation = 'My NULL'; +SET format_csv_null_representation = 'My NULL'; SELECT * FROM csv_custom_null FORMAT CSV; ``` @@ -2896,9 +2892,9 @@ My NULL My NULL ``` -## output_format_tsv_null_representation {#output_format_tsv_null_representation} +## format_tsv_null_representation {#format_tsv_null_representation} -Defines the representation of `NULL` for [TSV](../../interfaces/formats.md#tabseparated) output format. User can set any string as a value, for example, `My NULL`. +Defines the representation of `NULL` for [TSV](../../interfaces/formats.md#tabseparated) output and input formats. User can set any string as a value, for example, `My NULL`. Default value: `\N`. @@ -2921,7 +2917,7 @@ Result Query ```sql -SET output_format_tsv_null_representation = 'My NULL'; +SET format_tsv_null_representation = 'My NULL'; SELECT * FROM tsv_custom_null FORMAT TSV; ``` diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 402e2b2f6a4..ba0ffdb6f33 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -552,7 +552,6 @@ class IColumn; M(Bool, format_csv_allow_single_quotes, true, "If it is set to true, allow strings in single quotes.", 0) \ M(Bool, format_csv_allow_double_quotes, true, "If it is set to true, allow strings in double quotes.", 0) \ M(Bool, output_format_csv_crlf_end_of_line, false, "If it is set true, end of line in CSV format will be \\r\\n instead of \\n.", 0) \ - 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) \ diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index 75f9d40740b..0dbb140af8b 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -261,6 +261,7 @@ ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, R { const String & null_representation = settings.tsv.null_representation; + /// Some data types can deserialize absence of data (e.g. empty string), so eof is ok. if (istr.eof() || (!null_representation.empty() && *istr.position() != null_representation[0])) { /// This is not null, surely. @@ -269,7 +270,28 @@ ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, R [&nested, &istr, &settings] (IColumn & nested_column) { nested->deserializeTextEscaped(nested_column, istr, settings); }); } - PeekableReadBuffer buf(istr); + /// Check if we have enough data in buffer to check if it's a null. + if (istr.available() > null_representation.size()) + { + auto check_for_null = [&istr, &null_representation]() + { + auto * pos = istr.position(); + if (checkString(null_representation, istr) && (*istr.position() == '\t' || *istr.position() == '\n')) + return true; + istr.position() = pos; + return false; + }; + auto deserialize_nested = [&nested, &settings, &istr] (IColumn & nested_column) + { + nested->deserializeTextEscaped(nested_column, istr, settings); + }; + return safeDeserialize(column, *nested, check_for_null, deserialize_nested); + } + + /// We don't have enough data in buffer to check if it's a null. + /// Use PeekableReadBuffer to make a checkpoint before checking null + /// representation and rollback if check was failed. + PeekableReadBuffer buf(istr, true); auto check_for_null = [&buf, &null_representation]() { buf.setCheckpoint(); @@ -281,7 +303,7 @@ ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, R return false; }; - auto deserialize_nested = [&nested, &settings, &buf, &null_representation] (IColumn & nested_column) + auto deserialize_nested = [&nested, &settings, &buf, &null_representation, &istr] (IColumn & nested_column) { auto * pos = buf.position(); nested->deserializeTextEscaped(nested_column, buf, settings); @@ -299,8 +321,8 @@ ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, R WriteBufferFromOwnString parsed_value; nested->serializeTextEscaped(nested_column, nested_column.size() - 1, parsed_value, settings); - throw DB::ParsingException("Error while parsing \"" + std::string(pos, std::min(size_t{10}, buf.available())) + "\" as Nullable" - + " at position " + std::to_string(buf.count()) + ": got \"" + std::string(pos, buf.position() - pos) + throw DB::ParsingException("Error while parsing \"" + std::string(pos, buf.buffer().end()) + std::string(istr.position(), std::min(size_t(10), istr.available())) + "\" as Nullable" + + " at position " + std::to_string(istr.count()) + ": got \"" + std::string(pos, buf.position() - pos) + "\", which was deserialized as \"" + parsed_value.str() + "\". It seems that input data is ill-formatted.", ErrorCodes::CANNOT_READ_ALL_DATA); @@ -347,7 +369,7 @@ template ReturnType SerializationNullable::deserializeWholeTextImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested) { - PeekableReadBuffer buf(istr); + PeekableReadBuffer buf(istr, true); auto check_for_null = [&buf]() { buf.setCheckpoint(); @@ -402,7 +424,28 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB [&nested, &istr, &settings] (IColumn & nested_column) { nested->deserializeTextCSV(nested_column, istr, settings); }); } - PeekableReadBuffer buf(istr); + /// Check if we have enough data in buffer to check if it's a null. + if (istr.available() > null_representation.size()) + { + auto check_for_null = [&istr, &null_representation, &settings]() + { + auto * pos = istr.position(); + if (checkString(null_representation, istr) && (*istr.position() == settings.csv.delimiter || *istr.position() == '\r' || *istr.position() == '\n')) + return true; + istr.position() = pos; + return false; + }; + auto deserialize_nested = [&nested, &settings, &istr] (IColumn & nested_column) + { + nested->deserializeTextCSV(nested_column, istr, settings); + }; + return safeDeserialize(column, *nested, check_for_null, deserialize_nested); + } + + /// We don't have enough data in buffer to check if it's a null. + /// Use PeekableReadBuffer to make a checkpoint before checking null + /// representation and rollback if the check was failed. + PeekableReadBuffer buf(istr, true); auto check_for_null = [&buf, &null_representation, &settings]() { buf.setCheckpoint(); @@ -414,7 +457,7 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB return false; }; - auto deserialize_nested = [&nested, &settings, &buf, &null_representation] (IColumn & nested_column) + auto deserialize_nested = [&nested, &settings, &buf, &null_representation, &istr] (IColumn & nested_column) { auto * pos = buf.position(); nested->deserializeTextCSV(nested_column, buf, settings); @@ -433,8 +476,8 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB WriteBufferFromOwnString parsed_value; nested->serializeTextCSV(nested_column, nested_column.size() - 1, parsed_value, settings); - throw DB::ParsingException("Error while parsing \"" + std::string(pos, std::min(size_t{10}, buf.available())) + "\" as Nullable" - + " at position " + std::to_string(buf.count()) + ": got \"" + std::string(pos, buf.position() - pos) + throw DB::ParsingException("Error while parsing \"" + std::string(pos, buf.buffer().end()) + std::string(istr.position(), std::min(size_t(10), istr.available())) + "\" as Nullable" + + " at position " + std::to_string(istr.count()) + ": got \"" + std::string(pos, buf.position() - pos) + "\", which was deserialized as \"" + parsed_value.str() + "\". It seems that input data is ill-formatted.", ErrorCodes::CANNOT_READ_ALL_DATA); diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 152b58f9fa7..a34660a0197 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -59,7 +59,6 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.csv.empty_as_default = settings.input_format_defaults_for_omitted_fields; format_settings.csv.input_format_enum_as_number = settings.input_format_csv_enum_as_number; format_settings.csv.null_representation = settings.format_csv_null_representation; - format_settings.csv.unquoted_null_literal_as_null = settings.input_format_csv_unquoted_null_literal_as_null; format_settings.csv.input_format_arrays_as_nested_csv = settings.input_format_csv_arrays_as_nested_csv; format_settings.custom.escaping_rule = settings.format_custom_escaping_rule; format_settings.custom.field_delimiter = settings.format_custom_field_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 403ccbc6763..269ce9a8a53 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -71,7 +71,6 @@ struct FormatSettings char delimiter = ','; bool allow_single_quotes = true; bool allow_double_quotes = true; - bool unquoted_null_literal_as_null = false; bool empty_as_default = false; bool crlf_end_of_line = false; bool input_format_enum_as_number = false; diff --git a/src/IO/PeekableReadBuffer.cpp b/src/IO/PeekableReadBuffer.cpp index c7cef777afc..e2b1873283f 100644 --- a/src/IO/PeekableReadBuffer.cpp +++ b/src/IO/PeekableReadBuffer.cpp @@ -9,8 +9,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -PeekableReadBuffer::PeekableReadBuffer(ReadBuffer & sub_buf_, size_t start_size_ /*= DBMS_DEFAULT_BUFFER_SIZE*/) - : BufferWithOwnMemory(start_size_), sub_buf(sub_buf_) +PeekableReadBuffer::PeekableReadBuffer(ReadBuffer & sub_buf_, bool use_existing_memory /*= false*/, size_t start_size_ /*= DBMS_DEFAULT_BUFFER_SIZE*/) + : BufferWithOwnMemory(use_existing_memory ? sizeof(existing_memory) : start_size_, use_existing_memory ? existing_memory : nullptr), sub_buf(sub_buf_) { padded &= sub_buf.isPadded(); /// Read from sub-buffer diff --git a/src/IO/PeekableReadBuffer.h b/src/IO/PeekableReadBuffer.h index 4515c6f8ce5..a8eff09c4f2 100644 --- a/src/IO/PeekableReadBuffer.h +++ b/src/IO/PeekableReadBuffer.h @@ -20,7 +20,7 @@ class PeekableReadBuffer : public BufferWithOwnMemory { friend class PeekableReadBufferCheckpoint; public: - explicit PeekableReadBuffer(ReadBuffer & sub_buf_, size_t start_size_ = DBMS_DEFAULT_BUFFER_SIZE); + explicit PeekableReadBuffer(ReadBuffer & sub_buf_, bool use_existing_memory = false, size_t start_size_ = DBMS_DEFAULT_BUFFER_SIZE); ~PeekableReadBuffer() override; @@ -89,6 +89,11 @@ private: size_t peeked_size = 0; std::optional checkpoint = std::nullopt; bool checkpoint_in_own_memory = false; + + /// Small amount of memory on stack to use in BufferWithOwnMemory on + /// it's creation to prevent unnecessary allocation if PeekableReadBuffer + /// is often created. + char existing_memory[16]; }; diff --git a/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.reference b/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.reference new file mode 100644 index 00000000000..06618cc63b1 --- /dev/null +++ b/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.reference @@ -0,0 +1,55 @@ +TSV +\N +\N +Some text +\N +Some text +\N +Some more text +\N +\N +Some more text +1 Some text 1 +1 \N 1 +CustomNullSome text +CustomNullSome text +\N +Some more text +\N +\N +Some more text +1 \N 1 +1 \N 1 +CSV +\N +\N +\\NSome text +\N +\\NSome text +\N +Some more text +\N +\N +Some more text +1 \\NSome text 1 +1 \N 1 +CustomNullSome text +CustomNullSome text +\N +Some more text +\N +\N +Some more text +1 \N 1 +1 \N 1 +Corner cases +TSV +Some text \N +Some text CustomNull Some text +OK +OK +CSV +Some text \N +Some text CustomNull Some text +OK +OK diff --git a/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.sh b/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.sh new file mode 100755 index 00000000000..1d8e080c7b6 --- /dev/null +++ b/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.sh @@ -0,0 +1,125 @@ +#!/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_null.data + +echo "TSV" + +echo 'Custom NULL representation' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'TSV', 's Nullable(String)') SETTINGS format_tsv_null_representation='Custom NULL representation'" + +echo -e 'N\tU\tL\tL' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'TSV', 's Nullable(String)') SETTINGS format_tsv_null_representation='N\tU\tL\tL'" + +echo -e "\\NSome text" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'TSV', 's Nullable(String)')" + +echo -e "\\N" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'TSV', 's Nullable(String)')" + +echo -e "\\NSome text\n\\N\nSome more text" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'TSV', 's Nullable(String)')" + +echo -e "\\N\n\\N\nSome more text" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'TSV', 's Nullable(String)')" + +echo -e "1\t\\NSome text\t1" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'TSV', 'x Int32, s Nullable(String), y Int32')" + +echo -e "1\t\\N\t1" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'TSV', 'x Int32, s Nullable(String), y Int32')" + +echo -e "CustomNullSome text" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'TSV', 's Nullable(String)') SETTINGS format_tsv_null_representation='CustomNull'" + +echo -e "CustomNullSome text\nCustomNull\nSome more text" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'TSV', 's Nullable(String)') SETTINGS format_tsv_null_representation='CustomNull'" + +echo -e "CustomNull\nCustomNull\nSome more text" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'TSV', 's Nullable(String)') SETTINGS format_tsv_null_representation='CustomNull'" + +echo -e "1\tCustomNull\t1" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'TSV', 'x Int32, s Nullable(String), y Int32') SETTINGS format_tsv_null_representation='CustomNull'" + +echo -e "1\tCustomNull\t1" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'TSV', 'x Int32, s Nullable(String), y Int32') SETTINGS format_tsv_null_representation='CustomNull'" + + +echo "CSV" + +echo 'Custom NULL representation' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'CSV', 's Nullable(String)') SETTINGS format_csv_null_representation='Custom NULL representation'" + +echo -e 'N,U,L,L' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'CSV', 's Nullable(String)') SETTINGS format_csv_null_representation='N,U,L,L'" + +echo -e "\\NSome text" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'CSV', 's Nullable(String)')" + +echo -e "\\N" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'CSV', 's Nullable(String)')" + +echo -e "\\NSome text\n\\N\nSome more text" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'CSV', 's Nullable(String)')" + +echo -e "\\N\n\\N\nSome more text" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'CSV', 's Nullable(String)')" + +echo -e "1,\\NSome text,1" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'CSV', 'x Int32, s Nullable(String), y Int32')" + +echo -e "1,\\N,1" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'CSV', 'x Int32, s Nullable(String), y Int32')" + +echo -e "CustomNullSome text" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'CSV', 's Nullable(String)') SETTINGS format_csv_null_representation='CustomNull'" + +echo -e "CustomNullSome text\nCustomNull\nSome more text" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'CSV', 's Nullable(String)') SETTINGS format_csv_null_representation='CustomNull'" + +echo -e "CustomNull\nCustomNull\nSome more text" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'CSV', 's Nullable(String)') SETTINGS format_csv_null_representation='CustomNull'" + +echo -e "1,CustomNull,1" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'CSV', 'x Int32, s Nullable(String), y Int32') SETTINGS format_csv_null_representation='CustomNull'" + +echo -e "1,CustomNull,1" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'CSV', 'x Int32, s Nullable(String), y Int32') SETTINGS format_csv_null_representation='CustomNull'" + + +echo 'Corner cases' +echo 'TSV' + +echo -e "Some text\tCustomNull" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'TSV', 's String, n Nullable(String)') settings max_read_buffer_size=15, format_tsv_null_representation='CustomNull', input_format_parallel_parsing=0" + +echo -e "Some text\tCustomNull Some text" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'TSV', 's String, n Nullable(String)') settings max_read_buffer_size=15, format_tsv_null_representation='CustomNull', input_format_parallel_parsing=0" + +echo -e "Some text\t123NNN" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'TSV', 's String, n Nullable(Int32)') settings max_read_buffer_size=14, format_tsv_null_representation='123NN', input_format_parallel_parsing=0" 2>&1 | grep -F -q "CANNOT_READ_ALL_DATA" && echo 'OK' || echo 'FAIL' + +echo -e "Some text\tNU\tLL" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'TSV', 's String, n Nullable(String)') settings max_read_buffer_size=13, format_tsv_null_representation='NU\tL', input_format_parallel_parsing=0" 2>&1 | grep -F -q "CANNOT_READ_ALL_DATA" && echo 'OK' || echo 'FAIL' + +echo 'CSV' + +echo -e "Some text,CustomNull" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'CSV', 's String, n Nullable(String)') settings max_read_buffer_size=15, format_csv_null_representation='CustomNull', input_format_parallel_parsing=0" + +echo -e "Some text,CustomNull Some text" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'CSV', 's String, n Nullable(String)') settings max_read_buffer_size=15, format_csv_null_representation='CustomNull', input_format_parallel_parsing=0" + +echo -e "Some text,123NNN" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'CSV', 's String, n Nullable(Int32)') settings max_read_buffer_size=14, format_csv_null_representation='123NN', input_format_parallel_parsing=0" 2>&1 | grep -F -q "CANNOT_READ_ALL_DATA" && echo 'OK' || echo 'FAIL' + +echo -e "Some text,NU,LL" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'CSV', 's String, n Nullable(String)') settings max_read_buffer_size=13, format_csv_null_representation='NU,L', input_format_parallel_parsing=0" 2>&1 | grep -F -q "CANNOT_READ_ALL_DATA" && echo 'OK' || echo 'FAIL' + +rm $DATA_FILE + diff --git a/tests/queries/0_stateless/02104_json_strings_nullable_string.reference b/tests/queries/0_stateless/02104_json_strings_nullable_string.reference new file mode 100644 index 00000000000..a2b5b4ad2ec --- /dev/null +++ b/tests/queries/0_stateless/02104_json_strings_nullable_string.reference @@ -0,0 +1,2 @@ +NULLSome string +NULLSome string diff --git a/tests/queries/0_stateless/02104_json_strings_nullable_string.sh b/tests/queries/0_stateless/02104_json_strings_nullable_string.sh new file mode 100755 index 00000000000..5385c1282b2 --- /dev/null +++ b/tests/queries/0_stateless/02104_json_strings_nullable_string.sh @@ -0,0 +1,17 @@ +#!/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_02104_null.data + +echo -e '{"s" : "NULLSome string"}' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02104_null.data', 'JSONStringsEachRow', 's Nullable(String)')" + +echo -e '["NULLSome string"]' > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02104_null.data', 'JSONCompactStringsEachRow', 's Nullable(String)')" + +rm $DATA_FILE + From d30aecbda8d55f26e540d289fe4ae75f58e462e6 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 27 Oct 2021 17:29:36 +0300 Subject: [PATCH 180/396] Add performance test --- tests/performance/tsv_csv_nullable_parsing.xml | 15 +++++++++++++++ 1 file changed, 15 insertions(+) create mode 100644 tests/performance/tsv_csv_nullable_parsing.xml diff --git a/tests/performance/tsv_csv_nullable_parsing.xml b/tests/performance/tsv_csv_nullable_parsing.xml new file mode 100644 index 00000000000..2d5c5cec275 --- /dev/null +++ b/tests/performance/tsv_csv_nullable_parsing.xml @@ -0,0 +1,15 @@ + + +CREATE TABLE IF NOT EXISTS table_tsv (s Nullable(String)) ENGINE = File('TSV') +CREATE TABLE IF NOT EXISTS table_csv (s Nullable(String)) ENGINE = File('CSV') + +INSERT INTO table_tsv SELECT number % 2 ? 'Some text' : NULL FROM numbers(1000000) FORMAT TSV +INSERT INTO table_csv SELECT number % 2 ? 'Some text' : NULL FROM numbers(1000000) FORMAT CSV + +SELECT * FROM table_tsv FORMAT Null +SELECT * FROM table_csv FORMAT Null + +DROP TABLE IF EXISTS table_tsv +DROP TABLE IF EXISTS table_csv + + From b4aae86fff64feb341edaa4e4a39ed9c91fb3358 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 27 Oct 2021 22:16:34 +0300 Subject: [PATCH 181/396] Add comments, update docs --- docs/en/interfaces/formats.md | 59 ++++++++++++++++--- docs/en/operations/settings/settings.md | 44 +++++++++++++- .../Formats/Impl/BinaryRowInputFormat.h | 1 + .../Impl/JSONCompactEachRowRowInputFormat.cpp | 4 ++ .../Impl/TabSeparatedRowInputFormat.cpp | 2 +- .../Formats/Impl/ValuesBlockInputFormat.cpp | 1 - .../Formats/RowInputFormatWithNamesAndTypes.h | 20 ++++++- 7 files changed, 116 insertions(+), 15 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index d0e5c44b4f7..fb023ff1e36 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -16,10 +16,13 @@ The supported formats are: | [TabSeparatedRaw](#tabseparatedraw) | ✔ | ✔ | | [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ | | [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ | +| [TabSeparatedRawWithNames](#tabseparatedrawwithnames) | ✔ | ✔ | +| [TabSeparatedRawWithNamesAndTypes](#tabseparatedrawwithnamesandtypes) | ✔ | ✔ | | [Template](#format-template) | ✔ | ✔ | | [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ | | [CSV](#csv) | ✔ | ✔ | | [CSVWithNames](#csvwithnames) | ✔ | ✔ | +| [CSVWithNamesAndTypes](#csvwithnamesandtypes) | ✔ | ✔ | | [CustomSeparated](#format-customseparated) | ✔ | ✔ | | [Values](#data-format-values) | ✔ | ✔ | | [Vertical](#vertical) | ✗ | ✔ | @@ -33,8 +36,10 @@ The supported formats are: | [JSONStringsEachRow](#jsonstringseachrow) | ✔ | ✔ | | [JSONStringsEachRowWithProgress](#jsonstringseachrowwithprogress) | ✗ | ✔ | | [JSONCompactEachRow](#jsoncompacteachrow) | ✔ | ✔ | +| [JSONCompactEachRowWithNames](#jsoncompacteachrowwithnames) | ✔ | ✔ | | [JSONCompactEachRowWithNamesAndTypes](#jsoncompacteachrowwithnamesandtypes) | ✔ | ✔ | | [JSONCompactStringsEachRow](#jsoncompactstringseachrow) | ✔ | ✔ | +| [JSONCompactStringsEachRowWithNames](#jsoncompactstringseachrowwithnames) | ✔ | ✔ | | [JSONCompactStringsEachRowWithNamesAndTypes](#jsoncompactstringseachrowwithnamesandtypes) | ✔ | ✔ | | [TSKV](#tskv) | ✔ | ✔ | | [Pretty](#pretty) | ✗ | ✔ | @@ -51,6 +56,7 @@ The supported formats are: | [ArrowStream](#data-format-arrow-stream) | ✔ | ✔ | | [ORC](#data-format-orc) | ✔ | ✔ | | [RowBinary](#rowbinary) | ✔ | ✔ | +| [RowBinaryWithNames](#rowbinarywithnamesandtypes) | ✔ | ✔ | | [RowBinaryWithNamesAndTypes](#rowbinarywithnamesandtypes) | ✔ | ✔ | | [Native](#native) | ✔ | ✔ | | [Null](#null) | ✗ | ✔ | @@ -126,6 +132,9 @@ Arrays are written as a list of comma-separated values in square brackets. Numbe [NULL](../sql-reference/syntax.md) is formatted as `\N`. +If setting [input_format_tsv_empty_as_default](../operations/settings/settings.md#settings-input_format_tsv_empty_as_default) is enabled, +empty input fields are replaced with default values. For complex default expressions [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#settings-input_format_defaults_for_omitted_fields) must be enabled too. + Each element of [Nested](../sql-reference/data-types/nested-data-structures/nested.md) structures is represented as array. For example: @@ -164,18 +173,35 @@ This format is also available under the name `TSVRaw`. ## TabSeparatedWithNames {#tabseparatedwithnames} Differs from the `TabSeparated` format in that the column names are written in the first row. -During parsing, the first row is completely ignored. You can’t use column names to determine their position or to check their correctness. -(Support for parsing the header row may be added in the future.) +If setting [input_format_with_names_use_header](../operations/settings/settings.md#settings-input_format_with_names_use_header) is set to 1, +the columns from input data will be mapped to the columns from the table by their names, columns with unknown names will be skipped if setting [input_format_skip_unknown_fields](../operations/settings/settings.md#settings-input_format_skip_unknown_fields) is set to 1. +Otherwise, the first row will be skipped. This format is also available under the name `TSVWithNames`. ## TabSeparatedWithNamesAndTypes {#tabseparatedwithnamesandtypes} Differs from the `TabSeparated` format in that the column names are written to the first row, while the column types are in the second row. -During parsing, the first and second rows are completely ignored. +The first row with names is processed the same way as in `TabSeparatedWithNames` format. +If setting [input_format_with_types_use_header](../operations/settings/settings.md#settings-input_format_with_types_use_header) is set to 1, +the types from input data will be compared with the types of the corresponding columns from the table. Otherwise, the second row will be skipped. This format is also available under the name `TSVWithNamesAndTypes`. +## TabSeparatedRawWithNames {#tabseparatedrawwithnames} + +Differs from `TabSeparatedWithNames` format in that the rows are written without escaping. +When parsing with this format, tabs or linefeeds are not allowed in each field. + +This format is also available under the name `TSVRawWithNames`. + +## TabSeparatedWithNamesAndTypes {#tabseparatedwithnamesandtypes} + +Differs from `TabSeparatedWithNamesAndTypes` format in that the rows are written without escaping. +When parsing with this format, tabs or linefeeds are not allowed in each field. + +This format is also available under the name `TSVRawWithNamesAndNames`. + ## Template {#format-template} This format allows specifying a custom format string with placeholders for values with a specified escaping rule. @@ -196,7 +222,7 @@ where `delimiter_i` is a delimiter between values (`$` symbol can be escaped as - `Raw` (without escaping, similarly to `TSVRaw`) - `None` (no escaping rule, see further) -If an escaping rule is omitted, then `None` will be used. `XML` and `Raw` are suitable only for output. +If an escaping rule is omitted, then `None` will be used. `XML` is suitable only for output. So, for the following format string: @@ -376,9 +402,8 @@ $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FOR When parsing, all values can be parsed either with or without quotes. Both double and single quotes are supported. Rows can also be arranged without quotes. In this case, they are parsed up to the delimiter character or line feed (CR or LF). In violation of the RFC, when parsing rows without quotes, the leading and trailing spaces and tabs are ignored. For the line feed, Unix (LF), Windows (CR LF) and Mac OS Classic (CR LF) types are all supported. -Empty unquoted input values are replaced with default values for the respective columns, if -[input_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) -is enabled. +If setting [input_format_csv_empty_as_default](../operations/settings/settings.md#settings-input_format_csv_empty_as_default) is enabled, +empty unquoted input values are replaced with default values. For complex default expressions [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#settings-input_format_defaults_for_omitted_fields) must be enabled too. `NULL` is formatted as `\N` or `NULL` or an empty unquoted string (see settings [input_format_csv_unquoted_null_literal_as_null](../operations/settings/settings.md#settings-input_format_csv_unquoted_null_literal_as_null) and [input_format_defaults_for_omitted_fields](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields)). @@ -386,7 +411,11 @@ The CSV format supports the output of totals and extremes the same way as `TabSe ## CSVWithNames {#csvwithnames} -Also prints the header row, similar to [TabSeparatedWithNames](#tabseparatedwithnames). +Also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). + +## CSVWithNamesAndTypes {#csvwithnamesandtypes} + +Also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). ## CustomSeparated {#format-customseparated} @@ -658,7 +687,14 @@ Differs from `JSONEachRow`/`JSONStringsEachRow` in that ClickHouse will also yie {"progress":{"read_rows":"3","read_bytes":"24","written_rows":"0","written_bytes":"0","total_rows_to_read":"3"}} ``` +## JSONCompactEachRowWithNames {#jsoncompacteachrowwithnames} + +Differs from `JSONCompactEachRow` format in that the also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). + ## JSONCompactEachRowWithNamesAndTypes {#jsoncompacteachrowwithnamesandtypes} + +Differs from `JSONCompactEachRow` format in that the also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). + ## JSONCompactStringsEachRowWithNamesAndTypes {#jsoncompactstringseachrowwithnamesandtypes} Differs from `JSONCompactEachRow`/`JSONCompactStringsEachRow` in that the column names and types are written as the first two rows. @@ -911,6 +947,13 @@ Array is represented as a varint length (unsigned [LEB128](https://en.wikipedia. For [NULL](../sql-reference/syntax.md#null-literal) support, an additional byte containing 1 or 0 is added before each [Nullable](../sql-reference/data-types/nullable.md) value. If 1, then the value is `NULL` and this byte is interpreted as a separate value. If 0, the value after the byte is not `NULL`. +## RowBinaryWithNames {#rowbinarywithnames} + +Similar to [RowBinary](#rowbinary), but with added header: + +- [LEB128](https://en.wikipedia.org/wiki/LEB128)-encoded number of columns (N) +- N `String`s specifying column names + ## RowBinaryWithNamesAndTypes {#rowbinarywithnamesandtypes} Similar to [RowBinary](#rowbinary), but with added header: diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 9147d3e0553..c7c3ffddb31 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -399,7 +399,7 @@ Default value: 1. ## input_format_defaults_for_omitted_fields {#session_settings-input_format_defaults_for_omitted_fields} -When performing `INSERT` queries, replace omitted input column values with default values of the respective columns. This option only applies to [JSONEachRow](../../interfaces/formats.md#jsoneachrow), [CSV](../../interfaces/formats.md#csv) and [TabSeparated](../../interfaces/formats.md#tabseparated) formats. +When performing `INSERT` queries, replace omitted input column values with default values of the respective columns. This option only applies to [JSONEachRow](../../interfaces/formats.md#jsoneachrow), [CSV](../../interfaces/formats.md#csv), [TabSeparated](../../interfaces/formats.md#tabseparated) formats and formats with `WithNames`/`WithNamesAndTypes` suffixes. !!! note "Note" When this option is enabled, extended table metadata are sent from server to client. It consumes additional computing resources on the server and can reduce performance. @@ -417,6 +417,12 @@ When enabled, replace empty input fields in TSV with default values. For complex Disabled by default. +## input_format_csv_empty_as_default {#settings-input-format-csv-empty-as-default} + +When enabled, replace empty input fields in CSV with default values. For complex default expressions `input_format_defaults_for_omitted_fields` must be enabled too. + +Enabled by default. + ## input_format_tsv_enum_as_number {#settings-input_format_tsv_enum_as_number} Enables or disables parsing enum values as enum ids for TSV input format. @@ -540,8 +546,40 @@ To improve insert performance, we recommend disabling this check if you are sure Supported formats: -- [CSVWithNames](../../interfaces/formats.md#csvwithnames) -- [TabSeparatedWithNames](../../interfaces/formats.md#tabseparatedwithnames) +- [CSVWithNames](../../interfaces/formats.md#csvwithnames) +- [CSVWithNames](../../interfaces/formats.md#csvwithnamesandtypes) +- [TabSeparatedWithNames](../../interfaces/formats.md#tabseparatedwithnames) +- [TabSeparatedWithNamesAndTypes](../../interfaces/formats.md#tabseparatedwithnamesandtypes) +- [JSONCompactEachRowWithNames](../../interfaces/formats.md#jsoncompacteachrowwithnames) +- [JSONCompactEachRowWithNamesAndTypes](../../interfaces/formats.md#jsoncompacteachrowwithnamesandtypes) +- [JSONCompactStringsEachRowWithNames](../../interfaces/formats.md#jsoncompactstringseachrowwithnames) +- [JSONCompactStringsEachRowWithNamesAndTypes](../../interfaces/formats.md#jsoncompactstringseachrowwithnamesandtypes) +- [RowBinaryWithNames](../../interfaces/formats.md#rowbinarywithnames-rowbinarywithnames) +- [RowBinaryWithNamesAndTypes](../../interfaces/formats.md#rowbinarywithnamesandtypes-rowbinarywithnamesandtypes) + +Possible values: + +- 0 — Disabled. +- 1 — Enabled. + +Default value: 1. + +## input_format_with_types_use_header {#settings-input-format-with-types-use-header} + +Controls whether format parser should check if data types from the input data match data types from the target table. + +Supported formats: + +- [CSVWithNames](../../interfaces/formats.md#csvwithnames) +- [CSVWithNames](../../interfaces/formats.md#csvwithnamesandtypes) +- [TabSeparatedWithNames](../../interfaces/formats.md#tabseparatedwithnames) +- [TabSeparatedWithNamesAndTypes](../../interfaces/formats.md#tabseparatedwithnamesandtypes) +- [JSONCompactEachRowWithNames](../../interfaces/formats.md#jsoncompacteachrowwithnames) +- [JSONCompactEachRowWithNamesAndTypes](../../interfaces/formats.md#jsoncompacteachrowwithnamesandtypes) +- [JSONCompactStringsEachRowWithNames](../../interfaces/formats.md#jsoncompactstringseachrowwithnames) +- [JSONCompactStringsEachRowWithNamesAndTypes](../../interfaces/formats.md#jsoncompactstringseachrowwithnamesandtypes) +- [RowBinaryWithNames](../../interfaces/formats.md#rowbinarywithnames-rowbinarywithnames) +- [RowBinaryWithNamesAndTypes](../../interfaces/formats.md#rowbinarywithnamesandtypes-rowbinarywithnamesandtypes) Possible values: diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.h b/src/Processors/Formats/Impl/BinaryRowInputFormat.h index 1a9a873097b..61d6df77522 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.h +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.h @@ -37,6 +37,7 @@ private: std::vector readTypes() override; std::vector readHeaderRow(); + /// Data types read from input data. DataTypes read_data_types; UInt64 read_columns = 0; }; diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index ec0a0fcf9e6..88fb411ffbd 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -204,6 +204,10 @@ void registerFileSegmentationEngineJSONCompactEachRow(FormatFactory & factory) { auto register_func = [&](const String & format_name, bool with_names, bool with_types) { + /// In case when we have names and/or types in the first two/one rows, + /// we need to read at least one more row of actual data. So, set + /// the minimum of rows for segmentation engine according to + /// parameters with_names and with_types. 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) { diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 117875c5cf0..1e6d238b202 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -192,7 +192,7 @@ bool TabSeparatedRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out void TabSeparatedRowInputFormat::checkNullValueForNonNullable(DataTypePtr type) { - bool can_be_parsed_as_null = removeLowCardinality(type)->isNullable() || format_settings.null_as_default; + bool can_be_parsed_as_null = type->isNullable() || type->isLowCardinalityNullable() || 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()) diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index ab584a0b2de..5f471dc0151 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -182,7 +182,6 @@ bool ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx) const auto & serialization = serializations[column_idx]; if (format_settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable()) read = SerializationNullable::deserializeTextQuotedImpl(column, *buf, format_settings, serialization); - read = SerializationNullable::deserializeTextQuotedImpl(column, *buf, format_settings, serialization); else serialization->deserializeTextQuoted(column, *buf, format_settings); diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index f94a5d6b9c8..0fd83238f5f 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -8,6 +8,16 @@ namespace DB { /// Base class for input formats with -WithNames and -WithNamesAndTypes suffixes. +/// It accepts 2 parameters in constructor - with_names and with_types and implements +/// input format depending on them: +/// - if with_names is true, it will expect that the first row of data contains column +/// names. If setting input_format_with_names_use_header is set to 1, columns mapping +/// will be performed. +/// - if with_types is true, it will expect that the second row of data contains column +/// types. If setting input_format_with_types_use_header is set to 1, types from input +/// will be compared types from header. +/// It's important that firstly this class reads/skips names and only +/// then reads/skips types. So you can this invariant. class RowInputFormatWithNamesAndTypes : public RowInputFormatWithDiagnosticInfo { public: @@ -25,12 +35,17 @@ public: void resetParser() override; protected: - /// Return false if there was no real value and we inserted default value. + /// Read single field from input. 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; + /// Skip single field, it's used to skip unknown columns. virtual void skipField(size_t file_column) = 0; + /// Skip the whole row with names. virtual void skipNames() = 0; + /// Skip the whole row with types. virtual void skipTypes() = 0; + + /// Skip delimiters, if any. virtual void skipRowStartDelimiter() {} virtual void skipFieldDelimiter() {} virtual void skipRowEndDelimiter() {} @@ -43,10 +58,11 @@ protected: virtual bool parseRowEndWithDiagnosticInfo(WriteBuffer &) { return true;} bool isGarbageAfterField(size_t, ReadBuffer::Position) override {return false; } + /// Read row with names and return the list of them. virtual std::vector readNames() = 0; + /// Read row with types and return the list of them. virtual std::vector readTypes() = 0; - const FormatSettings format_settings; DataTypes data_types; From b4b48e09a57402ff9117ecbd21c47ec4b1376a28 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 27 Oct 2021 22:29:29 +0300 Subject: [PATCH 182/396] Document the database comment MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Задокументировал секцию comment. --- .../statements/create/database.md | 29 +++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/docs/en/sql-reference/statements/create/database.md b/docs/en/sql-reference/statements/create/database.md index 3c6f73d54db..9d1a4c614e7 100644 --- a/docs/en/sql-reference/statements/create/database.md +++ b/docs/en/sql-reference/statements/create/database.md @@ -27,3 +27,32 @@ 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’s also a [lazy](../../../engines/database-engines/lazy.md) engine. + +### COMMENT {#comment} + +You can add a comment to the database when you creating it. + +The comment is supported for all database engines. + +**Syntax** + +``` sql +CREATE DATABASE db_name ENGINE = engine(...) COMMENT 'Comment' +``` + +**Example** + +Query: + +``` sql +CREATE DATABASE db_comment ENGINE = Memory COMMENT 'The temporary database'; +SELECT name, comment FROM system.databases WHERE name = 'db_comment'; +``` + +Result: + +```text +┌─name───────┬─comment────────────────┐ +│ db_comment │ The temporary database │ +└────────────┴────────────────────────┘ +``` From a09746b4e2393934523c6d5ba4f510f9c5b0c8e3 Mon Sep 17 00:00:00 2001 From: Alexey Date: Thu, 28 Oct 2021 04:47:16 +0000 Subject: [PATCH 183/396] updated related data types --- docs/en/operations/settings/settings.md | 2 +- docs/en/sql-reference/statements/describe-table.md | 2 +- docs/ru/operations/settings/settings.md | 2 +- docs/ru/sql-reference/statements/describe-table.md | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ee16e820913..148eb4358c4 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 in columns of [Tuple](../../sql-reference/data-types/tuple.md) or [Map](../../sql-reference/data-types/map.md#map-subcolumns) data types. +Enables describing subcolumns for a [DESCRIBE](../../sql-reference/statements/describe-table.md) query. For example, members of a [Tuple](../../sql-reference/data-types/tuple.md) or subcolumns of a [Map](../../sql-reference/data-types/map.md#map-subcolumns), [Nullable](../../sql-reference/data-types/nullable.md#finding-null) or [Array](../../sql-reference/data-types/array.md/#array-size) data types. Possible values: diff --git a/docs/en/sql-reference/statements/describe-table.md b/docs/en/sql-reference/statements/describe-table.md index 97247912233..823a31ed313 100644 --- a/docs/en/sql-reference/statements/describe-table.md +++ b/docs/en/sql-reference/statements/describe-table.md @@ -26,7 +26,7 @@ The `DESCRIBE` statement returns a row for each table column with the following All columns in [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) data structures are described separately. The name of each column is prefixed with a parent column name and a dot. -To show internal subcolumns of a [Tuple](../../sql-reference/data-types/tuple.md) or a [Map](../../sql-reference/data-types/map.md#map-subcolumns), use the [describe_include_subcolumns](../../operations/settings/settings.md#describe_include_subcolumns) setting. +To show internal subcolumns of other data types, use the [describe_include_subcolumns](../../operations/settings/settings.md#describe_include_subcolumns) setting. **Example** diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 2ec9edb460b..ea42ae12fa1 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3606,7 +3606,7 @@ SELECT * FROM positional_arguments ORDER BY 2,3; ## describe_include_subcolumns {#describe_include_subcolumns} -Разрешает описание подстолбцов при выполнении запроса [DESCRIBE](../../sql-reference/statements/describe-table.md). Настройка действует на подстолбцы внутри столбцов типов [Tuple](../../sql-reference/data-types/tuple.md) или [Map](../../sql-reference/data-types/map.md#map-subcolumns). +Разрешает описание подстолбцов при выполнении запроса [DESCRIBE](../../sql-reference/statements/describe-table.md). Настройка действует, например, на элементы [Tuple](../../sql-reference/data-types/tuple.md) или подстолбцы типов [Map](../../sql-reference/data-types/map.md#map-subcolumns), [Nullable](../../sql-reference/data-types/nullable.md#finding-null) или [Array](../../sql-reference/data-types/array.md/#array-size). Возможные значения: diff --git a/docs/ru/sql-reference/statements/describe-table.md b/docs/ru/sql-reference/statements/describe-table.md index 3ebb28909ba..56c778f7c76 100644 --- a/docs/ru/sql-reference/statements/describe-table.md +++ b/docs/ru/sql-reference/statements/describe-table.md @@ -25,7 +25,7 @@ DESC|DESCRIBE TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] - `is_subcolumn` — флаг, который равен `1` для внутренних подстолбцов. Он появляется в результате, только если описание подстолбцов разрешено настройкой [describe_include_subcolumns](../../operations/settings/settings.md#describe_include_subcolumns). Каждый столбец [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) структур описывается отдельно. Перед его именем ставится имя родительского столбца с точкой. -Чтобы отобразить внутренние подстолбцы [Tuple](../../sql-reference/data-types/tuple.md) или [Map](../../sql-reference/data-types/map.md#map-subcolumns), нужно включить настройку [describe_include_subcolumns](../../operations/settings/settings.md#describe_include_subcolumns). +Чтобы отобразить внутренние подстолбцы других типов данных, нужно включить настройку [describe_include_subcolumns](../../operations/settings/settings.md#describe_include_subcolumns). **Пример** From 6e0cbe341fefb912ab31bd80903423bc34b0862f Mon Sep 17 00:00:00 2001 From: Alexey Date: Thu, 28 Oct 2021 05:51:08 +0000 Subject: [PATCH 184/396] minor update --- 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 148eb4358c4..6ccb91093e9 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. For example, members of a [Tuple](../../sql-reference/data-types/tuple.md) or subcolumns of a [Map](../../sql-reference/data-types/map.md#map-subcolumns), [Nullable](../../sql-reference/data-types/nullable.md#finding-null) or [Array](../../sql-reference/data-types/array.md/#array-size) data types. +Enables describing subcolumns for a [DESCRIBE](../../sql-reference/statements/describe-table.md) query. For example, members of a [Tuple](../../sql-reference/data-types/tuple.md) or subcolumns of a [Map](../../sql-reference/data-types/map.md#map-subcolumns), [Nullable](../../sql-reference/data-types/nullable.md#finding-null) or an [Array](../../sql-reference/data-types/array.md/#array-size) data type. Possible values: From 37cd91c0bf95c0482427bbb276096d62e90bb296 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Oct 2021 07:59:05 +0000 Subject: [PATCH 185/396] Fix --- src/Storages/FileLog/FileLogSettings.h | 2 +- src/Storages/FileLog/StorageFileLog.cpp | 6 +++++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Storages/FileLog/FileLogSettings.h b/src/Storages/FileLog/FileLogSettings.h index d14120d0ba0..865727bd864 100644 --- a/src/Storages/FileLog/FileLogSettings.h +++ b/src/Storages/FileLog/FileLogSettings.h @@ -14,7 +14,7 @@ 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, 0, "Number of max threads to parse files, default is 0, which means the number will be max(1, physical_cpu_cores / 4)", 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) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index d9b08762d28..1dc25485a47 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -750,7 +750,11 @@ void registerStorageFileLog(StorageFactory & factory) auto physical_cpu_cores = getNumberOfPhysicalCPUCores(); auto num_threads = filelog_settings->max_threads.value; - if (num_threads > physical_cpu_cores) + if (!num_threads) /// Default + { + num_threads = std::max(unsigned(1), physical_cpu_cores / 4); + } + else 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); } From 39322cdbe22c23e62d3456dc5eab678aec5d4700 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Thu, 28 Oct 2021 17:15:32 +0800 Subject: [PATCH 186/396] add a new function mapExtractKeyLike --- .../functions/tuple-map-functions.md | 38 +++++ src/Functions/map.cpp | 136 ++++++++++++++++++ ...02111_function_mapExtractKeyLike.reference | 23 +++ .../02111_function_mapExtractKeyLike.sql | 22 +++ 4 files changed, 219 insertions(+) create mode 100644 tests/queries/0_stateless/02111_function_mapExtractKeyLike.reference create mode 100644 tests/queries/0_stateless/02111_function_mapExtractKeyLike.sql diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index 46ce350377c..45cee5d24bc 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -390,5 +390,43 @@ Result: └─────────────────────────────┘ ``` +## mapExtractKeyLike {#mapExtractKeyLike} + +**Syntax** + +```sql +mapExtractKeyLike(map, pattern) +``` + +**Parameters** + +- `map` — Map. [Map](../../sql-reference/data-types/map.md). +- `pattern` - String pattern to match. + +**Returned value** + +- A map contained elements the key of which matchs the specified pattern. If there are no elements matched the pattern, it will return an empty map. + +**Example** + +Query: + +```sql +CREATE TABLE test (a Map(String,String)) ENGINE = Memory; + +INSERT INTO test VALUES ({'abc':'abc','def':'def'}), ({'hij':'hij','klm':'klm'}); + +SELECT mapExtractKeyLike(a, 'a%') FROM test; +``` + +Result: + +```text +┌─mapExtractKeyLike(a, 'a%')─┐ +│ {'abc':'abc'} │ +│ {} │ +└────────────────────────────┘ +``` + [Original article](https://clickhouse.com/docs/en/sql-reference/functions/tuple-map-functions/) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index edb0c28c980..029c4f6f80e 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -382,6 +382,141 @@ public: bool useDefaultImplementationForConstants() const override { return true; } }; +class FunctionExtractKeyLike : public IFunction +{ +public: + static constexpr auto name = "mapExtractKeyLike"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override + { + return name; + } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*info*/) const override { return true; } + + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() != 2) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 2", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + + const DataTypeMap * map_type = checkAndGetDataType(arguments[0].type.get()); + + if (!map_type) + throw Exception{"First argument for function " + getName() + " must be a map", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + + auto key_type = map_type->getKeyType(); + + WhichDataType which(key_type); + + if (!which.isStringOrFixedString()) + throw Exception{"Function " + getName() + "only support the map with String or FixedString key", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + if (!isStringOrFixedString(arguments[1].type)) + throw Exception{"Second argument passed to function " + getName() + " must be String or FixedString", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return std::make_shared(map_type->getKeyType(), map_type->getValueType()); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + bool is_const = isColumnConst(*arguments[0].column); + const ColumnMap * col_map = typeid_cast(arguments[0].column.get()); + + //It may not be necessary to check this condition, cause it will be checked in getReturnTypeImpl function + if (!col_map) + return nullptr; + + const DataTypeMap * map_type = checkAndGetDataType(arguments[0].type.get()); + auto key_type = map_type->getKeyType(); + auto value_type = map_type->getValueType(); + + const auto & nested_column = col_map->getNestedColumn(); + const auto & values_column = col_map->getNestedData().getColumn(1); + const ColumnString * keys_string_column = checkAndGetColumn(col_map->getNestedData().getColumn(0)); + const ColumnFixedString * keys_fixed_string_column = checkAndGetColumn(col_map->getNestedData().getColumn(0)); + + FunctionLike func_like; + + //create result data + MutableColumnPtr keys_data = key_type->createColumn(); + MutableColumnPtr values_data = value_type->createColumn(); + MutableColumnPtr offsets = DataTypeNumber().createColumn(); + + IColumn::Offset current_offset = 0; + + for (size_t row = 0; row < input_rows_count; row++) + { + size_t element_start_row = row != 0 ? nested_column.getOffsets()[row-1] : 0; + size_t element_size = nested_column.getOffsets()[row]- element_start_row; + + ColumnsWithTypeAndName new_arguments; + ColumnPtr sub_map_column; + DataTypePtr data_type; + + if (keys_string_column) + { + sub_map_column = keys_string_column->cut(element_start_row, element_size); + data_type = std::make_shared(); + } + else + { + sub_map_column = keys_fixed_string_column->cut(element_start_row, element_size); + data_type =std::make_shared(checkAndGetColumn(sub_map_column.get())->getN()); + } + + size_t col_key_size = sub_map_column->size(); + auto column = is_const? ColumnConst::create(std::move(sub_map_column), std::move(col_key_size)) : std::move(sub_map_column); + + new_arguments = { + { + column, + data_type, + "" + }, + arguments[1] + }; + + auto res = func_like.executeImpl(new_arguments, result_type, input_rows_count); + const auto & container = checkAndGetColumn(res.get())->getData(); + + for (size_t row_num = 0; row_num < element_size; row_num++) + { + if (container[row_num] == 1) + { + auto key_ref = keys_string_column ? + keys_string_column->getDataAt(element_start_row + row_num) : + keys_fixed_string_column->getDataAt(element_start_row + row_num); + auto value_ref = values_column.getDataAt(element_start_row + row_num); + + keys_data->insertData(key_ref.data, key_ref.size); + values_data->insertData(value_ref.data, value_ref.size); + current_offset += 1; + } + } + + offsets->insert(current_offset); + } + + auto result_nested_column = ColumnArray::create( + ColumnTuple::create(Columns{std::move(keys_data), std::move(values_data)}), + std::move(offsets)); + + return ColumnMap::create(result_nested_column); + } +}; + } void registerFunctionsMap(FunctionFactory & factory) @@ -391,6 +526,7 @@ void registerFunctionsMap(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); } } diff --git a/tests/queries/0_stateless/02111_function_mapExtractKeyLike.reference b/tests/queries/0_stateless/02111_function_mapExtractKeyLike.reference new file mode 100644 index 00000000000..45edbc24c75 --- /dev/null +++ b/tests/queries/0_stateless/02111_function_mapExtractKeyLike.reference @@ -0,0 +1,23 @@ +The data of table: +1 {'P1-K1':'1-V1','P2-K2':'1-V2'} +2 {'P1-K1':'2-V1','P2-K2':'2-V2'} +3 {'P1-K1':'3-V1','P2-K2':'3-V2'} +4 {'P1-K1':'4-V1','P2-K2':'4-V2'} +5 {'5-K1':'5-V1','5-K2':'5-V2'} +6 {'P3-K1':'6-V1','P4-K2':'6-V2'} + +The results of query: SELECT id, mapExtractKeyLike(map, \'P1%\') FROM map_extractKeyLike_test ORDER BY id; +1 {'P1-K1':'1-V1'} +2 {'P1-K1':'2-V1'} +3 {'P1-K1':'3-V1'} +4 {'P1-K1':'4-V1'} +5 {} +6 {} + +The results of query: SELECT id, mapExtractKeyLike(map, \'5-K1\') FROM map_extractKeyLike_test ORDER BY id; +1 {} +2 {} +3 {} +4 {} +5 {'5-K1':'5-V1'} +6 {} diff --git a/tests/queries/0_stateless/02111_function_mapExtractKeyLike.sql b/tests/queries/0_stateless/02111_function_mapExtractKeyLike.sql new file mode 100644 index 00000000000..31f53642b74 --- /dev/null +++ b/tests/queries/0_stateless/02111_function_mapExtractKeyLike.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS map_extractKeyLike_test; + +CREATE TABLE map_extractKeyLike_test (id UInt32, map Map(String, String)) Engine=MergeTree() ORDER BY id settings index_granularity=2; + +INSERT INTO map_extractKeyLike_test VALUES (1, {'P1-K1':'1-V1','P2-K2':'1-V2'}),(2,{'P1-K1':'2-V1','P2-K2':'2-V2'}); +INSERT INTO map_extractKeyLike_test VALUES (3, {'P1-K1':'3-V1','P2-K2':'3-V2'}),(4,{'P1-K1':'4-V1','P2-K2':'4-V2'}); +INSERT INTO map_extractKeyLike_test VALUES (5, {'5-K1':'5-V1','5-K2':'5-V2'}),(6, {'P3-K1':'6-V1','P4-K2':'6-V2'}); + +SELECT 'The data of table:'; +SELECT * FROM map_extractKeyLike_test ORDER BY id; + +SELECT ''; + +SELECT 'The results of query: SELECT id, mapExtractKeyLike(map, \'P1%\') FROM map_extractKeyLike_test ORDER BY id;'; +SELECT id, mapExtractKeyLike(map, 'P1%') FROM map_extractKeyLike_test ORDER BY id; + +SELECT ''; + +SELECT 'The results of query: SELECT id, mapExtractKeyLike(map, \'5-K1\') FROM map_extractKeyLike_test ORDER BY id;'; +SELECT id, mapExtractKeyLike(map, '5-K1') FROM map_extractKeyLike_test ORDER BY id; + +DROP TABLE map_extractKeyLike_test; From 8d488a46dd36bda795bffa42970820b4bc79025b Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 28 Oct 2021 17:38:51 +0800 Subject: [PATCH 187/396] fix snappy build error: https://github.com/ClickHouse/ClickHouse/issues/30790 --- .gitmodules | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.gitmodules b/.gitmodules index e0404c1269d..dd13597c183 100644 --- a/.gitmodules +++ b/.gitmodules @@ -76,7 +76,8 @@ url = https://github.com/ClickHouse-Extras/libcxxabi.git [submodule "contrib/snappy"] path = contrib/snappy - url = https://github.com/google/snappy + url = https://github.com/taiyang-li/snappy.git + branch = fix_snappy_build_error [submodule "contrib/cppkafka"] path = contrib/cppkafka url = https://github.com/mfontanini/cppkafka.git From d0fe359a23c3cf8add24f1083726e798fd7f22d6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 28 Oct 2021 12:34:38 +0200 Subject: [PATCH 188/396] Update docs to stop referencing insert_sample_with_metadata --- docs/en/interfaces/formats.md | 2 +- docs/ja/interfaces/formats.md | 2 +- docs/zh/interfaces/formats.md | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index d0e5c44b4f7..e7da2697fe4 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -704,7 +704,7 @@ CREATE TABLE IF NOT EXISTS example_table - If `input_format_defaults_for_omitted_fields = 1`, then the default value for `x` equals `0`, but the default value of `a` equals `x * 2`. !!! note "Warning" - When inserting data with `insert_sample_with_metadata = 1`, ClickHouse consumes more computational resources, compared to insertion with `insert_sample_with_metadata = 0`. + When inserting data with `input_format_defaults_for_omitted_fields = 1`, ClickHouse consumes more computational resources, compared to insertion with `input_format_defaults_for_omitted_fields = 0`. ### Selecting Data {#selecting-data} diff --git a/docs/ja/interfaces/formats.md b/docs/ja/interfaces/formats.md index bebfd438c1e..b34c467c980 100644 --- a/docs/ja/interfaces/formats.md +++ b/docs/ja/interfaces/formats.md @@ -559,7 +559,7 @@ CREATE TABLE IF NOT EXISTS example_table - もし `input_format_defaults_for_omitted_fields = 1` のデフォルト値 `x` 等しい `0` しかし、デフォルト値は `a` 等しい `x * 2`. !!! note "警告" - データを挿入するとき `insert_sample_with_metadata = 1`,ClickHouseは、挿入と比較して、より多くの計算リソースを消費します `insert_sample_with_metadata = 0`. + データを挿入するとき `input_format_defaults_for_omitted_fields = 1`,ClickHouseは、挿入と比較して、より多くの計算リソースを消費します `input_format_defaults_for_omitted_fields = 0`. ### データの選択 {#selecting-data} diff --git a/docs/zh/interfaces/formats.md b/docs/zh/interfaces/formats.md index 85af2abda6b..b579d57c634 100644 --- a/docs/zh/interfaces/formats.md +++ b/docs/zh/interfaces/formats.md @@ -685,7 +685,7 @@ CREATE TABLE IF NOT EXISTS example_table - 如果`input_format_defaults_for_omitted_fields = 1`, 那么`x`的默认值为`0`,但`a`的默认值为`x * 2`。 !!! note "注意" -当使用`insert_sample_with_metadata = 1`插入数据时,与使用`insert_sample_with_metadata = 0`相比,ClickHouse消耗更多的计算资源。 +当使用`input_format_defaults_for_omitted_fields = 1`插入数据时,与使用`input_format_defaults_for_omitted_fields = 0`相比,ClickHouse消耗更多的计算资源。 ### Selecting Data {#selecting-data} From 0ce893cb1017f75dff866ada69d8b85ac2ec7e27 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 28 Oct 2021 14:20:29 +0300 Subject: [PATCH 189/396] Fix kafka test --- tests/integration/test_storage_kafka/test.py | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 78066b36143..4bf778531a7 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2850,9 +2850,8 @@ def test_kafka_formats_with_broken_message(kafka_cluster): # broken message b'\x05\x02\x69\x64\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x04\x76\x61\x6c\x31\x04\x76\x61\x6c\x32\x04\x76\x61\x6c\x33\x05\x49\x6e\x74\x36\x34\x06\x53\x74\x72\x69\x6e\x67\x06\x53\x74\x72\x69\x6e\x67\x07\x46\x6c\x6f\x61\x74\x33\x32\x05\x55\x49\x6e\x74\x38\x00\x00\x00\x00\x00\x00\x00\x00\x03\x42\x41\x44\x02\x41\x4d\x00\x00\x00\x3f\x01', ], - 'expected':'{"raw_message":"0502696407626C6F636B4E6F0476616C310476616C320476616C3305496E74363406537472696E6706537472696E6707466C6F617433320555496E743800000000000000000342414402414D0000003F01","error":"Cannot read all data. Bytes read: 9. Bytes expected: 65.: (at row 1)\\n"}', + 'expected':'{"raw_message":"0502696407626C6F636B4E6F0476616C310476616C320476616C3305496E74363406537472696E6706537472696E6707466C6F617433320555496E743800000000000000000342414402414D0000003F01","error":"Type of \'blockNo\' must be UInt16, not String"}', 'printable':False, - 'format_settings':'input_format_with_types_use_header=0', }, 'ORC': { 'data_sample': [ @@ -2914,11 +2913,7 @@ def test_kafka_formats_with_broken_message(kafka_cluster): 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] - 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)) + result = instance.query('SELECT * FROM test.kafka_data_{format_name}_mv;'.format(format_name=format_name)) expected = '''\ 0 0 AM 0.5 1 {topic_name} 0 {offset_0} 1 0 AM 0.5 1 {topic_name} 0 {offset_1} From a37bcb1feb89aa9fe0faf9375c6aa49c891b0369 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 28 Oct 2021 14:32:49 +0300 Subject: [PATCH 190/396] Fix docs --- docs/en/interfaces/formats.md | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index fb023ff1e36..29dd81b128e 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -195,7 +195,7 @@ When parsing with this format, tabs or linefeeds are not allowed in each field. This format is also available under the name `TSVRawWithNames`. -## TabSeparatedWithNamesAndTypes {#tabseparatedwithnamesandtypes} +## TabSeparatedWithNamesAndTypes {#tabseparatedrawwithnamesandtypes} Differs from `TabSeparatedWithNamesAndTypes` format in that the rows are written without escaping. When parsing with this format, tabs or linefeeds are not allowed in each field. @@ -689,15 +689,19 @@ Differs from `JSONEachRow`/`JSONStringsEachRow` in that ClickHouse will also yie ## JSONCompactEachRowWithNames {#jsoncompacteachrowwithnames} -Differs from `JSONCompactEachRow` format in that the also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). +Differs from `JSONCompactEachRow` format in that it also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). ## JSONCompactEachRowWithNamesAndTypes {#jsoncompacteachrowwithnamesandtypes} -Differs from `JSONCompactEachRow` format in that the also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). +Differs from `JSONCompactEachRow` format in that it also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). + +## JSONCompactStringsEachRowWithNames {#jsoncompactstringseachrowwithnames} + +Differs from `JSONCompactStringsEachRow` in that in that it also prints the header row with column names, similar to [TabSeparatedWithNames](#tabseparatedwithnames). ## JSONCompactStringsEachRowWithNamesAndTypes {#jsoncompactstringseachrowwithnamesandtypes} -Differs from `JSONCompactEachRow`/`JSONCompactStringsEachRow` in that the column names and types are written as the first two rows. +Differs from `JSONCompactStringsEachRow` in that it also prints two header rows with column names and types, similar to [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes). ```json ["'hello'", "multiply(42, number)", "range(5)"] From b16ddddbeb1fa51eed0074d964f90edf1cfc9819 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 28 Oct 2021 15:28:39 +0300 Subject: [PATCH 191/396] Update distributed_push_down_limit starting version --- docs/en/operations/settings/settings.md | 2 +- docs/ru/operations/settings/settings.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index e12ba544b7b..46806d7d101 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1794,7 +1794,7 @@ This will allow to avoid: - Sending extra rows over network; - Processing rows behind the limit on the initiator. -You cannot get inaccurate results anymore, since `distributed_push_down_limit` changes query execution only if at least one of the conditions met: +Starting from 21.9 version you cannot get inaccurate results anymore, since `distributed_push_down_limit` changes query execution only if at least one of the conditions met: - [distributed_group_by_no_merge](#distributed-group-by-no-merge) > 0. - Query **does not have** `GROUP BY`/`DISTINCT`/`LIMIT BY`, but it has `ORDER BY`/`LIMIT`. - Query **has** `GROUP BY`/`DISTINCT`/`LIMIT BY` with `ORDER BY`/`LIMIT` and: diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 1bfb039069d..70dc0e349e4 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1713,7 +1713,7 @@ ClickHouse генерирует исключение - отправки дополнительных строк по сети; - обработки строк за пределами ограничения для инициатора. -Вы больше не сможете получить неточные результаты, так как `distributed_push_down_limit` изменяет выполнение запроса только в том случае, если выполнено хотя бы одно из условий: +Начиная с версии 21.9 вы больше не сможете получить неточные результаты, так как `distributed_push_down_limit` изменяет выполнение запроса только в том случае, если выполнено хотя бы одно из условий: - `distributed_group_by_no_merge` > 0. - запрос **не содержит** `GROUP BY`/`DISTINCT`/`LIMIT BY`, но содержит `ORDER BY`/`LIMIT`. - запрос **содержит** `GROUP BY`/`DISTINCT`/`LIMIT BY` с `ORDER BY`/`LIMIT` и: From d39840376fe3dea678ae3d99efe25d2fcb004dbe Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Oct 2021 15:44:12 +0300 Subject: [PATCH 192/396] Review fixes --- src/Processors/Sinks/SinkToStorage.h | 1 + .../ExternalDataSourceConfiguration.cpp | 6 +- .../ExternalDataSourceConfiguration.h | 2 +- src/Storages/StorageFile.cpp | 138 +++++++++++++----- src/Storages/StorageURL.cpp | 47 +++--- src/Storages/StorageURL.h | 2 +- src/TableFunctions/TableFunctionURL.cpp | 11 +- .../0_stateless/01944_insert_partition_by.sql | 1 - 8 files changed, 134 insertions(+), 74 deletions(-) diff --git a/src/Processors/Sinks/SinkToStorage.h b/src/Processors/Sinks/SinkToStorage.h index 051ecc8f20a..01d51940d64 100644 --- a/src/Processors/Sinks/SinkToStorage.h +++ b/src/Processors/Sinks/SinkToStorage.h @@ -8,6 +8,7 @@ namespace DB /// Sink which is returned from Storage::write. class SinkToStorage : public ExceptionKeepingTransform { +/// PartitionedSink owns nested sinks. friend class PartitionedSink; public: diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index c77acead06d..dcdcb2cbefb 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -259,7 +259,7 @@ void URLBasedDataSourceConfiguration::set(const URLBasedDataSourceConfiguration format = conf.format; compression_method = conf.compression_method; structure = conf.structure; - method = conf.method; + http_method = conf.http_method; } @@ -289,7 +289,7 @@ std::optional getURLBasedDataSourceConfiguration(const } else if (key == "method") { - configuration.method = config.getString(config_prefix + ".method", ""); + configuration.http_method = config.getString(config_prefix + ".method", ""); } else if (key == "format") { @@ -333,7 +333,7 @@ std::optional getURLBasedDataSourceConfiguration(const if (arg_name == "url") configuration.url = arg_value.safeGet(); if (arg_name == "method") - configuration.method = arg_value.safeGet(); + configuration.http_method = arg_value.safeGet(); else if (arg_name == "format") configuration.format = arg_value.safeGet(); else if (arg_name == "compression_method") diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index 203b4933336..7b6b4cc85b5 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -93,7 +93,7 @@ struct URLBasedDataSourceConfiguration String structure; std::vector> headers; - String method; + String http_method; void set(const URLBasedDataSourceConfiguration & conf); }; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 1f2b6ba6c60..6eec02e9414 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -548,37 +548,57 @@ class StorageFileSink final : public SinkToStorage { public: StorageFileSink( - StorageFile & storage_, const StorageMetadataPtr & metadata_snapshot_, + const String & table_name_for_log_, + int table_fd_, + bool use_table_fd_, + std::string base_path_, + std::vector paths_, const CompressionMethod compression_method_, - ContextPtr context_, const std::optional & format_settings_, + const String format_name_, + ContextPtr context_, int flags_) : SinkToStorage(metadata_snapshot_->getSampleBlock()) - , storage(storage_) , metadata_snapshot(metadata_snapshot_) + , table_name_for_log(table_name_for_log_) + , table_fd(table_fd_) + , use_table_fd(use_table_fd_) + , base_path(base_path_) + , paths(paths_) , compression_method(compression_method_) - , context(context_) + , format_name(format_name_) , format_settings(format_settings_) + , context(context_) , flags(flags_) { initialize(); } StorageFileSink( - StorageFile & storage_, const StorageMetadataPtr & metadata_snapshot_, + const String & table_name_for_log_, std::unique_lock && lock_, + int table_fd_, + bool use_table_fd_, + std::string base_path_, + std::vector paths_, const CompressionMethod compression_method_, - ContextPtr context_, const std::optional & format_settings_, + const String format_name_, + ContextPtr context_, int flags_) : SinkToStorage(metadata_snapshot_->getSampleBlock()) - , storage(storage_) , metadata_snapshot(metadata_snapshot_) + , table_name_for_log(table_name_for_log_) + , table_fd(table_fd_) + , use_table_fd(use_table_fd_) + , base_path(base_path_) + , paths(paths_) , compression_method(compression_method_) - , context(context_) + , format_name(format_name_) , format_settings(format_settings_) + , context(context_) , flags(flags_) , lock(std::move(lock_)) { @@ -590,16 +610,16 @@ public: void initialize() { std::unique_ptr naked_buffer = nullptr; - if (storage.use_table_fd) + if (use_table_fd) { - naked_buffer = std::make_unique(storage.table_fd, DBMS_DEFAULT_BUFFER_SIZE); + naked_buffer = std::make_unique(table_fd, DBMS_DEFAULT_BUFFER_SIZE); } else { - if (storage.paths.size() != 1) - throw Exception("Table '" + storage.getStorageID().getNameForLogs() + "' is in readonly mode because of globs in filepath", ErrorCodes::DATABASE_ACCESS_DENIED); + if (paths.size() != 1) + throw Exception("Table '" + table_name_for_log + "' is in readonly mode because of globs in filepath", ErrorCodes::DATABASE_ACCESS_DENIED); flags |= O_WRONLY | O_APPEND | O_CREAT; - naked_buffer = std::make_unique(storage.paths[0], DBMS_DEFAULT_BUFFER_SIZE, flags); + naked_buffer = std::make_unique(paths[0], DBMS_DEFAULT_BUFFER_SIZE, flags); } /// In case of CSVWithNames we have already written prefix. @@ -608,7 +628,7 @@ public: write_buf = wrapWriteBufferWithCompressionMethod(std::move(naked_buffer), compression_method, 3); - writer = FormatFactory::instance().getOutputFormatParallelIfPossible(storage.format_name, + writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format_name, *write_buf, metadata_snapshot->getSampleBlock(), context, {}, format_settings); } @@ -638,18 +658,24 @@ public: // } private: - StorageFile & storage; StorageMetadataPtr metadata_snapshot; - const CompressionMethod compression_method; - ContextPtr context; - std::optional format_settings; - int flags; - - std::unique_lock lock; + String table_name_for_log; std::unique_ptr write_buf; OutputFormatPtr writer; bool prefix_written{false}; + + int table_fd; + bool use_table_fd; + std::string base_path; + std::vector paths; + CompressionMethod compression_method; + std::string format_name; + std::optional format_settings; + + ContextPtr context; + int flags; + std::unique_lock lock; }; class PartitionedStorageFileSink : public PartitionedSink @@ -657,22 +683,32 @@ class PartitionedStorageFileSink : public PartitionedSink public: PartitionedStorageFileSink( const ASTPtr & partition_by, - StorageFile & storage_, const StorageMetadataPtr & metadata_snapshot_, + const String & table_name_for_log_, std::unique_lock && lock_, + int table_fd_, + bool use_table_fd_, + std::string base_path_, + std::vector paths_, const CompressionMethod compression_method_, - ContextPtr context_, const std::optional & format_settings_, - int & flags_) + const String format_name_, + ContextPtr context_, + int flags_) : PartitionedSink(partition_by, context_, metadata_snapshot_->getSampleBlock()) - , path(storage_.paths[0]) - , storage(storage_) + , path(paths_[0]) , metadata_snapshot(metadata_snapshot_) - , lock(std::move(lock_)) + , table_name_for_log(table_name_for_log_) + , table_fd(table_fd_) + , use_table_fd(use_table_fd_) + , base_path(base_path_) + , paths(paths_) , compression_method(compression_method_) - , context(context_) + , format_name(format_name_) , format_settings(format_settings_) + , context(context_) , flags(flags_) + , lock(std::move(lock_)) { } @@ -680,21 +716,37 @@ public: { auto partition_path = PartitionedSink::replaceWildcards(path, partition_id); PartitionedSink::validatePartitionKey(partition_path, true); - storage.paths[0] = partition_path; + Strings result_paths = {partition_path}; return std::make_shared( - storage, metadata_snapshot, compression_method, context, format_settings, flags); + metadata_snapshot, + table_name_for_log, + table_fd, + use_table_fd, + base_path, + result_paths, + compression_method, + format_settings, + format_name, + context, + flags); } private: const String path; - StorageFile & storage; StorageMetadataPtr metadata_snapshot; - std::unique_lock lock; - const CompressionMethod compression_method; + String table_name_for_log; + + int table_fd; + bool use_table_fd; + std::string base_path; + std::vector paths; + CompressionMethod compression_method; + std::string format_name; + std::optional format_settings; ContextPtr context; - std::optional format_settings; int flags; + std::unique_lock lock; }; @@ -731,23 +783,33 @@ SinkToStoragePtr StorageFile::write( return std::make_shared( insert_query->partition_by, - *this, metadata_snapshot, + getStorageID().getNameForLogs(), std::unique_lock{rwlock, getLockTimeout(context)}, + table_fd, + use_table_fd, + base_path, + paths, chooseCompressionMethod(path, compression_method), - context, format_settings, + format_name, + context, flags); } else { return std::make_shared( - *this, metadata_snapshot, + getStorageID().getNameForLogs(), std::unique_lock{rwlock, getLockTimeout(context)}, + table_fd, + use_table_fd, + base_path, + paths, chooseCompressionMethod(path, compression_method), - context, format_settings, + format_name, + context, flags); } } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 328da6949be..41a332ed999 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -49,7 +49,7 @@ IStorageURLBase::IStorageURLBase( const String & comment, const String & compression_method_, const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_, - const String & method_, + const String & http_method_, ASTPtr partition_by_) : IStorage(table_id_) , uri(uri_) @@ -57,7 +57,7 @@ IStorageURLBase::IStorageURLBase( , format_name(format_name_) , format_settings(format_settings_) , headers(headers_) - , method(method_) + , http_method(http_method_) , partition_by(partition_by_) { StorageInMemoryMetadata storage_metadata; @@ -100,7 +100,7 @@ namespace public: StorageURLSource( const std::vector & uri_options, - const std::string & method, + const std::string & http_method, std::function callback, const String & format, const std::optional & format_settings, @@ -131,7 +131,7 @@ namespace read_buf = wrapReadBufferWithCompressionMethod( std::make_unique( request_uri, - method, + http_method, callback, timeouts, context->getSettingsRef().max_http_get_redirects, @@ -212,11 +212,11 @@ StorageURLSink::StorageURLSink( ContextPtr context, const ConnectionTimeouts & timeouts, const CompressionMethod compression_method, - const String & method) + const String & http_method) : SinkToStorage(sample_block) { write_buf = wrapWriteBufferWithCompressionMethod( - std::make_unique(Poco::URI(uri), method, timeouts), + std::make_unique(Poco::URI(uri), http_method, timeouts), compression_method, 3); writer = FormatFactory::instance().getOutputFormat(format, *write_buf, sample_block, context, {} /* write callback */, format_settings); @@ -253,7 +253,7 @@ public: ContextPtr context_, const ConnectionTimeouts & timeouts_, const CompressionMethod compression_method_, - const String & method_) + const String & http_method_) : PartitionedSink(partition_by, context_, sample_block_) , uri(uri_) , format(format_) @@ -262,7 +262,7 @@ public: , context(context_) , timeouts(timeouts_) , compression_method(compression_method_) - , method(method_) + , http_method(http_method_) { } @@ -271,7 +271,7 @@ public: auto partition_path = PartitionedSink::replaceWildcards(uri, partition_id); context->getRemoteHostFilter().checkURL(Poco::URI(partition_path)); return std::make_shared(partition_path, format, - format_settings, sample_block, context, timeouts, compression_method, method); + format_settings, sample_block, context, timeouts, compression_method, http_method); } private: @@ -283,7 +283,7 @@ private: const ConnectionTimeouts timeouts; const CompressionMethod compression_method; - const String method; + const String http_method; }; std::string IStorageURLBase::getReadMethod() const @@ -324,7 +324,7 @@ Pipe IStorageURLBase::read( unsigned /*num_streams*/) { auto params = getReadURIParams(column_names, metadata_snapshot, query_info, local_context, processed_stage, max_block_size); - auto with_globs = (uri.find('{') != std::string::npos && uri.find('}') != std::string::npos) || uri.find('|') == std::string::npos; + bool with_globs = uri.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; if (with_globs) { size_t max_addresses = local_context->getSettingsRef().glob_expansion_max_elements; @@ -410,8 +410,8 @@ Pipe StorageURLWithFailover::read( SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) { - if (method.empty()) - method = Poco::Net::HTTPRequest::HTTP_POST; + if (http_method.empty()) + http_method = Poco::Net::HTTPRequest::HTTP_POST; bool has_wildcards = uri.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; const auto * insert_query = dynamic_cast(query.get()); @@ -425,14 +425,14 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad uri, format_name, format_settings, metadata_snapshot->getSampleBlock(), context, ConnectionTimeouts::getHTTPTimeouts(context), - chooseCompressionMethod(uri, compression_method), method); + chooseCompressionMethod(uri, compression_method), http_method); } else { return std::make_shared(uri, format_name, format_settings, metadata_snapshot->getSampleBlock(), context, ConnectionTimeouts::getHTTPTimeouts(context), - chooseCompressionMethod(uri, compression_method), method); + chooseCompressionMethod(uri, compression_method), http_method); } } @@ -447,10 +447,10 @@ StorageURL::StorageURL( ContextPtr context_, const String & compression_method_, const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_, - const String & method_, + const String & http_method_, ASTPtr partition_by_) : IStorageURLBase(uri_, context_, table_id_, format_name_, format_settings_, - columns_, constraints_, comment, compression_method_, headers_, method_, partition_by_) + columns_, constraints_, comment, compression_method_, headers_, http_method_, partition_by_) { context_->getRemoteHostFilter().checkURL(Poco::URI(uri)); } @@ -521,12 +521,12 @@ URLBasedDataSourceConfiguration StorageURL::getConfiguration(ASTs & args, Contex auto [common_configuration, storage_specific_args] = named_collection.value(); configuration.set(common_configuration); - if (!configuration.method.empty() - && configuration.method != Poco::Net::HTTPRequest::HTTP_POST - && configuration.method != Poco::Net::HTTPRequest::HTTP_PUT) + if (!configuration.http_method.empty() + && configuration.http_method != Poco::Net::HTTPRequest::HTTP_POST + && configuration.http_method != Poco::Net::HTTPRequest::HTTP_PUT) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Method can be POST or PUT (current: {}). For insert default is POST, for select GET", - configuration.method); + "Http method can be POST or PUT (current: {}). For insert default is POST, for select GET", + configuration.http_method); if (!storage_specific_args.empty()) { @@ -590,12 +590,11 @@ void registerStorageURL(StorageFactory & factory) args.getContext(), configuration.compression_method, headers, - configuration.method, + configuration.http_method, partition_by); }, { .supports_settings = true, - .supports_sort_order = true, // for partition by .source_access_type = AccessType::URL, }); } diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index fcc0c231778..a79100c8d70 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -65,7 +65,7 @@ protected: // In this case, format_settings is not set. std::optional format_settings; ReadWriteBufferFromHTTP::HTTPHeaderEntries headers; - String method; /// For insert can choose Put instead of default Post. + String http_method; /// For insert can choose Put instead of default Post. ASTPtr partition_by; virtual std::string getReadMethod() const; diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index 3fb6fcc2433..904bd5dcb90 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -29,12 +29,12 @@ void TableFunctionURL::parseArguments(const ASTPtr & ast_function, ContextPtr co auto [common_configuration, storage_specific_args] = with_named_collection.value(); configuration.set(common_configuration); - if (!configuration.method.empty() - && configuration.method != Poco::Net::HTTPRequest::HTTP_POST - && configuration.method != Poco::Net::HTTPRequest::HTTP_PUT) + if (!configuration.http_method.empty() + && configuration.http_method != Poco::Net::HTTPRequest::HTTP_POST + && configuration.http_method != Poco::Net::HTTPRequest::HTTP_PUT) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Method can be POST or PUT (current: {}). For insert default is POST, for select GET", - configuration.method); + configuration.http_method); if (!storage_specific_args.empty()) { @@ -82,8 +82,7 @@ StoragePtr TableFunctionURL::getStorage( global_context, compression_method_, headers, - configuration.method); -} + configuration.http_method);} void registerTableFunctionURL(TableFunctionFactory & factory) { diff --git a/tests/queries/0_stateless/01944_insert_partition_by.sql b/tests/queries/0_stateless/01944_insert_partition_by.sql index 5396ca4daf6..ac38fcee490 100644 --- a/tests/queries/0_stateless/01944_insert_partition_by.sql +++ b/tests/queries/0_stateless/01944_insert_partition_by.sql @@ -1,7 +1,6 @@ -- Tags: no-fasttest -- Tag no-fasttest: needs s3 -INSERT INTO TABLE FUNCTION file('foo.csv', 'CSV', 'id Int32, val Int32') PARTITION BY val VALUES (1, 1), (2, 2); -- { serverError NOT_IMPLEMENTED } INSERT INTO TABLE FUNCTION s3('http://localhost:9001/foo/test_{_partition_id}.csv', 'admin', 'admin', 'CSV', 'id Int32, val String') PARTITION BY val VALUES (1, '\r\n'); -- { serverError CANNOT_PARSE_TEXT } INSERT INTO TABLE FUNCTION s3('http://localhost:9001/foo/test_{_partition_id}.csv', 'admin', 'admin', 'CSV', 'id Int32, val String') PARTITION BY val VALUES (1, 'abc\x00abc'); -- { serverError CANNOT_PARSE_TEXT } INSERT INTO TABLE FUNCTION s3('http://localhost:9001/foo/test_{_partition_id}.csv', 'admin', 'admin', 'CSV', 'id Int32, val String') PARTITION BY val VALUES (1, 'abc\xc3\x28abc'); -- { serverError CANNOT_PARSE_TEXT } From e462c3be6c53f75bb7cefe439cd24a1db7cb438f Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 28 Oct 2021 16:04:17 +0300 Subject: [PATCH 193/396] Update adopters.md --- docs/en/introduction/adopters.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 6dd53fffb86..c03ea12a531 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -29,7 +29,7 @@ toc_title: Adopters | Benocs | Network Telemetry and Analytics | Main Product | — | — | [Slides in English, October 2017](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup9/lpm.pdf) | | BIGO | Video | Computing Platform | — | — | [Blog Article, August 2020](https://www.programmersought.com/article/44544895251/) | | BiliBili | Video sharing | — | — | — | [Blog post, June 2021](https://chowdera.com/2021/06/20210622012241476b.html) | -| Bloomberg | Finance, Media | Monitoring | — | — | [Slides, May 2018](https://www.slideshare.net/Altinity/http-analytics-for-6m-requests-per-second-using-clickhouse-by-alexander-bocharov) | +| Bloomberg | Finance, Media | Monitoring | — | — | [Job opening, October 2021](https://www.efinancialcareers.com/jobs-USA-NY-New_York-Senior_Software_Engineer_-_Bloomberg_Second_Measure_Platform.id12029277), [slides, May 2018](https://www.slideshare.net/Altinity/http-analytics-for-6m-requests-per-second-using-clickhouse-by-alexander-bocharov) | | Bloxy | Blockchain | Analytics | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/4_bloxy.pptx) | | Bytedance | Social platforms | — | — | — | [The ClickHouse Meetup East, October 2020](https://www.youtube.com/watch?v=ckChUkC3Pns) | | CardsMobile | Finance | Analytics | — | — | [VC.ru](https://vc.ru/s/cardsmobile/143449-rukovoditel-gruppy-analiza-dannyh) | From 8095833a9fe4883676ccdd7624b644ef74637b9a Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 28 Oct 2021 16:20:01 +0300 Subject: [PATCH 194/396] Update adopters.md --- docs/en/introduction/adopters.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index c03ea12a531..7d33a8c0796 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -29,7 +29,7 @@ toc_title: Adopters | Benocs | Network Telemetry and Analytics | Main Product | — | — | [Slides in English, October 2017](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup9/lpm.pdf) | | BIGO | Video | Computing Platform | — | — | [Blog Article, August 2020](https://www.programmersought.com/article/44544895251/) | | BiliBili | Video sharing | — | — | — | [Blog post, June 2021](https://chowdera.com/2021/06/20210622012241476b.html) | -| Bloomberg | Finance, Media | Monitoring | — | — | [Job opening, October 2021](https://www.efinancialcareers.com/jobs-USA-NY-New_York-Senior_Software_Engineer_-_Bloomberg_Second_Measure_Platform.id12029277), [slides, May 2018](https://www.slideshare.net/Altinity/http-analytics-for-6m-requests-per-second-using-clickhouse-by-alexander-bocharov) | +| Bloomberg | Finance, Media | Monitoring | — | — | [Job opening, September 2021](https://careers.bloomberg.com/job/detail/94913), [slides, May 2018](https://www.slideshare.net/Altinity/http-analytics-for-6m-requests-per-second-using-clickhouse-by-alexander-bocharov) | | Bloxy | Blockchain | Analytics | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/4_bloxy.pptx) | | Bytedance | Social platforms | — | — | — | [The ClickHouse Meetup East, October 2020](https://www.youtube.com/watch?v=ckChUkC3Pns) | | CardsMobile | Finance | Analytics | — | — | [VC.ru](https://vc.ru/s/cardsmobile/143449-rukovoditel-gruppy-analiza-dannyh) | From 6e8c2ab28f2da979d214caa8df90ed2fc8c5e87a Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 28 Oct 2021 17:02:07 +0300 Subject: [PATCH 195/396] Use small amount of memory on stack in PeekableReadBuffer --- src/IO/PeekableReadBuffer.cpp | 48 ++++++++++++++----- src/IO/PeekableReadBuffer.h | 15 ++++-- ...v_csv_custom_null_representation.reference | 21 ++++++++ ...2103_tsv_csv_custom_null_representation.sh | 7 +++ 4 files changed, 74 insertions(+), 17 deletions(-) diff --git a/src/IO/PeekableReadBuffer.cpp b/src/IO/PeekableReadBuffer.cpp index e2b1873283f..40929acd848 100644 --- a/src/IO/PeekableReadBuffer.cpp +++ b/src/IO/PeekableReadBuffer.cpp @@ -9,8 +9,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -PeekableReadBuffer::PeekableReadBuffer(ReadBuffer & sub_buf_, bool use_existing_memory /*= false*/, size_t start_size_ /*= DBMS_DEFAULT_BUFFER_SIZE*/) - : BufferWithOwnMemory(use_existing_memory ? sizeof(existing_memory) : start_size_, use_existing_memory ? existing_memory : nullptr), sub_buf(sub_buf_) +PeekableReadBuffer::PeekableReadBuffer(ReadBuffer & sub_buf_, size_t start_size_ /*= 0*/) + : BufferWithOwnMemory(start_size_), sub_buf(sub_buf_) { padded &= sub_buf.isPadded(); /// Read from sub-buffer @@ -27,6 +27,7 @@ void PeekableReadBuffer::reset() peeked_size = 0; checkpoint = std::nullopt; checkpoint_in_own_memory = false; + use_stack_memory = true; if (!currentlyReadFromOwnMemory()) sub_buf.position() = pos; @@ -72,21 +73,23 @@ bool PeekableReadBuffer::peekNext() sub_buf.position() = copy_from; } + char * memory_data = getMemoryData(); + /// Save unread data from sub-buffer to own memory - memcpy(memory.data() + peeked_size, sub_buf.position(), bytes_to_copy); + memcpy(memory_data + peeked_size, sub_buf.position(), bytes_to_copy); /// If useSubbufferOnly() is false, then checkpoint is in own memory and it was updated in resizeOwnMemoryIfNecessary /// Otherwise, checkpoint now at the beginning of own memory if (checkpoint && useSubbufferOnly()) { - checkpoint.emplace(memory.data()); + checkpoint.emplace(memory_data); checkpoint_in_own_memory = true; } if (currentlyReadFromOwnMemory()) { /// Update buffer size - BufferBase::set(memory.data(), peeked_size + bytes_to_copy, offset()); + BufferBase::set(memory_data, peeked_size + bytes_to_copy, offset()); } else { @@ -99,7 +102,7 @@ bool PeekableReadBuffer::peekNext() else pos_offset = 0; } - BufferBase::set(memory.data(), peeked_size + bytes_to_copy, pos_offset); + BufferBase::set(memory_data, peeked_size + bytes_to_copy, pos_offset); } peeked_size += bytes_to_copy; @@ -125,8 +128,9 @@ void PeekableReadBuffer::rollbackToCheckpoint(bool drop) /// Checkpoint is in own memory and position is not. assert(checkpointInOwnMemory()); + char * memory_data = getMemoryData(); /// Switch to reading from own memory. - BufferBase::set(memory.data(), peeked_size, *checkpoint - memory.data()); + BufferBase::set(memory_data, peeked_size, *checkpoint - memory_data); } if (drop) @@ -224,12 +228,31 @@ void PeekableReadBuffer::resizeOwnMemoryIfNecessary(size_t bytes_to_append) bool need_update_pos = currentlyReadFromOwnMemory(); size_t offset = 0; if (need_update_checkpoint) - offset = *checkpoint - memory.data(); + { + char * memory_data = getMemoryData(); + offset = *checkpoint - memory_data; + } else if (need_update_pos) offset = this->offset(); size_t new_size = peeked_size + bytes_to_append; - if (memory.size() < new_size) + + if (use_stack_memory) + { + /// If stack memory is still enough, do nothing. + if (sizeof(stack_memory) >= new_size) + return; + + /// Stack memory is not enough, allocate larger buffer. + use_stack_memory = false; + memory.resize(std::max(size_t(DBMS_DEFAULT_BUFFER_SIZE), new_size)); + memcpy(memory.data(), stack_memory, sizeof(stack_memory)); + if (need_update_checkpoint) + checkpoint.emplace(memory.data() + offset); + if (need_update_pos) + BufferBase::set(memory.data(), peeked_size, pos - stack_memory); + } + else if (memory.size() < new_size) { if (bytes_to_append < offset && 2 * (peeked_size - offset) <= memory.size()) { @@ -273,10 +296,11 @@ void PeekableReadBuffer::makeContinuousMemoryFromCheckpointToPos() size_t bytes_to_append = pos - sub_buf.position(); resizeOwnMemoryIfNecessary(bytes_to_append); - memcpy(memory.data() + peeked_size, sub_buf.position(), bytes_to_append); + char * memory_data = getMemoryData(); + memcpy(memory_data + peeked_size, sub_buf.position(), bytes_to_append); sub_buf.position() = pos; peeked_size += bytes_to_append; - BufferBase::set(memory.data(), peeked_size, peeked_size); + BufferBase::set(memory_data, peeked_size, peeked_size); } PeekableReadBuffer::~PeekableReadBuffer() @@ -287,7 +311,7 @@ PeekableReadBuffer::~PeekableReadBuffer() bool PeekableReadBuffer::hasUnreadData() const { - return peeked_size && pos != memory.data() + peeked_size; + return peeked_size && pos != getMemoryData() + peeked_size; } } diff --git a/src/IO/PeekableReadBuffer.h b/src/IO/PeekableReadBuffer.h index a8eff09c4f2..f22987d9daa 100644 --- a/src/IO/PeekableReadBuffer.h +++ b/src/IO/PeekableReadBuffer.h @@ -20,7 +20,7 @@ class PeekableReadBuffer : public BufferWithOwnMemory { friend class PeekableReadBufferCheckpoint; public: - explicit PeekableReadBuffer(ReadBuffer & sub_buf_, bool use_existing_memory = false, size_t start_size_ = DBMS_DEFAULT_BUFFER_SIZE); + explicit PeekableReadBuffer(ReadBuffer & sub_buf_, size_t start_size_ = 0); ~PeekableReadBuffer() override; @@ -84,16 +84,21 @@ private: /// Updates all invalidated pointers and sizes. void resizeOwnMemoryIfNecessary(size_t bytes_to_append); + char * getMemoryData() { return use_stack_memory ? stack_memory : memory.data(); } + const char * getMemoryData() const { return use_stack_memory ? stack_memory : memory.data(); } + ReadBuffer & sub_buf; size_t peeked_size = 0; std::optional checkpoint = std::nullopt; bool checkpoint_in_own_memory = false; - /// Small amount of memory on stack to use in BufferWithOwnMemory on - /// it's creation to prevent unnecessary allocation if PeekableReadBuffer - /// is often created. - char existing_memory[16]; + /// To prevent expensive and in some cases unnecessary memory allocations on PeekableReadBuffer + /// creation (for example if PeekableReadBuffer is often created or if we need to remember small amount of + /// data after checkpoint), at the beginning we will use small amount of memory on stack and allocate + /// larger buffer only if reserved memory is not enough. + char stack_memory[16]; + bool use_stack_memory = true; }; diff --git a/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.reference b/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.reference index 06618cc63b1..a89bc46acfb 100644 --- a/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.reference +++ b/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.reference @@ -53,3 +53,24 @@ Some text \N Some text CustomNull Some text OK OK +Large custom NULL +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +0000000000Custom NULL representation0000000000 +0000000000Custom NULL representation0000000000 +0000000000Custom NULL representation0000000000 +0000000000Custom NULL representation0000000000 +0000000000Custom NULL representation0000000000 +0000000000Custom NULL representation0000000000 +0000000000Custom NULL representation0000000000 +0000000000Custom NULL representation0000000000 +0000000000Custom NULL representation0000000000 +0000000000Custom NULL representation0000000000 diff --git a/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.sh b/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.sh index 1d8e080c7b6..676e8cb867f 100755 --- a/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.sh +++ b/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.sh @@ -121,5 +121,12 @@ $CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'CSV', 's Stri echo -e "Some text,NU,LL" > $DATA_FILE $CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'CSV', 's String, n Nullable(String)') settings max_read_buffer_size=13, format_csv_null_representation='NU,L', input_format_parallel_parsing=0" 2>&1 | grep -F -q "CANNOT_READ_ALL_DATA" && echo 'OK' || echo 'FAIL' + +echo 'Large custom NULL' + +$CLICKHOUSE_CLIENT -q "select '0000000000Custom NULL representation0000000000' FROM numbers(10)" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'TSV', 's Nullable(String)') SETTINGS max_read_buffer_size=5, input_format_parallel_parsing=0, format_tsv_null_representation='0000000000Custom NULL representation0000000000'" +$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103_null.data', 'TSV', 's Nullable(String)') SETTINGS max_read_buffer_size=5, input_format_parallel_parsing=0, format_tsv_null_representation='0000000000Custom NULL representation000000000'" + rm $DATA_FILE From 6cda420b56dfe9d836af1f106475bb6fd49edc5f Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Fri, 22 Oct 2021 21:20:31 +0300 Subject: [PATCH 196/396] Cleanup after fail in localBackup --- src/Disks/S3/DiskS3.cpp | 41 ++++++++++++++++++++++---- src/Storages/MergeTree/localBackup.cpp | 26 +++++++++++++++- 2 files changed, 60 insertions(+), 7 deletions(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index be07f036d47..a9bc2c97c79 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -127,6 +127,32 @@ void throwIfError(const Aws::Utils::Outcome & response) } } +template +void logIfError(Aws::Utils::Outcome & response, Fn auto && msg) +{ + try + { + throwIfError(response); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__, msg()); + } +} + +template +void logIfError(const Aws::Utils::Outcome & response, Fn auto && msg) +{ + try + { + throwIfError(response); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__, msg()); + } +} + /// Reads data from S3 using stored paths in metadata. class ReadIndirectBufferFromS3 final : public ReadIndirectBufferFromRemoteFS { @@ -185,15 +211,16 @@ void DiskS3::removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) if (s3_paths_keeper) s3_paths_keeper->removePaths([&](S3PathKeeper::Chunk && chunk) { - LOG_TRACE(log, "Remove AWS keys {}", S3PathKeeper::getChunkKeys(chunk)); + String keys = S3PathKeeper::getChunkKeys(chunk); + LOG_TRACE(log, "Remove AWS keys {}", keys); Aws::S3::Model::Delete delkeys; delkeys.SetObjects(chunk); - /// TODO: Make operation idempotent. Do not throw exception if key is already deleted. Aws::S3::Model::DeleteObjectsRequest request; request.SetBucket(bucket); request.SetDelete(delkeys); auto outcome = settings->client->DeleteObjects(request); - throwIfError(outcome); + // Do not throw here, continue deleting other chunks + logIfError(outcome, [&](){return "Can't remove AWS keys: " + keys;}); }); } @@ -512,9 +539,11 @@ bool DiskS3::checkUniqueId(const String & id) const Aws::S3::Model::ListObjectsV2Request request; request.SetBucket(bucket); request.SetPrefix(id); - auto resp = settings->client->ListObjectsV2(request); - throwIfError(resp); - Aws::Vector object_list = resp.GetResult().GetContents(); + + auto outcome = settings->client->ListObjectsV2(request); + throwIfError(outcome); + + Aws::Vector object_list = outcome.GetResult().GetContents(); for (const auto & object : object_list) if (object.GetKey() == id) diff --git a/src/Storages/MergeTree/localBackup.cpp b/src/Storages/MergeTree/localBackup.cpp index 2e02481e065..236a0c5b5e4 100644 --- a/src/Storages/MergeTree/localBackup.cpp +++ b/src/Storages/MergeTree/localBackup.cpp @@ -4,7 +4,6 @@ #include #include - namespace DB { @@ -43,6 +42,27 @@ static void localBackupImpl(const DiskPtr & disk, const String & source_path, co } } +class CleanupOnFail +{ +public: + explicit CleanupOnFail(std::function && cleaner_) : cleaner(cleaner_), is_success(false) {} + + ~CleanupOnFail() + { + if (!is_success) + cleaner(); + } + + void success() + { + is_success = true; + } + +private: + std::function cleaner; + bool is_success; +}; + void localBackup(const DiskPtr & disk, const String & source_path, const String & destination_path, std::optional max_level) { if (disk->exists(destination_path) && !disk->isDirectoryEmpty(destination_path)) @@ -53,6 +73,8 @@ void localBackup(const DiskPtr & disk, const String & source_path, const String size_t try_no = 0; const size_t max_tries = 10; + CleanupOnFail cleanup([&](){disk->removeRecursive(destination_path);}); + /** Files in the directory can be permanently added and deleted. * If some file is deleted during an attempt to make a backup, then try again, * because it's important to take into account any new files that might appear. @@ -88,6 +110,8 @@ void localBackup(const DiskPtr & disk, const String & source_path, const String break; } + + cleanup.success(); } } From 88084c715b94e26dd4718f4e8c530bca3ef41941 Mon Sep 17 00:00:00 2001 From: Konstantin Ilchenko Date: Thu, 28 Oct 2021 17:34:09 +0200 Subject: [PATCH 197/396] [DOCS] Fix ADD PROJECTION example in ru/en docs --- docs/en/sql-reference/statements/alter/projection.md | 2 +- docs/ru/sql-reference/statements/alter/projection.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index 429241ebf13..96cd8f5d607 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -7,7 +7,7 @@ toc_title: PROJECTION The following operations with [projections](../../../engines/table-engines/mergetree-family/mergetree.md#projections) are available: -- `ALTER TABLE [db].name ADD PROJECTION name AS SELECT [GROUP BY] [ORDER BY]` - Adds projection description to tables metadata. +- `ALTER TABLE [db].name ADD PROJECTION name ( SELECT [GROUP BY] [ORDER BY] )` - Adds projection description to tables metadata. - `ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. diff --git a/docs/ru/sql-reference/statements/alter/projection.md b/docs/ru/sql-reference/statements/alter/projection.md index 4b0d7f7865b..fa5f517594c 100644 --- a/docs/ru/sql-reference/statements/alter/projection.md +++ b/docs/ru/sql-reference/statements/alter/projection.md @@ -7,7 +7,7 @@ toc_title: PROJECTION Доступны следующие операции с [проекциями](../../../engines/table-engines/mergetree-family/mergetree.md#projections): -- `ALTER TABLE [db].name ADD PROJECTION name AS SELECT [GROUP BY] [ORDER BY]` — добавляет описание проекции в метаданные. +- `ALTER TABLE [db].name ADD PROJECTION name ( SELECT [GROUP BY] [ORDER BY] )` — добавляет описание проекции в метаданные. - `ALTER TABLE [db].name DROP PROJECTION name` — удаляет описание проекции из метаданных и удаляет файлы проекции с диска. From 536c9990bdca2c2beb8f1ee21739413b70caca69 Mon Sep 17 00:00:00 2001 From: Alexey Date: Thu, 28 Oct 2021 16:12:30 +0000 Subject: [PATCH 198/396] fixed links --- docs/en/operations/settings/settings.md | 2 +- docs/ru/operations/settings/settings.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 6ccb91093e9..520f1f81e39 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. For example, members of a [Tuple](../../sql-reference/data-types/tuple.md) or subcolumns of a [Map](../../sql-reference/data-types/map.md#map-subcolumns), [Nullable](../../sql-reference/data-types/nullable.md#finding-null) or an [Array](../../sql-reference/data-types/array.md/#array-size) data type. +Enables describing subcolumns for a [DESCRIBE](../../sql-reference/statements/describe-table.md) query. For example, members of a [Tuple](../../sql-reference/data-types/tuple.md) or subcolumns of a [Map](../../sql-reference/data-types/map.md#map-subcolumns), [Nullable](../../sql-reference/data-types/nullable.md#finding-null) or an [Array](../../sql-reference/data-types/array.md#array-size) data type. Possible values: diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 02e8d30be18..4c58490680d 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3606,7 +3606,7 @@ SELECT * FROM positional_arguments ORDER BY 2,3; ## describe_include_subcolumns {#describe_include_subcolumns} -Включает или отключает описание подстолбцов при выполнении запроса [DESCRIBE](../../sql-reference/statements/describe-table.md). Настройка действует, например, на элементы [Tuple](../../sql-reference/data-types/tuple.md) или подстолбцы типов [Map](../../sql-reference/data-types/map.md#map-subcolumns), [Nullable](../../sql-reference/data-types/nullable.md#finding-null) или [Array](../../sql-reference/data-types/array.md/#array-size). +Включает или отключает описание подстолбцов при выполнении запроса [DESCRIBE](../../sql-reference/statements/describe-table.md). Настройка действует, например, на элементы [Tuple](../../sql-reference/data-types/tuple.md) или подстолбцы типов [Map](../../sql-reference/data-types/map.md#map-subcolumns), [Nullable](../../sql-reference/data-types/nullable.md#finding-null) или [Array](../../sql-reference/data-types/array.md#array-size). Возможные значения: From c1310841d9a14f95d31beb2361289c455b7e53df Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 28 Oct 2021 19:19:41 +0300 Subject: [PATCH 199/396] fix ambiguity when extracting auxiliary zk name --- src/Interpreters/Context.cpp | 3 ++ src/Storages/StorageReplicatedMergeTree.cpp | 41 ++++++++++++------- .../test.py | 10 +++++ 3 files changed, 39 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 69f9518a912..9490c43203b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1971,6 +1971,9 @@ zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const auto zookeeper = shared->auxiliary_zookeepers.find(name); if (zookeeper == shared->auxiliary_zookeepers.end()) { + if (name.find(':') != std::string::npos || name.find('/') != std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid auxiliary ZooKeeper name {}: ':' and '/' are not allowed", name); + const auto & config = shared->auxiliary_zookeepers_config ? *shared->auxiliary_zookeepers_config : getConfigRef(); if (!config.has("auxiliary_zookeepers." + name)) throw Exception( diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c8ec9b91211..ff2676cd3e9 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -192,43 +192,54 @@ zkutil::ZooKeeperPtr StorageReplicatedMergeTree::getZooKeeper() const return res; } -static std::string normalizeZooKeeperPath(std::string zookeeper_path) +static std::string normalizeZooKeeperPath(std::string zookeeper_path, bool check_starts_with_slash, Poco::Logger * log = nullptr) { if (!zookeeper_path.empty() && zookeeper_path.back() == '/') zookeeper_path.resize(zookeeper_path.size() - 1); /// If zookeeper chroot prefix is used, path should start with '/', because chroot concatenates without it. if (!zookeeper_path.empty() && zookeeper_path.front() != '/') + { + /// Do not allow this for new tables, print warning for tables created in old versions + if (check_starts_with_slash) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "ZooKeeper path must starts with '/', got '{}'", zookeeper_path); + if (log) + LOG_WARNING(log, "ZooKeeper path ('{}') does not start with '/'. It will not be supported in future releases"); zookeeper_path = "/" + zookeeper_path; + } return zookeeper_path; } static String extractZooKeeperName(const String & path) { + static constexpr auto default_zookeeper_name = "default"; if (path.empty()) - throw Exception("ZooKeeper path should not be empty", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - auto pos = path.find(':'); - if (pos != String::npos) + throw Exception("ZooKeeper path should not be empty", ErrorCodes::BAD_ARGUMENTS); + if (path[0] == '/') + return default_zookeeper_name; + auto pos = path.find(":/"); + if (pos != String::npos && pos < path.find('/')) { auto zookeeper_name = path.substr(0, pos); if (zookeeper_name.empty()) - throw Exception("Zookeeper path should start with '/' or ':/'", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception("Zookeeper path should start with '/' or ':/'", ErrorCodes::BAD_ARGUMENTS); return zookeeper_name; } - static constexpr auto default_zookeeper_name = "default"; return default_zookeeper_name; } -static String extractZooKeeperPath(const String & path) +static String extractZooKeeperPath(const String & path, bool check_starts_with_slash, Poco::Logger * log = nullptr) { if (path.empty()) - throw Exception("ZooKeeper path should not be empty", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - auto pos = path.find(':'); - if (pos != String::npos) + throw Exception("ZooKeeper path should not be empty", ErrorCodes::BAD_ARGUMENTS); + if (path[0] == '/') + return path; + auto pos = path.find(":/"); + if (pos != String::npos && pos < path.find('/')) { - return normalizeZooKeeperPath(path.substr(pos + 1, String::npos)); + return normalizeZooKeeperPath(path.substr(pos + 1, String::npos), check_starts_with_slash, log); } - return normalizeZooKeeperPath(path); + return normalizeZooKeeperPath(path, check_starts_with_slash, log); } static MergeTreePartInfo makeDummyDropRangeForMovePartitionOrAttachPartitionFrom(const String & partition_id) @@ -275,7 +286,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( attach, [this] (const std::string & name) { enqueuePartForCheck(name); }) , zookeeper_name(extractZooKeeperName(zookeeper_path_)) - , zookeeper_path(extractZooKeeperPath(zookeeper_path_)) + , zookeeper_path(extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ !attach, log)) , replica_name(replica_name_) , replica_path(fs::path(zookeeper_path) / "replicas" / replica_name_) , reader(*this) @@ -5425,7 +5436,7 @@ void StorageReplicatedMergeTree::fetchPartition( info.table_id.uuid = UUIDHelpers::Nil; auto expand_from = query_context->getMacros()->expand(from_, info); String auxiliary_zookeeper_name = extractZooKeeperName(expand_from); - String from = extractZooKeeperPath(expand_from); + String from = extractZooKeeperPath(expand_from, /* check_starts_with_slash */ true); if (from.empty()) throw Exception("ZooKeeper path should not be empty", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -6490,7 +6501,7 @@ void StorageReplicatedMergeTree::movePartitionToShard( if (!move_part) throw Exception("MOVE PARTITION TO SHARD is not supported, use MOVE PART instead", ErrorCodes::NOT_IMPLEMENTED); - if (normalizeZooKeeperPath(zookeeper_path) == normalizeZooKeeperPath(to)) + if (normalizeZooKeeperPath(zookeeper_path, /* check_starts_with_slash */ true) == normalizeZooKeeperPath(to, /* check_starts_with_slash */ true)) throw Exception("Source and destination are the same", ErrorCodes::BAD_ARGUMENTS); auto zookeeper = getZooKeeper(); diff --git a/tests/integration/test_replicated_merge_tree_with_auxiliary_zookeepers/test.py b/tests/integration/test_replicated_merge_tree_with_auxiliary_zookeepers/test.py index a9dcce1b9d4..4644790ff94 100644 --- a/tests/integration/test_replicated_merge_tree_with_auxiliary_zookeepers/test.py +++ b/tests/integration/test_replicated_merge_tree_with_auxiliary_zookeepers/test.py @@ -101,3 +101,13 @@ def test_drop_replicated_merge_tree_with_auxiliary_zookeeper(started_cluster): assert zk.exists('/clickhouse/tables/test/test_auxiliary_zookeeper') drop_table([node1, node2], "test_auxiliary_zookeeper") assert zk.exists('/clickhouse/tables/test/test_auxiliary_zookeeper') is None + +def test_path_ambiguity(started_cluster): + drop_table([node1, node2], "test_path_ambiguity1") + drop_table([node1, node2], "test_path_ambiguity2") + node1.query("create table test_path_ambiguity1 (n int) engine=ReplicatedMergeTree('/test:bad:/path', '1') order by n") + assert "Invalid auxiliary ZooKeeper name" in node1.query_and_get_error("create table test_path_ambiguity2 (n int) engine=ReplicatedMergeTree('test:bad:/path', '1') order by n") + assert "ZooKeeper path must starts with '/'" in node1.query_and_get_error("create table test_path_ambiguity2 (n int) engine=ReplicatedMergeTree('test/bad:/path', '1') order by n") + node1.query("create table test_path_ambiguity2 (n int) engine=ReplicatedMergeTree('zookeeper2:/bad:/path', '1') order by n") + drop_table([node1, node2], "test_path_ambiguity1") + drop_table([node1, node2], "test_path_ambiguity2") From 173e5ac3ce19cc7a8fb19f143b2988e7afbc2bb4 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 28 Oct 2021 19:29:00 +0300 Subject: [PATCH 200/396] Update docs/ru/operations/system-tables/information_schema.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- .../operations/system-tables/information_schema.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/ru/operations/system-tables/information_schema.md b/docs/ru/operations/system-tables/information_schema.md index 1e8e69a002e..e5fc110c55c 100644 --- a/docs/ru/operations/system-tables/information_schema.md +++ b/docs/ru/operations/system-tables/information_schema.md @@ -38,12 +38,12 @@ SHOW TABLES FROM INFORMATION_SCHEMA; - `column_default` ([String](../../sql-reference/data-types/string.md)) — выражение для значения по умолчанию или пустая строка. - `is_nullable` ([UInt8](../../sql-reference/data-types/int-uint.md)) — флаг, показывающий является ли столбец типа `Nullable`. - `data_type` ([String](../../sql-reference/data-types/string.md)) — тип столбца. -- `character_maximum_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Максимальная длина в байтах для двоичных данных, символьных данных или текстовых данных и изображений. Иначе возвращается значение `NULL`. -- `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Максимальная длина в байтах для двоичных данных, символьных данных или текстовых данных и изображений. Иначе возвращается значение `NULL`. -- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Точность приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. Иначе возвращается значение `NULL`. -- `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Основание системы счисления точности приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. Иначе возвращается значение `NULL`. -- `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Масштаб приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. Иначе возвращается значение `NULL`. -- `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Код подтипа для типов данных интервала `DateTime` и `ISO`. Для других типов данных возвращается значение `NULL`. +- `character_maximum_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — максимальная длина в байтах для двоичных данных, символьных данных или текстовых данных и изображений. Иначе возвращается значение `NULL`. +- `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — максимальная длина в байтах для двоичных данных, символьных данных или текстовых данных и изображений. Иначе возвращается значение `NULL`. +- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — точность приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. Иначе возвращается значение `NULL`. +- `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — основание системы счисления точности приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. Иначе возвращается значение `NULL`. +- `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — масштаб приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. Иначе возвращается значение `NULL`. +- `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — код подтипа для типов данных интервала `DateTime` и `ISO`. Для других типов данных возвращается значение `NULL`. - `character_set_catalog` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. - `character_set_schema` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. - `character_set_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. From 05bd672415adc7898b411801d454a92d84fa8c0c Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 28 Oct 2021 19:29:12 +0300 Subject: [PATCH 201/396] Update docs/ru/operations/system-tables/information_schema.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/operations/system-tables/information_schema.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/system-tables/information_schema.md b/docs/ru/operations/system-tables/information_schema.md index e5fc110c55c..1cc970757b3 100644 --- a/docs/ru/operations/system-tables/information_schema.md +++ b/docs/ru/operations/system-tables/information_schema.md @@ -2,7 +2,7 @@ `INFORMATION_SCHEMA` (`information_schema`) — это системная база данных, содержащая представления. Используя эти представления, вы можете получить информацию о метаданных объектов базы данных. Эти представления считывают данные из столбцов системных таблиц [system.columns](../../operations/system-tables/columns.md), [system.databases](../../operations/system-tables/databases.md) и [system.tables](../../operations/system-tables/tables.md). -Структура и состав системных таблиц могут меняться в разных версиях продукта, но поддержка `information_schema` позволяет изменять структуру системных таблиц без изменения способа доступа к метаданным. Запросы метаданных не зависят от используемой СУБД. +Структура и состав системных таблиц могут меняться в разных версиях СУБД ClickHouse, но поддержка `information_schema` позволяет изменять структуру системных таблиц без изменения способа доступа к метаданным. Запросы метаданных не зависят от используемой СУБД. ``` sql SHOW TABLES FROM INFORMATION_SCHEMA; From bb4a8e91f7e774dfa0a10fd4697d2eb2dd426750 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 28 Oct 2021 20:25:31 +0300 Subject: [PATCH 202/396] Create 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 ff2676cd3e9..cd7c9352c6e 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -233,7 +233,7 @@ static String extractZooKeeperPath(const String & path, bool check_starts_with_s if (path.empty()) throw Exception("ZooKeeper path should not be empty", ErrorCodes::BAD_ARGUMENTS); if (path[0] == '/') - return path; + return normalizeZooKeeperPath(path, check_starts_with_slash, log); auto pos = path.find(":/"); if (pos != String::npos && pos < path.find('/')) { From 3515ab99b6e6b7320747d85ef3385fbbbe757698 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Thu, 28 Oct 2021 20:27:29 +0300 Subject: [PATCH 203/396] Translate to Russian MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Выполнил перевод на русский язык. --- .../statements/create/database.md | 4 +- .../statements/create/database.md | 50 ++++++++++++++----- 2 files changed, 39 insertions(+), 15 deletions(-) diff --git a/docs/en/sql-reference/statements/create/database.md b/docs/en/sql-reference/statements/create/database.md index 9d1a4c614e7..5a6d360f1ee 100644 --- a/docs/en/sql-reference/statements/create/database.md +++ b/docs/en/sql-reference/statements/create/database.md @@ -8,7 +8,7 @@ toc_title: DATABASE Creates a new database. ``` sql -CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] [ENGINE = engine(...)] +CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] [ENGINE = engine(...)] [COMMENT 'Comment'] ``` ## Clauses {#clauses} @@ -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’s also a [lazy](../../../engines/database-engines/lazy.md) 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. ### COMMENT {#comment} diff --git a/docs/ru/sql-reference/statements/create/database.md b/docs/ru/sql-reference/statements/create/database.md index 7d19f3e8f17..6ba3db32419 100644 --- a/docs/ru/sql-reference/statements/create/database.md +++ b/docs/ru/sql-reference/statements/create/database.md @@ -8,27 +8,51 @@ toc_title: "База данных" Создает базу данных. ``` sql -CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] [ENGINE = engine(...)] +CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] [ENGINE = engine(...)] [COMMENT 'Comment'] ``` -### Секции {#sektsii} +## Секции {#clauses} -- `IF NOT EXISTS` +### IF NOT EXISTS {#if-not-exists} - Если база данных с именем `db_name` уже существует, то ClickHouse не создаёт базу данных и: - - Не генерирует исключение, если секция указана. - - Генерирует исключение, если секция не указана. +Если база данных с именем `db_name` уже существует, то ClickHouse не создает базу данных и: -- `ON CLUSTER` +- Не генерирует исключение, если секция указана. +- Генерирует исключение, если секция не указана. - ClickHouse создаёт базу данных `db_name` на всех серверах указанного кластера. +### ON CLUSTER {#on-cluster} -- `ENGINE` +ClickHouse создаёт базу данных с именем `db_name` на всех серверах указанного кластера. Более подробную информацию смотрите в разделе [Распределенные DDL запросы](../../../sql-reference/distributed-ddl.md). - - MySQL +### ENGINE {#engine} - Позволяет получать данные с удаленного сервера MySQL. +[MySQL](../../../engines/database-engines/mysql.md) позволяет получать данные с удаленного сервера MySQL. По умолчанию ClickHouse использует собственный [движок баз данных](../../../engines/database-engines/index.md). Есть также движок баз данных [lazy](../../../engines/database-engines/lazy.md). - По умолчанию ClickHouse использует собственный движок баз данных. +### COMMENT {#comment} - +Вы можете добавить комментарий к базе данных при ее создании. + +Комментарий поддерживается для всех движков баз данных. + +**Синтаксис** + +``` sql +CREATE DATABASE db_name ENGINE = engine(...) COMMENT 'Comment' +``` + +**Пример** + +Запрос: + +``` sql +CREATE DATABASE db_comment ENGINE = Memory COMMENT 'The temporary database'; +SELECT name, comment FROM system.databases WHERE name = 'db_comment'; +``` + +Результат: + +```text +┌─name───────┬─comment────────────────┐ +│ db_comment │ The temporary database │ +└────────────┴────────────────────────┘ +``` From 4d5cf1a3251adab3f725b958859211aa662f5142 Mon Sep 17 00:00:00 2001 From: Alexey Date: Thu, 28 Oct 2021 19:53:26 +0000 Subject: [PATCH 204/396] forgotten merge --- docs/en/operations/settings/settings.md | 1 + docs/ru/operations/settings/settings.md | 4 +--- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 02587533dcb..9740b522545 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3845,6 +3845,7 @@ Default value: `0`. **Example** See an example for the [DESCRIBE](../../sql-reference/statements/describe-table.md) statement. + ## async_insert {#async-insert} Enables or disables asynchronous inserts. This makes sense only for insertion over HTTP protocol. Note that deduplication isn't working for such inserts. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 41a69dc6d0a..21c62381a3a 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3641,7 +3641,6 @@ SELECT * FROM positional_arguments ORDER BY 2,3; - настройка [optimize_move_to_prewhere](#optimize_move_to_prewhere) -<<<<<<< HEAD ## describe_include_subcolumns {#describe_include_subcolumns} Включает или отключает описание подстолбцов при выполнении запроса [DESCRIBE](../../sql-reference/statements/describe-table.md). Настройка действует, например, на элементы [Tuple](../../sql-reference/data-types/tuple.md) или подстолбцы типов [Map](../../sql-reference/data-types/map.md#map-subcolumns), [Nullable](../../sql-reference/data-types/nullable.md#finding-null) или [Array](../../sql-reference/data-types/array.md#array-size). @@ -3656,7 +3655,7 @@ SELECT * FROM positional_arguments ORDER BY 2,3; **Пример** Смотрите пример запроса [DESCRIBE](../../sql-reference/statements/describe-table.md). -======= + ## async_insert {#async-insert} Включает или отключает асинхронные вставки. Работает только для вставок по протоколу HTTP. Обратите внимание, что при таких вставках дедупликация не производится. @@ -3740,4 +3739,3 @@ SELECT * FROM positional_arguments ORDER BY 2,3; Значение по умолчанию: `0`. ->>>>>>> 8b59ca5905c40df52adf3b03302f8cc3177403d9 From 734bb5b0263af1af44ea2ba3746f423238cb5170 Mon Sep 17 00:00:00 2001 From: Nickita Taranov Date: Fri, 22 Oct 2021 23:19:32 +0300 Subject: [PATCH 205/396] support any serializable column --- src/Functions/FunctionsStringArray.cpp | 22 +------ src/Functions/FunctionsStringArray.h | 57 ++++++++++++------- .../00255_array_concat_string.reference | 14 ++++- .../0_stateless/00255_array_concat_string.sql | 14 ++++- 4 files changed, 62 insertions(+), 45 deletions(-) diff --git a/src/Functions/FunctionsStringArray.cpp b/src/Functions/FunctionsStringArray.cpp index 0c76cde701a..0e73d6a33f5 100644 --- a/src/Functions/FunctionsStringArray.cpp +++ b/src/Functions/FunctionsStringArray.cpp @@ -1,21 +1,6 @@ #include #include -namespace -{ -bool isNullableStringOrNullableNothing(DB::DataTypePtr type) -{ - if (type->isNullable()) - { - const auto & nested_type = assert_cast(*type).getNestedType(); - if (isString(nested_type) || isNothing(nested_type)) - return true; - } - return false; -} - -} - namespace DB { namespace ErrorCodes @@ -33,11 +18,8 @@ DataTypePtr FunctionArrayStringConcat::getReturnTypeImpl(const DataTypes & argum ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); const DataTypeArray * array_type = checkAndGetDataType(arguments[0].get()); - // An array consisting of only Null-s has type Array(Nullable(Nothing)) - if (!array_type || !(isString(array_type->getNestedType()) || isNullableStringOrNullableNothing(array_type->getNestedType()))) - throw Exception( - "First argument for function " + getName() + " must be an array of String-s or Nullable(String)-s.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (!array_type) + throw Exception("First argument for function " + getName() + " must be an array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (arguments.size() == 2 && !isString(arguments[1])) throw Exception("Second argument for function " + getName() + " must be constant string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/src/Functions/FunctionsStringArray.h b/src/Functions/FunctionsStringArray.h index c15a8db8186..27907626971 100644 --- a/src/Functions/FunctionsStringArray.h +++ b/src/Functions/FunctionsStringArray.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -17,7 +18,6 @@ #include #include - namespace DB { @@ -648,7 +648,7 @@ public: }; -/// Joins an array of strings into one string via a separator. +/// Joins an array of type serializable to string into one string via a separator. class FunctionArrayStringConcat : public IFunction { private: @@ -734,6 +734,25 @@ private: null_map); } + static ColumnPtr serializeNestedColumn(const ColumnArray & col_arr, const DataTypePtr & nested_type) + { + if (isString(nested_type)) + { + return col_arr.getDataPtr(); + } + else if (const ColumnNullable * col_nullable = checkAndGetColumn(col_arr.getData()); + col_nullable && isString(col_nullable->getNestedColumn().getDataType())) + { + return col_nullable->getNestedColumnPtr(); + } + else + { + ColumnsWithTypeAndName cols; + cols.emplace_back(col_arr.getDataPtr(), nested_type, "tmp"); + return ConvertImplGenericToString::execute(cols, std::make_shared()); + } + } + public: static constexpr auto name = "arrayStringConcat"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } @@ -761,7 +780,9 @@ public: delimiter = col_delim->getValue(); } - if (const ColumnConst * col_const_arr = checkAndGetColumnConst(arguments[0].column.get())) + const auto & nested_type = assert_cast(*arguments[0].type).getNestedType(); + if (const ColumnConst * col_const_arr = checkAndGetColumnConst(arguments[0].column.get()); + col_const_arr && isString(nested_type)) { Array src_arr = col_const_arr->getValue(); String dst_str; @@ -778,25 +799,19 @@ public: return result_type->createColumnConst(col_const_arr->size(), dst_str); } + + ColumnPtr src_column = arguments[0].column->convertToFullColumnIfConst(); + const ColumnArray & col_arr = assert_cast(*src_column.get()); + + ColumnPtr str_subcolumn = serializeNestedColumn(col_arr, nested_type); + const ColumnString & col_string = assert_cast(*str_subcolumn.get()); + + auto col_res = ColumnString::create(); + if (const ColumnNullable * col_nullable = checkAndGetColumn(col_arr.getData())) + executeInternal(col_string, col_arr, delimiter, *col_res, col_nullable->getNullMapData().data()); else - { - const ColumnArray & col_arr = assert_cast(*arguments[0].column); - auto col_res = ColumnString::create(); - if (WhichDataType(col_arr.getData().getDataType()).isString()) - { - const ColumnString & col_string = assert_cast(col_arr.getData()); - executeInternal(col_string, col_arr, delimiter, *col_res); - } - else - { - const ColumnNullable & col_nullable = assert_cast(col_arr.getData()); - if (const ColumnString * col_string = typeid_cast(col_nullable.getNestedColumnPtr().get())) - executeInternal(*col_string, col_arr, delimiter, *col_res, col_nullable.getNullMapData().data()); - else - col_res->insertManyDefaults(col_arr.size()); - } - return col_res; - } + executeInternal(col_string, col_arr, delimiter, *col_res); + return col_res; } }; diff --git a/tests/queries/0_stateless/00255_array_concat_string.reference b/tests/queries/0_stateless/00255_array_concat_string.reference index e9fafe93ed1..4ffac8e5de0 100644 --- a/tests/queries/0_stateless/00255_array_concat_string.reference +++ b/tests/queries/0_stateless/00255_array_concat_string.reference @@ -65,7 +65,17 @@ yandex google test 123 hello world goodbye xyz yandex google test 123 hello wo 0 hello;world;xyz;def - +1;23;456 +1;23;456 +127.0.0.1; 1.0.0.1 +127.0.0.1; 1.0.0.1 +2021-10-01; 2021-10-02 +2021-10-01; 2021-10-02 hello;world;xyz;def - +1;23;456 +1;23;456 +127.0.0.1; 1.0.0.1 +127.0.0.1; 1.0.0.1 +2021-10-01; 2021-10-02 +2021-10-01; 2021-10-02 diff --git a/tests/queries/0_stateless/00255_array_concat_string.sql b/tests/queries/0_stateless/00255_array_concat_string.sql index 3bdae0821cd..f4f95956a16 100644 --- a/tests/queries/0_stateless/00255_array_concat_string.sql +++ b/tests/queries/0_stateless/00255_array_concat_string.sql @@ -9,8 +9,18 @@ SELECT arrayStringConcat(arrayMap(x -> toString(x), range(number)), ',') FROM sy SELECT arrayStringConcat(arrayMap(x -> transform(x, [0, 1, 2, 3, 4, 5, 6, 7, 8], ['yandex', 'google', 'test', '123', '', 'hello', 'world', 'goodbye', 'xyz'], ''), arrayMap(x -> x % 9, range(number))), ' ') FROM system.numbers LIMIT 20; SELECT arrayStringConcat(arrayMap(x -> toString(x), range(number % 4))) FROM system.numbers LIMIT 10; SELECT arrayStringConcat([Null, 'hello', Null, 'world', Null, 'xyz', 'def', Null], ';'); -SELECT arrayStringConcat([Null, Null], ';'); SELECT arrayStringConcat([Null::Nullable(String), Null::Nullable(String)], ';'); +SELECT arrayStringConcat(arr, ';') FROM (SELECT [1, 23, 456] AS arr); +SELECT arrayStringConcat(arr, ';') FROM (SELECT [Null, 1, Null, 23, Null, 456, Null] AS arr); +SELECT arrayStringConcat(arr, '; ') FROM (SELECT [toIPv4('127.0.0.1'), toIPv4('1.0.0.1')] AS arr); +SELECT arrayStringConcat(arr, '; ') FROM (SELECT [toIPv4('127.0.0.1'), Null, toIPv4('1.0.0.1')] AS arr); +SELECT arrayStringConcat(arr, '; ') FROM (SELECT [toDate('2021-10-01'), toDate('2021-10-02')] AS arr); +SELECT arrayStringConcat(arr, '; ') FROM (SELECT [toDate('2021-10-01'), Null, toDate('2021-10-02')] AS arr); SELECT arrayStringConcat(materialize([Null, 'hello', Null, 'world', Null, 'xyz', 'def', Null]), ';'); -SELECT arrayStringConcat(materialize([Null, Null]), ';'); SELECT arrayStringConcat(materialize([Null::Nullable(String), Null::Nullable(String)]), ';'); +SELECT arrayStringConcat(arr, ';') FROM (SELECT materialize([1, 23, 456]) AS arr); +SELECT arrayStringConcat(arr, ';') FROM (SELECT materialize([Null, 1, Null, 23, Null, 456, Null]) AS arr); +SELECT arrayStringConcat(arr, '; ') FROM (SELECT materialize([toIPv4('127.0.0.1'), toIPv4('1.0.0.1')]) AS arr); +SELECT arrayStringConcat(arr, '; ') FROM (SELECT materialize([toIPv4('127.0.0.1'), Null, toIPv4('1.0.0.1')]) AS arr); +SELECT arrayStringConcat(arr, '; ') FROM (SELECT materialize([toDate('2021-10-01'), toDate('2021-10-02')]) AS arr); +SELECT arrayStringConcat(arr, '; ') FROM (SELECT materialize([toDate('2021-10-01'), Null, toDate('2021-10-02')]) AS arr); From 982d38e60c2a3890b6564c0ccecd1948d1920492 Mon Sep 17 00:00:00 2001 From: Nickita Taranov Date: Fri, 29 Oct 2021 00:40:42 +0300 Subject: [PATCH 206/396] update docs --- .../en/sql-reference/functions/splitting-merging-functions.md | 2 +- .../ru/sql-reference/functions/splitting-merging-functions.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/splitting-merging-functions.md b/docs/en/sql-reference/functions/splitting-merging-functions.md index 179d7934bb1..67d9ac89dd3 100644 --- a/docs/en/sql-reference/functions/splitting-merging-functions.md +++ b/docs/en/sql-reference/functions/splitting-merging-functions.md @@ -213,7 +213,7 @@ SELECT splitByNonAlpha(' 1! a, b. '); ## arrayStringConcat(arr\[, separator\]) {#arraystringconcatarr-separator} -Concatenates the strings (values of type String or Nullable(String)) listed in the array with the separator. ’separator’ is an optional parameter: a constant string, set to an empty string by default. +Concatenates string representations of values listed in the array with the separator. `separator` is an optional parameter: a constant string, set to an empty string by default. Returns the string. ## alphaTokens(s) {#alphatokenss} diff --git a/docs/ru/sql-reference/functions/splitting-merging-functions.md b/docs/ru/sql-reference/functions/splitting-merging-functions.md index efe74dba043..96e6abae555 100644 --- a/docs/ru/sql-reference/functions/splitting-merging-functions.md +++ b/docs/ru/sql-reference/functions/splitting-merging-functions.md @@ -212,8 +212,8 @@ SELECT splitByNonAlpha(' 1! a, b. '); ## arrayStringConcat(arr\[, separator\]) {#arraystringconcatarr-separator} -Склеивает строки, перечисленные в массиве, с разделителем separator. -separator - необязательный параметр, константная строка, по умолчанию равен пустой строке. +Склеивает строковые представления элементов массива с разделителем `separator`. +`separator` - необязательный параметр, константная строка, по умолчанию равен пустой строке. Возвращается строка. ## alphaTokens(s) {#alphatokenss} From 03d7476864270fa83ad04a7aa164085c3b6f93a3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Oct 2021 16:56:45 +0300 Subject: [PATCH 207/396] One more fix --- src/Storages/StorageFile.cpp | 53 ++++++++++--------------- src/Storages/StorageFile.h | 2 + src/Storages/StorageURL.cpp | 4 +- src/TableFunctions/TableFunctionURL.cpp | 3 +- 4 files changed, 28 insertions(+), 34 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 6eec02e9414..4a1eac2a39e 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -62,6 +62,7 @@ namespace ErrorCodes extern const int TIMEOUT_EXCEEDED; extern const int INCOMPATIBLE_COLUMNS; extern const int CANNOT_STAT; + extern const int LOGICAL_ERROR; } namespace @@ -133,6 +134,7 @@ void checkCreationIsAllowed(ContextPtr context_global, const std::string & db_di if (fs::exists(table_path) && fs::is_directory(table_path)) throw Exception("File must not be a directory", ErrorCodes::INCORRECT_FILE_NAME); } + } Strings StorageFile::getPathsList(const String & table_path, const String & user_files_path, ContextPtr context, size_t & total_bytes_to_read) @@ -191,11 +193,8 @@ StorageFile::StorageFile(const std::string & table_path_, const std::string & us : StorageFile(args) { is_db_table = false; - bool has_wildcards = table_path_.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; - if (has_wildcards) - paths = {table_path_}; - else - paths = getPathsList(table_path_, user_files_path, args.getContext(), total_bytes_to_read); + paths = getPathsList(table_path_, user_files_path, args.getContext(), total_bytes_to_read); + path_for_partitioned_write = table_path_; if (args.format_name == "Distributed") { @@ -686,23 +685,18 @@ public: const StorageMetadataPtr & metadata_snapshot_, const String & table_name_for_log_, std::unique_lock && lock_, - int table_fd_, - bool use_table_fd_, - std::string base_path_, - std::vector paths_, + String base_path_, + String path_, const CompressionMethod compression_method_, const std::optional & format_settings_, const String format_name_, ContextPtr context_, int flags_) : PartitionedSink(partition_by, context_, metadata_snapshot_->getSampleBlock()) - , path(paths_[0]) + , path(path_) , metadata_snapshot(metadata_snapshot_) , table_name_for_log(table_name_for_log_) - , table_fd(table_fd_) - , use_table_fd(use_table_fd_) , base_path(base_path_) - , paths(paths_) , compression_method(compression_method_) , format_name(format_name_) , format_settings(format_settings_) @@ -717,11 +711,12 @@ public: auto partition_path = PartitionedSink::replaceWildcards(path, partition_id); PartitionedSink::validatePartitionKey(partition_path, true); Strings result_paths = {partition_path}; + checkCreationIsAllowed(context, context->getUserFilesPath(), partition_path); return std::make_shared( metadata_snapshot, table_name_for_log, - table_fd, - use_table_fd, + -1, + /* use_table_fd */false, base_path, result_paths, compression_method, @@ -736,10 +731,7 @@ private: StorageMetadataPtr metadata_snapshot; String table_name_for_log; - int table_fd; - bool use_table_fd; std::string base_path; - std::vector paths; CompressionMethod compression_method; std::string format_name; std::optional format_settings; @@ -764,32 +756,23 @@ SinkToStoragePtr StorageFile::write( if (context->getSettingsRef().engine_file_truncate_on_insert) flags |= O_TRUNC; - if (!paths.empty()) - { - path = paths[0]; - fs::create_directories(fs::path(path).parent_path()); - } - - bool has_wildcards = path.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; + bool has_wildcards = path_for_partitioned_write.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; const auto * insert_query = dynamic_cast(query.get()); bool is_partitioned_implementation = insert_query && insert_query->partition_by && has_wildcards; if (is_partitioned_implementation) { - if (paths.size() != 1) - throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, - "Table '{}' is in readonly mode because of globs in filepath", - getStorageID().getNameForLogs()); + if (path_for_partitioned_write.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty path for partitioned write"); + fs::create_directories(fs::path(path_for_partitioned_write).parent_path()); return std::make_shared( insert_query->partition_by, metadata_snapshot, getStorageID().getNameForLogs(), std::unique_lock{rwlock, getLockTimeout(context)}, - table_fd, - use_table_fd, base_path, - paths, + path_for_partitioned_write, chooseCompressionMethod(path, compression_method), format_settings, format_name, @@ -798,6 +781,12 @@ SinkToStoragePtr StorageFile::write( } else { + if (!paths.empty()) + { + path = paths[0]; + fs::create_directories(fs::path(path).parent_path()); + } + return std::make_shared( metadata_snapshot, getStorageID().getNameForLogs(), diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 9a58bb2a251..f48d1c285da 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -108,6 +108,8 @@ private: /// Total number of bytes to read (sums for multiple files in case of globs). Needed for progress bar. size_t total_bytes_to_read = 0; + + String path_for_partitioned_write; }; } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 41a332ed999..ab1e9713701 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -324,7 +324,9 @@ Pipe IStorageURLBase::read( unsigned /*num_streams*/) { auto params = getReadURIParams(column_names, metadata_snapshot, query_info, local_context, processed_stage, max_block_size); - bool with_globs = uri.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; + bool with_globs = (uri.find('{') != std::string::npos && uri.find('}') != std::string::npos) + || uri.find('|') == std::string::npos; + if (with_globs) { size_t max_addresses = local_context->getSettingsRef().glob_expansion_max_elements; diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index 904bd5dcb90..11c2d99de50 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -82,7 +82,8 @@ StoragePtr TableFunctionURL::getStorage( global_context, compression_method_, headers, - configuration.http_method);} + configuration.http_method); +} void registerTableFunctionURL(TableFunctionFactory & factory) { From a4b653fdca809502243088494ad58431b127bec8 Mon Sep 17 00:00:00 2001 From: Teja Srivastasa Date: Fri, 29 Oct 2021 02:00:57 +0200 Subject: [PATCH 208/396] Update installation success message #30672 --- programs/install/Install.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 606af7ecd0d..57e56c8b9ea 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -809,13 +809,23 @@ int mainEntryClickHouseInstall(int argc, char ** argv) if (has_password_for_default_user) maybe_password = " --password"; - fmt::print( + if (fs::exists(pid_file)) { + fmt::print( + "\nClickHouse has been successfully installed.\n" + "\nRestart clickhouse-server with:\n" + " sudo clickhouse restart\n" + "\nStart clickhouse-client with:\n" + " clickhouse-client{}\n\n", + maybe_password); + } else { + fmt::print( "\nClickHouse has been successfully installed.\n" "\nStart clickhouse-server with:\n" " sudo clickhouse start\n" "\nStart clickhouse-client with:\n" " clickhouse-client{}\n\n", maybe_password); + } } catch (const fs::filesystem_error &) { From 13a305d6489ea29ba6c7c7b5c3629fdf37ec6a5d Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 29 Oct 2021 02:39:27 +0000 Subject: [PATCH 209/396] filelog engine tests improve fix --- cmake/find/filelog.cmake | 7 ---- src/Storages/FileLog/DirectoryWatcherBase.cpp | 3 +- .../02022_storage_filelog_one_file.sh | 10 ++--- .../0_stateless/02023_storage_filelog.sh | 38 +++++++++---------- .../0_stateless/02024_storage_filelog_mv.sh | 20 +++++----- .../02025_storage_filelog_virtual_col.sh | 26 ++++++------- .../02026_storage_filelog_largefile.sh | 16 ++++---- 7 files changed, 57 insertions(+), 63 deletions(-) diff --git a/cmake/find/filelog.cmake b/cmake/find/filelog.cmake index 4d2f94f3f20..f545ee9d0ed 100644 --- a/cmake/find/filelog.cmake +++ b/cmake/find/filelog.cmake @@ -1,10 +1,3 @@ -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 (OS_LINUX) set (USE_FILELOG 1) diff --git a/src/Storages/FileLog/DirectoryWatcherBase.cpp b/src/Storages/FileLog/DirectoryWatcherBase.cpp index 37a82ad4ee3..f2737219fd8 100644 --- a/src/Storages/FileLog/DirectoryWatcherBase.cpp +++ b/src/Storages/FileLog/DirectoryWatcherBase.cpp @@ -35,7 +35,7 @@ DirectoryWatcherBase::DirectoryWatcherBase( fd = inotify_init(); if (fd == -1) - throw Exception("Cannot initialize inotify", ErrorCodes::IO_SETUP_ERROR); + throwFromErrno("Cannot initialize inotify", ErrorCodes::IO_SETUP_ERROR); watch_task = getContext()->getSchedulePool().createTask("directory_watch", [this] { watchFunc(); }); start(); @@ -59,6 +59,7 @@ void DirectoryWatcherBase::watchFunc() if (wd == -1) { owner.onError(Exception(ErrorCodes::IO_SETUP_ERROR, "Watch directory {} failed", path)); + throwFromErrnoWithPath("Watch directory {} failed", path, ErrorCodes::IO_SETUP_ERROR); } std::string buffer; 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 600e537a352..bed97c7c0be 100755 --- a/tests/queries/0_stateless/02022_storage_filelog_one_file.sh +++ b/tests/queries/0_stateless/02022_storage_filelog_one_file.sh @@ -14,23 +14,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}/a.txt + echo $i, $i >> ${user_files_path}/02022_storage_filelog_one_file.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}/a.txt', 'CSV');" +${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 "select * from file_log order by k;" for i in {100..120} do - echo $i, $i >> ${user_files_path}/a.txt + echo $i, $i >> ${user_files_path}/02022_storage_filelog_one_file.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 +touch ${user_files_path}/02022_storage_filelog_one_file.txt ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" ${CLICKHOUSE_CLIENT} --query "detach table file_log;" @@ -39,4 +39,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}/a.txt +rm -rf ${user_files_path}/02022_storage_filelog_one_file.txt diff --git a/tests/queries/0_stateless/02023_storage_filelog.sh b/tests/queries/0_stateless/02023_storage_filelog.sh index 8279e09ff65..fd242aad359 100755 --- a/tests/queries/0_stateless/02023_storage_filelog.sh +++ b/tests/queries/0_stateless/02023_storage_filelog.sh @@ -12,52 +12,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}/logs/ +mkdir -p ${user_files_path}/02023_storage_filelog/ -rm -rf ${user_files_path}/logs/* +rm -rf ${user_files_path}/02023_storage_filelog/* for i in {1..20} do - echo $i, $i >> ${user_files_path}/logs/a.txt + echo $i, $i >> ${user_files_path}/02023_storage_filelog/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 "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/02023_storage_filelog/', 'CSV');" ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" -cp ${user_files_path}/logs/a.txt ${user_files_path}/logs/b.txt +cp ${user_files_path}/02023_storage_filelog/a.txt ${user_files_path}/02023_storage_filelog/b.txt ${CLICKHOUSE_CLIENT} --query "select * from file_log order by k;" for i in {100..120} do - echo $i, $i >> ${user_files_path}/logs/a.txt + echo $i, $i >> ${user_files_path}/02023_storage_filelog/a.txt done # touch does not change file content, no event -touch ${user_files_path}/logs/a.txt +touch ${user_files_path}/02023_storage_filelog/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 +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 -rm ${user_files_path}/logs/d.txt +rm ${user_files_path}/02023_storage_filelog/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 +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 for i in {150..200} do - echo $i, $i >> ${user_files_path}/logs/h.txt + echo $i, $i >> ${user_files_path}/02023_storage_filelog/h.txt done for i in {200..250} do - echo $i, $i >> ${user_files_path}/logs/i.txt + echo $i, $i >> ${user_files_path}/02023_storage_filelog/i.txt done ${CLICKHOUSE_CLIENT} --query "attach table file_log;" @@ -69,11 +69,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}/logs/a.txt --size 0 +truncate ${user_files_path}/02023_storage_filelog/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}/logs +rm -rf ${user_files_path}/02023_storage_filelog diff --git a/tests/queries/0_stateless/02024_storage_filelog_mv.sh b/tests/queries/0_stateless/02024_storage_filelog_mv.sh index 9ac0c95eae0..1e2f990c135 100755 --- a/tests/queries/0_stateless/02024_storage_filelog_mv.sh +++ b/tests/queries/0_stateless/02024_storage_filelog_mv.sh @@ -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}/logs/ -rm -rf ${user_files_path}/logs/* +mkdir -p ${user_files_path}/02024_storage_filelog_mv/ +rm -rf ${user_files_path}/02024_storage_filelog_mv/* for i in {1..20} do - echo $i, $i >> ${user_files_path}/logs/a.txt + echo $i, $i >> ${user_files_path}/02024_storage_filelog_mv/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 "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/02024_storage_filelog_mv/', '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}/logs/a.txt ${user_files_path}/logs/b.txt +cp ${user_files_path}/02024_storage_filelog_mv/a.txt ${user_files_path}/02024_storage_filelog_mv/b.txt # touch does not change file content, no event -touch ${user_files_path}/logs/a.txt +touch ${user_files_path}/02024_storage_filelog_mv/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}/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 for i in {100..120} do - echo $i, $i >> ${user_files_path}/logs/d.txt + echo $i, $i >> ${user_files_path}/02024_storage_filelog_mv/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}/logs +rm -rf ${user_files_path}/02024_storage_filelog_mv 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 a92f93991ef..4545a18e650 100755 --- a/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh +++ b/tests/queries/0_stateless/02025_storage_filelog_virtual_col.sh @@ -12,37 +12,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}/logs/ +mkdir -p ${user_files_path}/02025_storage_filelog_virtual_col/ -rm -rf ${user_files_path}/logs/* +rm -rf ${user_files_path}/02025_storage_filelog_virtual_col/* for i in {1..20} do - echo $i, $i >> ${user_files_path}/logs/a.txt + echo $i, $i >> ${user_files_path}/02025_storage_filelog_virtual_col/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 "create table file_log(k UInt8, v UInt8) engine=FileLog('${user_files_path}/02025_storage_filelog_virtual_col/', 'CSV');" ${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 +cp ${user_files_path}/02025_storage_filelog_virtual_col/a.txt ${user_files_path}/02025_storage_filelog_virtual_col/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}/logs/a.txt + echo $i, $i >> ${user_files_path}/02025_storage_filelog_virtual_col/a.txt done # touch does not change file content, no event -touch ${user_files_path}/logs/a.txt +touch ${user_files_path}/02025_storage_filelog_virtual_col/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 +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 -rm ${user_files_path}/logs/d.txt +rm ${user_files_path}/02025_storage_filelog_virtual_col/d.txt ${CLICKHOUSE_CLIENT} --query "select *, _filename, _offset from file_log order by _filename, _offset;" @@ -52,11 +52,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}/logs/a.txt --size 0 +truncate ${user_files_path}/02025_storage_filelog_virtual_col/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}/logs +rm -rf ${user_files_path}/02025_storage_filelog_virtual_col diff --git a/tests/queries/0_stateless/02026_storage_filelog_largefile.sh b/tests/queries/0_stateless/02026_storage_filelog_largefile.sh index a6c5d19287e..c480fcc05d3 100755 --- a/tests/queries/0_stateless/02026_storage_filelog_largefile.sh +++ b/tests/queries/0_stateless/02026_storage_filelog_largefile.sh @@ -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}/logs/ +mkdir -p ${user_files_path}/02026_storage_filelog_largefile/ -rm -rf ${user_files_path}/logs/* +rm -rf ${user_files_path}/02026_storage_filelog_largefile/* -chmod 777 ${user_files_path}/logs/ +chmod 777 ${user_files_path}/02026_storage_filelog_largefile/ 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);" + ${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);" 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 "create table file_log(k UInt32, v UInt32) engine=FileLog('${user_files_path}/02026_storage_filelog_largefile/', '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);" + ${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);" 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);" + ${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);" done ${CLICKHOUSE_CLIENT} --query "select count() from file_log " ${CLICKHOUSE_CLIENT} --query "drop table file_log;" -rm -rf ${user_files_path}/logs +rm -rf ${user_files_path}/02026_storage_filelog_largefile From 921bdbdd7d78ce5982e0d759d6b41f8e6c094c71 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Fri, 29 Oct 2021 14:51:34 +0800 Subject: [PATCH 210/396] Update src/Functions/map.cpp Co-authored-by: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> --- src/Functions/map.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 029c4f6f80e..03a9da404c2 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -443,9 +443,10 @@ public: auto value_type = map_type->getValueType(); const auto & nested_column = col_map->getNestedColumn(); + const auto & keys_column = col_map->getNestedData().getColumn(0); const auto & values_column = col_map->getNestedData().getColumn(1); - const ColumnString * keys_string_column = checkAndGetColumn(col_map->getNestedData().getColumn(0)); - const ColumnFixedString * keys_fixed_string_column = checkAndGetColumn(col_map->getNestedData().getColumn(0)); + const ColumnString * keys_string_column = checkAndGetColumn(keys_column); + const ColumnFixedString * keys_fixed_string_column = checkAndGetColumn(keys_column); FunctionLike func_like; From 1cc1417ba4ef146b584a67b086dc306842c8f838 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 29 Oct 2021 09:53:48 +0300 Subject: [PATCH 211/396] Update StorageFileLog.cpp --- src/Storages/FileLog/StorageFileLog.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index 1dc25485a47..1ba16fc821c 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -753,6 +753,7 @@ void registerStorageFileLog(StorageFactory & factory) if (!num_threads) /// Default { num_threads = std::max(unsigned(1), physical_cpu_cores / 4); + filelog_settings->set("max_threads", num_threads); } else if (num_threads > physical_cpu_cores) { From b56c2f9499c742f46f8fbf51551112eaa4aadd17 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 29 Oct 2021 11:40:21 +0300 Subject: [PATCH 212/396] Remove redundant lines from http buffer --- src/IO/ReadWriteBufferFromHTTP.h | 3 --- .../test_disk_over_web_server/configs/async_read.xml | 7 +++++++ tests/integration/test_disk_over_web_server/test.py | 9 ++++++--- 3 files changed, 13 insertions(+), 6 deletions(-) create mode 100644 tests/integration/test_disk_over_web_server/configs/async_read.xml diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 5254b140679..16259e8057e 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -252,9 +252,6 @@ namespace detail impl->position() = position(); } - if (!working_buffer.empty()) - impl->position() = position(); - if (!impl->next()) return false; diff --git a/tests/integration/test_disk_over_web_server/configs/async_read.xml b/tests/integration/test_disk_over_web_server/configs/async_read.xml new file mode 100644 index 00000000000..4449d83779a --- /dev/null +++ b/tests/integration/test_disk_over_web_server/configs/async_read.xml @@ -0,0 +1,7 @@ + + + + read_threadpool + + + diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index b8629fdbe9e..55f760f514f 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -11,6 +11,7 @@ def cluster(): cluster.add_instance("node1", main_configs=["configs/storage_conf.xml"], with_nginx=True) cluster.add_instance("node2", main_configs=["configs/storage_conf_web.xml"], with_nginx=True) cluster.add_instance("node3", main_configs=["configs/storage_conf_web.xml"], with_nginx=True) + cluster.add_instance("node_async_read", main_configs=["configs/storage_conf_web.xml"], user_configs=["configs/async_read.xml"], with_nginx=True) cluster.start() node1 = cluster.instances["node1"] @@ -37,9 +38,10 @@ def cluster(): cluster.shutdown() -def test_usage(cluster): +@pytest.mark.parametrize("node_name", ["node2", "node_async_read"]) +def test_usage(cluster, node_name): node1 = cluster.instances["node1"] - node2 = cluster.instances["node2"] + node2 = cluster.instances[node_name] global uuids assert(len(uuids) == 3) for i in range(3): @@ -49,6 +51,8 @@ def test_usage(cluster): SETTINGS storage_policy = 'web'; """.format(i, uuids[i], i, i)) + result = node2.query("SELECT * FROM test{} settings max_threads=20".format(i)) + result = node2.query("SELECT count() FROM test{}".format(i)) assert(int(result) == 500000 * (i+1)) @@ -82,4 +86,3 @@ def test_incorrect_usage(cluster): assert("Table is read-only" in result) node2.query("DROP TABLE test0") - From 44b5dd116192fe3778d2676ee2ab7fe588f4df4a Mon Sep 17 00:00:00 2001 From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com> Date: Fri, 29 Oct 2021 11:44:28 +0300 Subject: [PATCH 213/396] Refactoring in codec encrypted (#30564) * refactoring * remove mistake in docs --- .../settings.md | 2 +- src/Compression/CompressionCodecEncrypted.cpp | 40 ++++++++++--------- 2 files changed, 23 insertions(+), 19 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index cdf49678570..b67747f769c 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -131,7 +131,7 @@ Also, users can add nonce that must be 12 bytes long (by default encryption and ```xml - 0123456789101 + 012345678910 ``` diff --git a/src/Compression/CompressionCodecEncrypted.cpp b/src/Compression/CompressionCodecEncrypted.cpp index 96d264fea1e..c3f14fe1449 100644 --- a/src/Compression/CompressionCodecEncrypted.cpp +++ b/src/Compression/CompressionCodecEncrypted.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -81,9 +82,11 @@ namespace ErrorCodes namespace { -constexpr size_t tag_size = 16; /// AES-GCM-SIV always uses a tag of 16 bytes length -constexpr size_t key_id_max_size = 8; /// Max size of varint. -constexpr size_t nonce_max_size = 13; /// Nonce size and one byte to show if nonce in in text +constexpr size_t tag_size = 16; /// AES-GCM-SIV always uses a tag of 16 bytes length +constexpr size_t key_id_max_size = 8; /// Max size of varint. +constexpr size_t nonce_max_size = 13; /// Nonce size and one byte to show if nonce in in text +constexpr size_t actual_nonce_size = 12; /// Nonce actual size +const String empty_nonce = {"\0\0\0\0\0\0\0\0\0\0\0\0", actual_nonce_size}; /// Get encryption/decryption algorithms. auto getMethod(EncryptionMethod Method) @@ -137,7 +140,7 @@ size_t encrypt(const std::string_view & plaintext, char * ciphertext_and_tag, En EVP_AEAD_CTX_zero(&encrypt_ctx); const int ok_init = EVP_AEAD_CTX_init(&encrypt_ctx, getMethod(method)(), reinterpret_cast(key.data()), key.size(), - 16 /* tag size */, nullptr); + tag_size, nullptr); if (!ok_init) throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR); @@ -145,7 +148,7 @@ size_t encrypt(const std::string_view & plaintext, char * ciphertext_and_tag, En size_t out_len; const int ok_open = EVP_AEAD_CTX_seal(&encrypt_ctx, reinterpret_cast(ciphertext_and_tag), - &out_len, plaintext.size() + 16, + &out_len, plaintext.size() + tag_size, reinterpret_cast(nonce.data()), nonce.size(), reinterpret_cast(plaintext.data()), plaintext.size(), nullptr, 0); @@ -167,7 +170,7 @@ size_t decrypt(const std::string_view & ciphertext, char * plaintext, Encryption const int ok_init = EVP_AEAD_CTX_init(&decrypt_ctx, getMethod(method)(), reinterpret_cast(key.data()), key.size(), - 16 /* tag size */, nullptr); + tag_size, nullptr); if (!ok_init) throw Exception(lastErrorString(), ErrorCodes::OPENSSL_ERROR); @@ -221,7 +224,7 @@ inline char* writeNonce(const String& nonce, char* dest) { /// If nonce consists of nul bytes, it shouldn't be in dest. Zero byte is the only byte that should be written. /// Otherwise, 1 is written and data from nonce is copied - if (nonce != String("\0\0\0\0\0\0\0\0\0\0\0\0", 12)) + if (nonce != empty_nonce) { *dest = 1; ++dest; @@ -246,15 +249,15 @@ inline const char* readNonce(String& nonce, const char* source) /// If first is zero byte: move source and set zero-bytes nonce if (!*source) { - nonce = {"\0\0\0\0\0\0\0\0\0\0\0\0", 12}; + nonce = empty_nonce; return ++source; } /// Move to next byte. Nonce will begin from there ++source; /// Otherwise, use data from source in nonce - nonce = {source, 12}; - source += 12; + nonce = {source, actual_nonce_size}; + source += actual_nonce_size; return source; } @@ -332,14 +335,14 @@ void CompressionCodecEncrypted::Configuration::loadImpl( if (!new_params->keys_storage[method].contains(new_params->current_key_id[method])) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not found a key with the current ID {}", new_params->current_key_id[method]); - /// Read nonce (in hex or in string). Its length should be 12 bytes. + /// Read nonce (in hex or in string). Its length should be 12 bytes (actual_nonce_size). if (config.has(config_prefix + ".nonce_hex")) new_params->nonce[method] = unhexKey(config.getString(config_prefix + ".nonce_hex")); else new_params->nonce[method] = config.getString(config_prefix + ".nonce", ""); - if (new_params->nonce[method].size() != 12 && !new_params->nonce[method].empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Got nonce with unexpected size {}, the size should be 12", new_params->nonce[method].size()); + if (new_params->nonce[method].size() != actual_nonce_size && !new_params->nonce[method].empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Got nonce with unexpected size {}, the size should be {}", new_params->nonce[method].size(), actual_nonce_size); } bool CompressionCodecEncrypted::Configuration::tryLoad(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) @@ -399,7 +402,7 @@ void CompressionCodecEncrypted::Configuration::getCurrentKeyAndNonce(EncryptionM /// This will lead to data loss. nonce = current_params->nonce[method]; if (nonce.empty()) - nonce = {"\0\0\0\0\0\0\0\0\0\0\0\0", 12}; + nonce = empty_nonce; } String CompressionCodecEncrypted::Configuration::getKey(EncryptionMethod method, const UInt64 & key_id) const @@ -448,8 +451,10 @@ UInt32 CompressionCodecEncrypted::getMaxCompressedDataSize(UInt32 uncompressed_s UInt32 CompressionCodecEncrypted::doCompressData(const char * source, UInt32 source_size, char * dest) const { - // Generate an IV out of the data block and the key-generation - // key. It is completely deterministic, but does not leak any + // Nonce, key and plaintext will be used to generate authentication tag + // and message encryption key. AES-GCM-SIV authenticates the encoded additional data and plaintext. + // For this purpose message_authentication_key is used. + // Algorithm is completely deterministic, but does not leak any // information about the data block except for equivalence of // identical blocks (under the same key). @@ -470,8 +475,7 @@ UInt32 CompressionCodecEncrypted::doCompressData(const char * source, UInt32 sou char* ciphertext = writeNonce(nonce, ciphertext_with_nonce); UInt64 nonce_size = ciphertext - ciphertext_with_nonce; - // The IV will be used as an authentication tag. The ciphertext and the - // tag will be written directly in the dest buffer. + // The ciphertext and the authentication tag will be written directly in the dest buffer. size_t out_len = encrypt(plaintext, ciphertext, encryption_method, current_key, nonce); /// Length of encrypted text should be equal to text length plus tag_size (which was added by algorithm). From 2dc1da30d3a600fa227166c7bc9cca96c6f37d28 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 29 Oct 2021 12:32:32 +0300 Subject: [PATCH 214/396] Update adopters.md --- docs/en/introduction/adopters.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 6dd53fffb86..787cd451d96 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -170,5 +170,6 @@ toc_title: Adopters | ЦФТ | Banking, Financial products, Payments | — | — | — | [Meetup in Russian, April 2020](https://team.cft.ru/events/162) | | Цифровой Рабочий | Industrial IoT, Analytics | — | — | — | [Blog post in Russian, March 2021](https://habr.com/en/company/croc/blog/548018/) | | ООО «МПЗ Богородский» | Agriculture | — | — | — | [Article in Russian, November 2020](https://cloud.yandex.ru/cases/okraina) | +| ДомКлик | Real Estate | — | — | — | [Article in Russian, October 2021](https://habr.com/ru/company/domclick/blog/585936/) | [Original article](https://clickhouse.com/docs/en/introduction/adopters/) From a2866de25c0317da95a9d21fd5c23cccb358dc70 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 12:58:25 +0300 Subject: [PATCH 215/396] Add docs check to documentation --- .github/workflows/main.yml | 27 +++++- docker/docs/builder/Dockerfile | 7 +- docker/images.json | 7 +- tests/ci/docs_check.py | 148 +++++++++++++++++++++++++++++++++ tests/ci/pr_info.py | 15 ++++ 5 files changed, 197 insertions(+), 7 deletions(-) create mode 100644 tests/ci/docs_check.py diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 69dc2e94e15..22f6aa775f3 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -21,7 +21,6 @@ jobs: python3 run_check.py DockerHubPush: needs: CheckLabels - if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, style-checker] steps: - name: Check out repository code @@ -57,8 +56,33 @@ jobs: run: | docker kill $(docker ps -q) ||: sudo rm -fr $TEMP_PATH + DocsCheck: + needs: DockerHubPush + runs-on: [self-hosted, style-checker] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/docs_check + - name: Check out repository code + uses: actions/checkout@v2 + - name: Style Check + env: + TEMP_PATH: ${{runner.temp}}/docs_check + REPO_COPY: ${{runner.temp}}/docs_check/ClickHouse + run: | + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 docs_check.py + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + sudo rm -fr $TEMP_PATH BuilderDebDebug: needs: DockerHubPush + if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: - name: Download changed images @@ -179,6 +203,7 @@ jobs: 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') }} runs-on: [self-hosted, builder] steps: - name: Check out repository code diff --git a/docker/docs/builder/Dockerfile b/docker/docs/builder/Dockerfile index 1fc27c5fc18..0b0f4ef71b5 100644 --- a/docker/docs/builder/Dockerfile +++ b/docker/docs/builder/Dockerfile @@ -1,4 +1,4 @@ -# docker build -t yandex/clickhouse-docs-build . +# docker build -t clickhouse/docs-build . FROM ubuntu:20.04 ENV LANG=C.UTF-8 @@ -44,7 +44,4 @@ certifi==2020.4.5.2 chardet==3.0.4 click==7.1.2 closure==20191111 cssmin==0.2.0 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 * / +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 diff --git a/docker/images.json b/docker/images.json index 1cb999bd16f..c49f10d676f 100644 --- a/docker/images.json +++ b/docker/images.json @@ -169,6 +169,11 @@ }, "docker/docs/builder": { "name": "clickhouse/docs-builder", - "dependent": [] + "dependent": ["docker/docs/check"] + }, + "docker/docs/check": { + "name": "clickhouse/docs-check", + "dependent": [] } + } diff --git a/tests/ci/docs_check.py b/tests/ci/docs_check.py new file mode 100644 index 00000000000..d8a68d29cd3 --- /dev/null +++ b/tests/ci/docs_check.py @@ -0,0 +1,148 @@ +#!/usr/bin/env python3 +import logging +import subprocess +import os +import time +import json +import sys +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 + +NAME = "Docs Check (actions)" + +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, additional_files): + s3_path_prefix = f"{pr_number}/{commit_sha}/docs_check" + 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(NAME, test_results, raw_log_url, task_url, branch_url, branch_name, commit_url, additional_urls) + 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 + +if __name__ == "__main__": + logging.basicConfig(level=logging.INFO) + + temp_path = os.path.join(os.getenv("TEMP_PATH")) + repo_path = os.path.join(os.getenv("REPO_COPY")) + + with open(os.getenv('GITHUB_EVENT_PATH'), 'r', encoding='utf-8') as event_file: + event = json.load(event_file) + + pr_info = PRInfo(event, need_changed_files=True) + + gh = Github(get_best_robot_token()) + if not pr_info.has_changes_in_documentation(): + logging.info ("No changes in documentation") + commit = get_commit(gh, pr_info.sha) + commit.create_status(context=NAME, description="No changes in docs", state="success") + sys.exit(0) + + logging.info("Has changes in docs") + + if not os.path.exists(temp_path): + os.makedirs(temp_path) + + images_path = os.path.join(temp_path, 'changed_images.json') + + docker_image = 'clickhouse/docs-check' + if 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 'clickhouse/docs-check' in images: + docker_image += ':' + images['clickhouse/docs-check'] + + logging.info("Got docker image %s", docker_image) + for i in range(10): + try: + subprocess.check_output(f"docker pull {docker_image}", 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_image}") + + test_output = os.path.join(temp_path, 'docs_check_log') + if not os.path.exists(test_output): + os.makedirs(test_output) + + cmd = "docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={test_output}:/test_output {docker_image}" + + run_log_path = os.path.join(test_output, 'runlog.log') + + with open(run_log_path, 'w', encoding='utf-8') as log: + with subprocess.Popen(cmd, shell=True, stderr=log, stdout=log) as process: + retcode = process.wait() + if retcode == 0: + logging.info("Run successfully") + status = "Success" + description = "Run Ok" + else: + description = "Run failed (non zero exit code)" + status = "failure" + logging.info("Run failed") + + subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) + files = os.listdir(test_output) + lines = [] + additional_files = [] + if not files: + logging.error("No output files after docs check") + description = "No output files after docs check" + status = "failure" + else: + for f in files: + path = os.path.join(test_output, f) + additional_files.append(path) + with open(path, 'r', encoding='utf-8') as check_file: + for line in check_file: + if "ERROR" in line: + lines.append((line.split(':')[-1], "FAIL")) + if lines: + status = "failure" + description = "Found errors during docs check" + else: + lines.append(("No errors found", "OK")) + + s3_helper = S3Helper('https://s3.amazonaws.com') + + 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 0b4aeb56699..672043bd7b7 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -2,8 +2,11 @@ import urllib import requests from unidiff import PatchSet +import os +DIFF_IN_DOCUMENTATION_EXT = [".html", ".md", ".yml", ".txt", ".css", ".js", ".xml", ".ico", ".conf", ".svg", ".png", ".jpg", ".py", ".sh"] + class PRInfo: def __init__(self, github_event, need_orgs=False, need_changed_files=False): self.number = github_event['number'] @@ -37,6 +40,18 @@ class PRInfo: 'user_orgs': self.user_orgs, } + def has_changes_in_documentation(self): + # If the list wasn't built yet the best we can do is to + # assume that there were changes. + if self.changed_files is None or not self.changed_files: + return True + + for f in self.changed_files: + _, ext = os.path.splitext(f) + if ext in DIFF_IN_DOCUMENTATION_EXT or 'Dockerfile' in f: + return True + return False + class FakePRInfo: def __init__(self): From 86f634b65c8290c4afb0ee3d5caea2815cd3a8b5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 12:59:50 +0300 Subject: [PATCH 216/396] Tabs to spaces --- docker/images.json | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/images.json b/docker/images.json index c49f10d676f..adbaf794e71 100644 --- a/docker/images.json +++ b/docker/images.json @@ -169,11 +169,11 @@ }, "docker/docs/builder": { "name": "clickhouse/docs-builder", - "dependent": ["docker/docs/check"] + "dependent": ["docker/docs/check"] }, "docker/docs/check": { "name": "clickhouse/docs-check", - "dependent": [] + "dependent": [] } } From a5097c3d0028be0fc1aa5877fdeadb9e6e1165d3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 13:00:27 +0300 Subject: [PATCH 217/396] Add docs check image --- docker/docs/check/Dockerfile | 9 +++++++++ docker/docs/check/run.sh | 6 ++++++ 2 files changed, 15 insertions(+) create mode 100644 docker/docs/check/Dockerfile create mode 100644 docker/docs/check/run.sh diff --git a/docker/docs/check/Dockerfile b/docker/docs/check/Dockerfile new file mode 100644 index 00000000000..55647df5c3e --- /dev/null +++ b/docker/docs/check/Dockerfile @@ -0,0 +1,9 @@ +# docker build -t clickhouse/docs-check . +FROM clickhouse/docs-builder + +COPY run.sh / + +ENV REPO_PATH=/repo_path +ENV OUTPUT_PATH=/output_path + +CMD ["/bin/bash", "/run.sh"] diff --git a/docker/docs/check/run.sh b/docker/docs/check/run.sh new file mode 100644 index 00000000000..57f39afec27 --- /dev/null +++ b/docker/docs/check/run.sh @@ -0,0 +1,6 @@ +#!/usr/bin/env bash +set -euo pipefail + + +cd $REPO_PATH/docs/tools +./build.py --skip-git-log 2>&1 | tee $OUTPUT_PATH/output.log From 9a9d5950aa04b755aff3e45b071b008d277c1629 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 13:01:33 +0300 Subject: [PATCH 218/396] Fix yml --- .github/workflows/main.yml | 48 +++++++++++++++++++------------------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 22f6aa775f3..36da6c4e40f 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -56,30 +56,30 @@ jobs: run: | docker kill $(docker ps -q) ||: sudo rm -fr $TEMP_PATH - DocsCheck: - needs: DockerHubPush - runs-on: [self-hosted, style-checker] - steps: - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ runner.temp }}/docs_check - - name: Check out repository code - uses: actions/checkout@v2 - - name: Style Check - env: - TEMP_PATH: ${{runner.temp}}/docs_check - REPO_COPY: ${{runner.temp}}/docs_check/ClickHouse - run: | - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 docs_check.py - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - sudo rm -fr $TEMP_PATH + DocsCheck: + needs: DockerHubPush + runs-on: [self-hosted, style-checker] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/docs_check + - name: Check out repository code + uses: actions/checkout@v2 + - name: Style Check + env: + TEMP_PATH: ${{runner.temp}}/docs_check + REPO_COPY: ${{runner.temp}}/docs_check/ClickHouse + run: | + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 docs_check.py + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + sudo rm -fr $TEMP_PATH BuilderDebDebug: needs: DockerHubPush if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} From 4eca493ab8d86b7d06e8d3b7cd2a11e4ff3ebbf9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 13:47:29 +0300 Subject: [PATCH 219/396] Fix fuzzer on master --- docker/test/fuzzer/run-fuzzer.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index fdf5a1eab40..0c11e0a615d 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -36,10 +36,11 @@ function clone git diff --name-only master HEAD | tee ci-changed-files.txt else if [ -v COMMIT_SHA ]; then - git fetch --depth 1 origin "$SHA_TO_TEST" + git fetch --depth 2 origin "$SHA_TO_TEST" git checkout "$SHA_TO_TEST" echo "Checked out nominal SHA $SHA_TO_TEST for master" else + git fetch --depth 2 origin echo "Using default repository head $(git rev-parse HEAD)" fi git diff --name-only HEAD~1 HEAD | tee ci-changed-files.txt From 2ac17c57ec175659ee546502bba32e5a1f6716ad Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 29 Oct 2021 14:28:41 +0300 Subject: [PATCH 220/396] DOCSUP-15710: [CLICKHOUSEDOCS] Edit and translate to Russian (s2 functions) (#29787) --- docs/en/sql-reference/functions/geo/s2.md | 126 ++++---- docs/ru/sql-reference/functions/geo/s2.md | 376 ++++++++++++++++++++++ 2 files changed, 434 insertions(+), 68 deletions(-) create mode 100644 docs/ru/sql-reference/functions/geo/s2.md diff --git a/docs/en/sql-reference/functions/geo/s2.md b/docs/en/sql-reference/functions/geo/s2.md index d669b1c8b32..f8736bcc61a 100644 --- a/docs/en/sql-reference/functions/geo/s2.md +++ b/docs/en/sql-reference/functions/geo/s2.md @@ -2,13 +2,13 @@ toc_title: S2 Geometry --- -# Functions for Working with S2 Index {#s2Index} +# Functions for Working with S2 Index {#s2index} [S2](https://s2geometry.io/) is a geographical indexing system where all geographical data is represented on a three-dimensional sphere (similar to a globe). -In the S2 library points are represented as unit length vectors called S2 point indices (points on the surface of a three dimensional unit sphere) as opposed to traditional (latitude, longitude) pairs. +In the S2 library points are represented as the S2 Index - a specific number which encodes internally a point on the surface of a unit sphere, unlike traditional (latitude, longitude) pairs. To get the S2 point index for a given point specified in the format (latitude, longitude) use the [geoToS2](#geotos2) function. Also, you can use the [s2ToGeo](#s2togeo) function for getting geographical coordinates corresponding to the specified S2 point index. -## geoToS2 {#geoToS2} +## geoToS2 {#geotos2} Returns [S2](#s2index) point index corresponding to the provided coordinates `(longitude, latitude)`. @@ -34,7 +34,7 @@ Type: [UInt64](../../../sql-reference/data-types/int-uint.md). Query: ``` sql -SELECT geoToS2(37.79506683, 55.71290588) as s2Index; +SELECT geoToS2(37.79506683, 55.71290588) AS s2Index; ``` Result: @@ -45,7 +45,7 @@ Result: └─────────────────────┘ ``` -## s2ToGeo {#s2ToGeo} +## s2ToGeo {#s2togeo} Returns geo coordinates `(longitude, latitude)` corresponding to the provided [S2](#s2index) point index. @@ -57,20 +57,20 @@ s2ToGeo(s2index) **Arguments** -- `s2Index` — S2 Index. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2index` — S2 Index. [UInt64](../../../sql-reference/data-types/int-uint.md). **Returned values** - A tuple consisting of two values: `tuple(lon,lat)`. -Type: `lon` - [Float64](../../../sql-reference/data-types/float.md). `lat` — [Float64](../../../sql-reference/data-types/float.md). +Type: `lon` — [Float64](../../../sql-reference/data-types/float.md). `lat` — [Float64](../../../sql-reference/data-types/float.md). **Example** Query: ``` sql -SELECT s2ToGeo(4704772434919038107) as s2Coodrinates; +SELECT s2ToGeo(4704772434919038107) AS s2Coodrinates; ``` Result: @@ -81,9 +81,9 @@ Result: └──────────────────────────────────────┘ ``` -## s2GetNeighbors {#s2GetNeighbors} +## s2GetNeighbors {#s2getneighbors} -Returns S2 neighbor indices corresponding to the provided [S2](#s2index)). Each cell in the S2 system is a quadrilateral bounded by four geodesics. So, each cell has 4 neighbors. +Returns S2 neighbor indixes corresponding to the provided [S2](#s2index). Each cell in the S2 system is a quadrilateral bounded by four geodesics. So, each cell has 4 neighbors. **Syntax** @@ -97,16 +97,16 @@ s2GetNeighbors(s2index) **Returned values** -- An array consisting of the 4 neighbor indices: `array[s2index1, s2index3, s2index2, s2index4]`. +- An array consisting of 4 neighbor indexes: `array[s2index1, s2index3, s2index2, s2index4]`. -Type: Each S2 index is [UInt64](../../../sql-reference/data-types/int-uint.md). +Type: [UInt64](../../../sql-reference/data-types/int-uint.md). **Example** Query: ``` sql - select s2GetNeighbors(5074766849661468672) AS s2Neighbors; +SELECT s2GetNeighbors(5074766849661468672) AS s2Neighbors; ``` Result: @@ -117,9 +117,9 @@ Result: └───────────────────────────────────────────────────────────────────────────────────┘ ``` -## s2CellsIntersect {#s2CellsIntersect} +## s2CellsIntersect {#s2cellsintersect} -Determines if the two provided [S2](#s2index)) cell indices intersect or not. +Determines if the two provided [S2](#s2index) cells intersect or not. **Syntax** @@ -133,8 +133,8 @@ s2CellsIntersect(s2index1, s2index2) **Returned values** -- 1 — If the S2 cell indices intersect. -- 0 — If the S2 cell indices don't intersect. +- 1 — If the cells intersect. +- 0 — If the cells don't intersect. Type: [UInt8](../../../sql-reference/data-types/int-uint.md). @@ -143,7 +143,7 @@ Type: [UInt8](../../../sql-reference/data-types/int-uint.md). Query: ``` sql - select s2CellsIntersect(9926595209846587392, 9926594385212866560) as intersect; +SELECT s2CellsIntersect(9926595209846587392, 9926594385212866560) AS intersect; ``` Result: @@ -154,11 +154,9 @@ Result: └───────────┘ ``` -## s2CapContains {#s2CapContains} +## s2CapContains {#s2capcontains} -A cap represents a portion of the sphere that has been cut off by a plane. It is defined by a point on a sphere and a radius in degrees. - -Determines if a cap contains a s2 point index. +Determines if a cap contains a S2 point. A cap represents a part of the sphere that has been cut off by a plane. It is defined by a point on a sphere and a radius in degrees. **Syntax** @@ -168,9 +166,9 @@ s2CapContains(center, degrees, point) **Arguments** -- `center` - S2 point index corresponding to the cap. [UInt64](../../../sql-reference/data-types/int-uint.md). - - `degrees` - Radius of the cap in degrees. [Float64](../../../sql-reference/data-types/float.md). - - `point` - S2 point index. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `center` — S2 point index corresponding to the cap. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `degrees` — Radius of the cap in degrees. [Float64](../../../sql-reference/data-types/float.md). +- `point` — S2 point index. [UInt64](../../../sql-reference/data-types/int-uint.md). **Returned values** @@ -184,7 +182,7 @@ Type: [UInt8](../../../sql-reference/data-types/int-uint.md). Query: ``` sql -select s2CapContains(1157339245694594829, 1.0, 1157347770437378819) as capContains; +SELECT s2CapContains(1157339245694594829, 1.0, 1157347770437378819) AS capContains; ``` Result: @@ -195,11 +193,9 @@ Result: └─────────────┘ ``` -## s2CapUnion {#s2CapUnion} +## s2CapUnion {#s2capunion} -A cap represents a portion of the sphere that has been cut off by a plane. It is defined by a point on a sphere and a radius in degrees. - -Determines the smallest cap that contains the given two input caps. +Determines the smallest cap that contains the given two input caps. A cap represents a portion of the sphere that has been cut off by a plane. It is defined by a point on a sphere and a radius in degrees. **Syntax** @@ -209,13 +205,13 @@ s2CapUnion(center1, radius1, center2, radius2) **Arguments** -- `center1`, `center2` - S2 point indices corresponding to the two input caps. [UInt64](../../../sql-reference/data-types/int-uint.md). - - `radius1`, `radius2` - Radii of the two input caps in degrees. [Float64](../../../sql-reference/data-types/float.md). +- `center1`, `center2` — S2 point indixes corresponding to the two input caps. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `radius1`, `radius2` — Radius of the two input caps in degrees. [Float64](../../../sql-reference/data-types/float.md). **Returned values** -- `center` - S2 point index corresponding the center of the smallest cap containing the two input caps. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). - - `radius` - Radius of the smallest cap containing the two input caps. Type: [Float64](../../../sql-reference/data-types/float.md). +- `center` — S2 point index corresponding the center of the smallest cap containing the two input caps. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). +- `radius` — Radius of the smallest cap containing the two input caps. Type: [Float64](../../../sql-reference/data-types/float.md). **Example** @@ -233,11 +229,9 @@ Result: └────────────────────────────────────────┘ ``` -## s2RectAdd{#s2RectAdd} +## s2RectAdd {#s2rectadd} -In the S2 system, a rectangle is represented by a type of S2Region called a S2LatLngRect that represents a rectangle in latitude-longitude space. - -Increases the size of the bounding rectangle to include the given S2 point index. +Increases the size of the bounding rectangle to include the given S2 point. In the S2 system, a rectangle is represented by a type of S2Region called a `S2LatLngRect` that represents a rectangle in latitude-longitude space. **Syntax** @@ -247,21 +241,21 @@ s2RectAdd(s2pointLow, s2pointHigh, s2Point) **Arguments** -- `s2PointLow` - Low S2 point index corresponding to the rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). -- `s2PointHigh` - High S2 point index corresponding to the rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). -- `s2Point` - Target S2 point index that the bound rectangle should be grown to include. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2PointLow` — Low S2 point index corresponding to the rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2PointHigh` — High S2 point index corresponding to the rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2Point` — Target S2 point index that the bound rectangle should be grown to include. [UInt64](../../../sql-reference/data-types/int-uint.md). **Returned values** -- `s2PointLow` - Low S2 cell id corresponding to the grown rectangle. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). - - `s2PointHigh` - Hight S2 cell id corresponding to the grown rectangle. Type: [UInt64](../../../sql-reference/data-types/float.md). +- `s2PointLow` — Low S2 cell id corresponding to the grown rectangle. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2PointHigh` — Hight S2 cell id corresponding to the grown rectangle. Type: [UInt64](../../../sql-reference/data-types/float.md). **Example** Query: ``` sql -SELECT s2RectAdd(5178914411069187297, 5177056748191934217, 5179056748191934217) as rectAdd; +SELECT s2RectAdd(5178914411069187297, 5177056748191934217, 5179056748191934217) AS rectAdd; ``` Result: @@ -272,11 +266,9 @@ Result: └───────────────────────────────────────────┘ ``` -## s2RectContains{#s2RectContains} +## s2RectContains {#s2rectcontains} -In the S2 system, a rectangle is represented by a type of S2Region called a S2LatLngRect that represents a rectangle in latitude-longitude space. - -Determines if a given rectangle contains a S2 point index. +Determines if a given rectangle contains a S2 point. In the S2 system, a rectangle is represented by a type of S2Region called a `S2LatLngRect` that represents a rectangle in latitude-longitude space. **Syntax** @@ -286,9 +278,9 @@ s2RectContains(s2PointLow, s2PointHi, s2Point) **Arguments** -- `s2PointLow` - Low S2 point index corresponding to the rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). -- `s2PointHigh` - High S2 point index corresponding to the rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). -- `s2Point` - Target S2 point index. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2PointLow` — Low S2 point index corresponding to the rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2PointHigh` — High S2 point index corresponding to the rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2Point` — Target S2 point index. [UInt64](../../../sql-reference/data-types/int-uint.md). **Returned values** @@ -300,7 +292,7 @@ s2RectContains(s2PointLow, s2PointHi, s2Point) Query: ``` sql -SELECT s2RectContains(5179062030687166815, 5177056748191934217, 5177914411069187297) AS rectContains +SELECT s2RectContains(5179062030687166815, 5177056748191934217, 5177914411069187297) AS rectContains; ``` Result: @@ -311,11 +303,9 @@ Result: └──────────────┘ ``` -## s2RectUinion{#s2RectUnion} +## s2RectUinion {#s2rectunion} -In the S2 system, a rectangle is represented by a type of S2Region called a S2LatLngRect that represents a rectangle in latitude-longitude space. - -Returns the smallest rectangle containing the union of this rectangle and the given rectangle. +Returns the smallest rectangle containing the union of this rectangle and the given rectangle. In the S2 system, a rectangle is represented by a type of S2Region called a `S2LatLngRect` that represents a rectangle in latitude-longitude space. **Syntax** @@ -325,20 +315,20 @@ s2RectUnion(s2Rect1PointLow, s2Rect1PointHi, s2Rect2PointLow, s2Rect2PointHi) **Arguments** -- `s2Rect1PointLow`, `s2Rect1PointHi` - Low and High S2 point indices corresponding to the first rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). -- `s2Rect2PointLow`, `s2Rect2PointHi` - Low and High S2 point indices corresponding to the second rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2Rect1PointLow`, `s2Rect1PointHi` — Low and High S2 point indexes corresponding to the first rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2Rect2PointLow`, `s2Rect2PointHi` — Low and High S2 point indexes corresponding to the second rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). **Returned values** -- `s2UnionRect2PointLow` - Low S2 cell id corresponding to the union rectangle. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). -- `s2UnionRect2PointHi` - High S2 cell id corresponding to the union rectangle. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2UnionRect2PointLow` — Low S2 cell id corresponding to the union rectangle. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2UnionRect2PointHi` — High S2 cell id corresponding to the union rectangle. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). **Example** Query: ``` sql -SELECT s2RectUnion(5178914411069187297, 5177056748191934217, 5179062030687166815, 5177056748191934217) AS rectUnion +SELECT s2RectUnion(5178914411069187297, 5177056748191934217, 5179062030687166815, 5177056748191934217) AS rectUnion; ``` Result: @@ -349,9 +339,9 @@ Result: └───────────────────────────────────────────┘ ``` -## s2RectIntersection{#s2RectIntersection} +## s2RectIntersection {#s2rectintersection} -Returns the smallest Rectangle containing the intersection of this rectangle and the given rectangle. +Returns the smallest rectangle containing the intersection of this rectangle and the given rectangle. In the S2 system, a rectangle is represented by a type of S2Region called a `S2LatLngRect` that represents a rectangle in latitude-longitude space. **Syntax** @@ -361,20 +351,20 @@ s2RectIntersection(s2Rect1PointLow, s2Rect1PointHi, s2Rect2PointLow, s2Rect2Poin **Arguments** -- `s2Rect1PointLow`, `s2Rect1PointHi` - Low and High S2 point indices corresponding to the first rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). -- `s2Rect2PointLow`, `s2Rect2PointHi` - Low and High S2 point indices corresponding to the second rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2Rect1PointLow`, `s2Rect1PointHi` — Low and High S2 point indexes corresponding to the first rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2Rect2PointLow`, `s2Rect2PointHi` — Low and High S2 point indexes corresponding to the second rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). **Returned values** -- `s2UnionRect2PointLow` - Low S2 cell id corresponding to the rectangle containing the intersection of the given rectangles. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). -- `s2UnionRect2PointHi` - Hi S2 cell id corresponding to the rectangle containing the intersection of the given rectangles. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2UnionRect2PointLow` — Low S2 cell id corresponding to the rectangle containing the intersection of the given rectangles. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2UnionRect2PointHi` — High S2 cell id corresponding to the rectangle containing the intersection of the given rectangles. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). **Example** Query: ``` sql -SELECT s2RectIntersection(5178914411069187297, 5177056748191934217, 5179062030687166815, 5177056748191934217) AS rectIntersection +SELECT s2RectIntersection(5178914411069187297, 5177056748191934217, 5179062030687166815, 5177056748191934217) AS rectIntersection; ``` Result: diff --git a/docs/ru/sql-reference/functions/geo/s2.md b/docs/ru/sql-reference/functions/geo/s2.md new file mode 100644 index 00000000000..6b801e1d08f --- /dev/null +++ b/docs/ru/sql-reference/functions/geo/s2.md @@ -0,0 +1,376 @@ +--- +toc_title: "Функции для работы с индексами S2" +--- + +# Функции для работы с индексами S2 {#s2index} + +[S2](https://s2geometry.io/) — это система геокодирования, в которой все географические данные представлены на трехмерной сфере (аналогично глобусу). + +В библиотеке S2 точки представлены в виде индекса S2 — определенного числа, которое внутренне кодирует точку на поверхности трехмерной единичной сферы, в отличие от традиционных пар (широта, долгота). Чтобы получить индекс S2 для точки, заданной в формате (широта, долгота), используйте функцию [geoToS2](#geotools2). Также вы можете использовать функцию [s2togeo](#s2togeo) для получения географических координат, соответствующих заданному S2 индексу точки. + +## geoToS2 {#geotos2} + +Возвращает [S2](#s2index) индекс точки, соответствующий заданным координатам в формате `(долгота, широта)`. + +**Синтаксис** + +``` sql +geoToS2(lon, lat) +``` + +**Аргументы** + +- `lon` — долгота. [Float64](../../../sql-reference/data-types/float.md). +- `lat` — широта. [Float64](../../../sql-reference/data-types/float.md). + +**Возвращаемое значение** + +- S2 индекс точки. + +Тип: [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT geoToS2(37.79506683, 55.71290588) AS s2Index; +``` + +Результат: + +``` text +┌─────────────s2Index─┐ +│ 4704772434919038107 │ +└─────────────────────┘ +``` + +## s2ToGeo {#s2togeo} + +Возвращает географические координаты `(долгота, широта)`, соответствующие заданному [S2](#s2index) индексу точки. + +**Синтаксис** + +``` sql +s2ToGeo(s2index) +``` + +**Аргументы** + +- `s2index` — [S2](#s2index) индекс. [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Возвращаемые значения** + +- Кортеж их двух значений: `tuple(lon,lat)`. + +Тип: `lon` — [Float64](../../../sql-reference/data-types/float.md). `lat` — [Float64](../../../sql-reference/data-types/float.md). + +**Пример** + +Запрос: + +``` sql +SELECT s2ToGeo(4704772434919038107) AS s2Coodrinates; +``` + +Результат: + +``` text +┌─s2Coodrinates────────────────────────┐ +│ (37.79506681471008,55.7129059052841) │ +└──────────────────────────────────────┘ +``` + +## s2GetNeighbors {#s2getneighbors} + +Возвращает [S2](#s2index) индексы ячеек, которые являются соседними для заданного S2 индекса. Ячейка в системе S2 представляет собой прямоугольник, ограниченный четырьмя сторонами. Соответственно, у каждой ячейки есть 4 соседние ячейки. + +**Синтаксис** + +``` sql +s2GetNeighbors(s2index) +``` + +**Аргументы** + +- `s2index` — [S2](#s2index) индекс. [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Возвращаемые значения** + +- Массив, содержащий 4 значения — S2 индекса соседних ячеек: `array[s2index1, s2index3, s2index2, s2index4]`. + +Тип: [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT s2GetNeighbors(5074766849661468672) AS s2Neighbors; +``` + +Результат: + +``` text +┌─s2Neighbors───────────────────────────────────────────────────────────────────────┐ +│ [5074766987100422144,5074766712222515200,5074767536856236032,5074767261978329088] │ +└───────────────────────────────────────────────────────────────────────────────────┘ +``` + +## s2CellsIntersect {#s2cellsintersect} + +Проверяет, пересекаются ли две заданные ячейки или нет. + +**Синтаксис** + +``` sql +s2CellsIntersect(s2index1, s2index2) +``` + +**Аргументы** + +- `siIndex1`, `s2index2` — S2 индексы первой и второй ячейки. [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Возвращаемые значения** + +- 1 — ячейки пересекаются. +- 0 — ячейки не пересекаются. + +Тип: [UInt8](../../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT s2CellsIntersect(9926595209846587392, 9926594385212866560) AS intersect; +``` + +Результат: + +``` text +┌─intersect─┐ +│ 1 │ +└───────────┘ +``` + +## s2CapContains {#s2capcontains} + +Определяет, содержит ли заданный купол указанную точку. Купол представляет собой часть сферы, которая была отрезана плоскостью. Купол задается точкой на сфере и радиусом в градусах. + +**Синтаксис** + +``` sql +s2CapContains(center, degrees, point) +``` + +**Аргументы** + +- `center` — S2 индекс точки, определяющей центр купола. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `degrees` — радиус купола в градусах. [Float64](../../../sql-reference/data-types/float.md). +- `point` — S2 индекс проверяемой точки. [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Возвращаемые значения** + +- 1 — купол содержит точку. +- 0 — купол не содержит точку. + +Тип: [UInt8](../../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT s2CapContains(1157339245694594829, 1.0, 1157347770437378819) AS capContains; +``` + +Результат: + +``` text +┌─capContains─┐ +│ 1 │ +└─────────────┘ +``` + +## s2CapUnion {#s2capunion} + +Определяет наименьший купол, содержащий два заданных купола. Купол представляет собой часть сферы, которая была отрезана плоскостью. Купол задается точкой на сфере и радиусом в градусах. + +**Синтаксис** + +``` sql +s2CapUnion(center1, radius1, center2, radius2) +``` + +**Аргументы** + +- `center1`, `center2` — S2 индексы точек, определяющие два центра куполов. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `radius1`, `radius2` — значения радиусов в градусах, определяющие два радиуса куполов. [Float64](../../../sql-reference/data-types/float.md). + +**Возвращаемые значения** + +- `center` — S2 индекс точки, соответствующий центру наименьшего купола, содержащего заданные купола. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md). +- `radius` — радиус в градусах наименьшего купола, содержащего заданные купола. Тип: [Float64](../../../sql-reference/data-types/float.md). + +**Пример** + +Запрос: + +``` sql +SELECT s2CapUnion(3814912406305146967, 1.0, 1157347770437378819, 1.0) AS capUnion; +``` + +Результат: + +``` text +┌─capUnion───────────────────────────────┐ +│ (4534655147792050737,60.2088283994957) │ +└────────────────────────────────────────┘ +``` + +## s2RectAdd {#s2rectadd} + +Увеличивает размер ограничивающего прямоугольника, чтобы включить в себя точку, заданную S2 индексом. В системе S2 прямоугольник представлен типом S2Region, называемым `S2LatLngRect`, который задает прямоугольник в пространстве широта-долгота. + +**Синтаксис** + +``` sql +s2RectAdd(s2pointLow, s2pointHigh, s2Point) +``` + +**Аргументы** + +- `s2PointLow` — S2 индекс нижней точки, которая задает ограничиваюший прямоугольник. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2PointHigh` — S2 индекс верхний точки, которая задает ограничиваюший прямоугольник. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2Point` — S2 индекс целевой точки, которая будет содержаться увеличенным ограничивающим прямоугольником. [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Возвращаемые значения** + +- `s2PointLow` — идентификатор нижней S2 ячейки, соответствующий увеличенному прямоугольнику. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2PointHigh` — идентификатор верхней S2 ячейки, соответствующий увеличенному прямоугольнику. Тип: [UInt64](../../../sql-reference/data-types/float.md). + +**Пример** + +Запрос: + +``` sql +SELECT s2RectAdd(5178914411069187297, 5177056748191934217, 5179056748191934217) AS rectAdd; +``` + +Результат: + +``` text +┌─rectAdd───────────────────────────────────┐ +│ (5179062030687166815,5177056748191934217) │ +└───────────────────────────────────────────┘ +``` + +## s2RectContains {#s2rectcontains} + +Проверяет, содержит ли заданный прямоугольник указанную S2 точку. В системе S2 прямоугольник представлен типом S2Region, называемым `S2LatLngRect`, который задает прямоугольник в пространстве широта-долгота. + +**Синтаксис** + +``` sql +s2RectContains(s2PointLow, s2PointHi, s2Point) +``` + +**Аргументы** + +- `s2PointLow` — S2 индекс самой низкой точки, которая задает прямоугольник. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2PointHigh` — S2 индекс самой высокой точки, которая задает прямоугольник. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2Point` — S2 индекс проверяемой точки. [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Возвращаемые значения** + +- 1 — прямоугольник содержит заданную точку. +- 0 — прямоугольник не содержит заданную точку. + +**Пример** + +Запрос: + +``` sql +SELECT s2RectContains(5179062030687166815, 5177056748191934217, 5177914411069187297) AS rectContains; +``` + +Результат: + +``` text +┌─rectContains─┐ +│ 0 │ +└──────────────┘ +``` + +## s2RectUinion {#s2rectunion} + +Возвращает наименьший прямоугольник, содержащий объединение двух заданных прямоугольников. В системе S2 прямоугольник представлен типом S2Region, называемым `S2LatLngRect`, который задает прямоугольник в пространстве широта-долгота. + +**Синтаксис** + +``` sql +s2RectUnion(s2Rect1PointLow, s2Rect1PointHi, s2Rect2PointLow, s2Rect2PointHi) +``` + +**Аргументы** + +- `s2Rect1PointLow`, `s2Rect1PointHi` — значения S2 индекса для самой низкой и самой высокой точек, которые задают первый прямоугольник. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2Rect2PointLow`, `s2Rect2PointHi` — значения S2 индекса для самой низкой и самой высокой точек, которые задают второй прямоугольник. [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Возвращаемые значения** + +- `s2UnionRect2PointLow` — идентификатор нижней ячейки, соответствующей объединенному прямоугольнику. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2UnionRect2PointHi` — идентификатор верхней ячейки, соответствующей объединенному прямоугольнику. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT s2RectUnion(5178914411069187297, 5177056748191934217, 5179062030687166815, 5177056748191934217) AS rectUnion; +``` + +Результат: + +``` text +┌─rectUnion─────────────────────────────────┐ +│ (5179062030687166815,5177056748191934217) │ +└───────────────────────────────────────────┘ +``` + +## s2RectIntersection {#s2rectintersection} + +Возвращает наименьший прямоугольник, содержащий пересечение двух заданных прямоугольников. В системе S2 прямоугольник представлен типом S2Region, называемым `S2LatLngRect`, который задает прямоугольник в пространстве широта-долгота. + +**Синтаксис** + +``` sql +s2RectIntersection(s2Rect1PointLow, s2Rect1PointHi, s2Rect2PointLow, s2Rect2PointHi) +``` + +**Аргументы** + +- `s2Rect1PointLow`, `s2Rect1PointHi` — значения S2 индекса для самой низкой и самой высокой точек, которые задают первый прямоугольник. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2Rect2PointLow`, `s2Rect2PointHi` — значения S2 индекса для самой низкой и самой высокой точек, которые задают второй прямоугольник. [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Возвращаемые значения** + +- `s2UnionRect2PointLow` — идентификатор нижней ячейки, соответствующей результирующему прямоугольнику. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2UnionRect2PointHi` — идентификатор верхней ячейки, соответствующей результирующему прямоугольнику. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT s2RectIntersection(5178914411069187297, 5177056748191934217, 5179062030687166815, 5177056748191934217) AS rectIntersection; +``` + +Результат: + +``` text +┌─rectIntersection──────────────────────────┐ +│ (5178914411069187297,5177056748191934217) │ +└───────────────────────────────────────────┘ +``` From 339d3ff130bec221452dac27ba7732d071e6cad4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 14:39:05 +0300 Subject: [PATCH 221/396] Fix docs --- .github/workflows/main.yml | 2 +- tests/ci/docs_check.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 36da6c4e40f..a3ce64e0701 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -67,7 +67,7 @@ jobs: path: ${{ runner.temp }}/docs_check - name: Check out repository code uses: actions/checkout@v2 - - name: Style Check + - name: Docs Check env: TEMP_PATH: ${{runner.temp}}/docs_check REPO_COPY: ${{runner.temp}}/docs_check/ClickHouse diff --git a/tests/ci/docs_check.py b/tests/ci/docs_check.py index d8a68d29cd3..10bb2f3a396 100644 --- a/tests/ci/docs_check.py +++ b/tests/ci/docs_check.py @@ -102,7 +102,7 @@ if __name__ == "__main__": if not os.path.exists(test_output): os.makedirs(test_output) - cmd = "docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={test_output}:/test_output {docker_image}" + cmd = f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/repo_path --volume={test_output}:/output_path {docker_image}" run_log_path = os.path.join(test_output, 'runlog.log') From 75de52aa3e0cdb39d0356b588a54f1c0c9705ba4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 14:40:16 +0300 Subject: [PATCH 222/396] Fix padding: --- tests/ci/docs_check.py | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/ci/docs_check.py b/tests/ci/docs_check.py index 10bb2f3a396..b86ab9d9871 100644 --- a/tests/ci/docs_check.py +++ b/tests/ci/docs_check.py @@ -109,14 +109,14 @@ if __name__ == "__main__": with open(run_log_path, 'w', encoding='utf-8') as log: with subprocess.Popen(cmd, shell=True, stderr=log, stdout=log) as process: retcode = process.wait() - if retcode == 0: - logging.info("Run successfully") - status = "Success" - description = "Run Ok" - else: - description = "Run failed (non zero exit code)" - status = "failure" - logging.info("Run failed") + if retcode == 0: + logging.info("Run successfully") + status = "Success" + description = "Run Ok" + else: + description = "Run failed (non zero exit code)" + status = "failure" + logging.info("Run failed") subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) files = os.listdir(test_output) From d095cfe4c10c683b55a656a40b573549036f9d2b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 29 Oct 2021 14:45:09 +0300 Subject: [PATCH 223/396] disable check for zk session uptime by default --- docker/test/fasttest/run.sh | 2 +- docker/test/stateful/run.sh | 2 +- docker/test/stateless/run.sh | 2 +- tests/clickhouse-test | 16 ++++++++++------ 4 files changed, 13 insertions(+), 9 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 2e37522f1b4..925ac6f797c 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -264,7 +264,7 @@ function run_tests set +e time clickhouse-test --hung-check -j 8 --order=random \ - --fast-tests-only --no-long --testname --shard --zookeeper \ + --fast-tests-only --no-long --testname --shard --zookeeper --check-zookeeper-session \ -- "$FASTTEST_FOCUS" 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' \ | tee "$FASTTEST_OUTPUT/test_result.txt" diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 69b435857d9..29be060d255 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -109,7 +109,7 @@ function run_tests() fi set +e - clickhouse-test --testname --shard --zookeeper --no-stateless --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \ + clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --no-stateless --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \ "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt set -e } diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 44e87fb68fd..ad0a805488f 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -97,7 +97,7 @@ function run_tests() fi set +e - clickhouse-test --testname --shard --zookeeper --hung-check --print-time \ + clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' \ | tee -a test_output/test_result.txt diff --git a/tests/clickhouse-test b/tests/clickhouse-test index aaec4497ef6..3fd49477906 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -170,12 +170,15 @@ def get_zookeeper_session_uptime(args): def need_retry(args, stdout, stderr, total_time): - # Sometimes we may get unexpected exception like "Replica is readonly" or "Shutdown is called for table" - # instead of "Session expired" or "Connection loss" - # Retry if session was expired during test execution - session_uptime = get_zookeeper_session_uptime(args) - if session_uptime is not None and session_uptime < math.ceil(total_time): - return True + if args.check_zookeeper_session: + # Sometimes we may get unexpected exception like "Replica is readonly" or "Shutdown is called for table" + # instead of "Session expired" or "Connection loss" + # Retry if session was expired during test execution. + # If ZooKeeper is configured, then it's more reliable than checking stderr, + # but the following condition is always true if ZooKeeper is not configured. + session_uptime = get_zookeeper_session_uptime(args) + if session_uptime is not None and session_uptime < math.ceil(total_time): + return True return any(msg in stdout for msg in MESSAGES_TO_RETRY) or any(msg in stderr for msg in MESSAGES_TO_RETRY) @@ -1320,6 +1323,7 @@ if __name__ == '__main__': parser.add_argument('--no-long', action='store_true', dest='no_long', help='Do not run long tests') parser.add_argument('--client-option', nargs='+', help='Specify additional client argument') parser.add_argument('--print-time', action='store_true', dest='print_time', help='Print test time') + parser.add_argument('--check-zookeeper-session', action='store_true', help='Check ZooKeeper session uptime to determine if failed test should be retried') group = parser.add_mutually_exclusive_group(required=False) group.add_argument('--zookeeper', action='store_true', default=None, dest='zookeeper', help='Run zookeeper related tests') From 0bf1efff97f1332e8b7c819da1dc064fe8861f26 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 29 Oct 2021 19:51:34 +0800 Subject: [PATCH 224/396] Fix build --- src/Interpreters/Context.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 983e2c5afea..0c39070823b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1951,7 +1951,7 @@ void Context::shutdownKeeperDispatcher() const } -void Context::updateKeeperConfiguration(const Poco::Util::AbstractConfiguration & config) +void Context::updateKeeperConfiguration([[maybe_unused]] const Poco::Util::AbstractConfiguration & config) { #if USE_NURAFT std::lock_guard lock(shared->keeper_dispatcher_mutex); From 211c3d4d4965bbc3b5d225d448d21c0f009c17db Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 29 Oct 2021 19:51:41 +0800 Subject: [PATCH 225/396] Better Projection IN --- src/Interpreters/InterpreterSelectQuery.cpp | 16 +++++++++++++--- src/Interpreters/InterpreterSelectQuery.h | 16 ++++++++++++++-- src/Storages/MergeTree/MergeTreeData.cpp | 8 +++++++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 7 +++++++ src/Storages/SelectQueryInfo.h | 4 ++++ .../01710_projection_in_set.reference | 2 ++ .../0_stateless/01710_projection_in_set.sql | 12 ++++++++++++ 7 files changed, 59 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/01710_projection_in_set.reference create mode 100644 tests/queries/0_stateless/01710_projection_in_set.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 0ed7df2d18f..0050df1bf52 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -156,6 +156,15 @@ InterpreterSelectQuery::InterpreterSelectQuery( { } +InterpreterSelectQuery::InterpreterSelectQuery( + const ASTPtr & query_ptr_, + ContextPtr context_, + const SelectQueryOptions & options_, + PreparedSets prepared_sets_) + : InterpreterSelectQuery(query_ptr_, context_, std::nullopt, nullptr, options_, {}, {}, std::move(prepared_sets_)) +{ +} + InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, ContextPtr context_, @@ -258,13 +267,15 @@ InterpreterSelectQuery::InterpreterSelectQuery( const StoragePtr & storage_, const SelectQueryOptions & options_, const Names & required_result_column_names, - const StorageMetadataPtr & metadata_snapshot_) + const StorageMetadataPtr & metadata_snapshot_, + PreparedSets prepared_sets_) /// NOTE: the query almost always should be cloned because it will be modified during analysis. : IInterpreterUnionOrSelectQuery(options_.modify_inplace ? query_ptr_ : query_ptr_->clone(), context_, options_) , storage(storage_) , input_pipe(std::move(input_pipe_)) , log(&Poco::Logger::get("InterpreterSelectQuery")) , metadata_snapshot(metadata_snapshot_) + , prepared_sets(std::move(prepared_sets_)) { checkStackSize(); @@ -354,7 +365,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Reuse already built sets for multiple passes of analysis SubqueriesForSets subquery_for_sets; - PreparedSets prepared_sets; auto analyze = [&] (bool try_move_to_prewhere) { @@ -517,7 +527,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Reuse already built sets for multiple passes of analysis subquery_for_sets = std::move(query_analyzer->getSubqueriesForSets()); - prepared_sets = std::move(query_analyzer->getPreparedSets()); + prepared_sets = query_info.sets.empty() ? std::move(query_analyzer->getPreparedSets()) : std::move(query_info.sets); /// Do not try move conditions to PREWHERE for the second time. /// Otherwise, we won't be able to fallback from inefficient PREWHERE to WHERE later. diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index cf24d14b737..4298cbbb794 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -66,6 +67,13 @@ public: const StorageMetadataPtr & metadata_snapshot_ = nullptr, const SelectQueryOptions & = {}); + /// Read data not from the table specified in the query, but from the specified `storage_`. + InterpreterSelectQuery( + const ASTPtr & query_ptr_, + ContextPtr context_, + const SelectQueryOptions &, + PreparedSets prepared_sets_); + ~InterpreterSelectQuery() override; /// Execute a query. Get the stream of blocks to read. @@ -83,7 +91,7 @@ public: const SelectQueryInfo & getQueryInfo() const { return query_info; } - const SelectQueryExpressionAnalyzer * getQueryAnalyzer() const { return query_analyzer.get(); } + SelectQueryExpressionAnalyzer * getQueryAnalyzer() const { return query_analyzer.get(); } const ExpressionAnalysisResult & getAnalysisResult() const { return analysis_result; } @@ -104,7 +112,8 @@ private: const StoragePtr & storage_, const SelectQueryOptions &, const Names & required_result_column_names = {}, - const StorageMetadataPtr & metadata_snapshot_ = nullptr); + const StorageMetadataPtr & metadata_snapshot_ = nullptr, + PreparedSets prepared_sets_ = {}); ASTSelectQuery & getSelectQuery() { return query_ptr->as(); } @@ -193,6 +202,9 @@ private: Poco::Logger * log; StorageMetadataPtr metadata_snapshot; + + /// Reuse already built sets for multiple passes of analysis, possibly across interpreters. + PreparedSets prepared_sets; }; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index aa3b593bf4f..daa496fa517 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4552,8 +4552,12 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( return false; InterpreterSelectQuery select( - query_ptr, query_context, SelectQueryOptions{QueryProcessingStage::WithMergeableState}.ignoreProjections().ignoreAlias()); + query_ptr, + query_context, + SelectQueryOptions{QueryProcessingStage::WithMergeableState}.ignoreProjections().ignoreAlias(), + query_info.sets /* prepared_sets */); const auto & analysis_result = select.getAnalysisResult(); + query_info.sets = std::move(select.getQueryAnalyzer()->getPreparedSets()); bool can_use_aggregate_projection = true; /// If the first stage of the query pipeline is more complex than Aggregating - Expression - Filter - ReadFromStorage, @@ -4897,6 +4901,8 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( { selected_candidate->aggregation_keys = select.getQueryAnalyzer()->aggregationKeys(); selected_candidate->aggregate_descriptions = select.getQueryAnalyzer()->aggregates(); + selected_candidate->subqueries_for_sets + = std::make_shared(std::move(select.getQueryAnalyzer()->getSubqueriesForSets())); } query_info.projection = std::move(*selected_candidate); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 55de92735c3..106bca97a38 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -374,6 +375,12 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( std::move(pipe), fmt::format("MergeTree(with {} projection {})", query_info.projection->desc->type, query_info.projection->desc->name)); plan->addStep(std::move(step)); + + if (query_info.projection->subqueries_for_sets && !query_info.projection->subqueries_for_sets->empty()) + { + SizeLimits limits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode); + addCreatingSetsStep(*plan, std::move(*query_info.projection->subqueries_for_sets), limits, context); + } return plan; } diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index fe7b22d331b..e53f5adec52 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -42,6 +42,9 @@ using ClusterPtr = std::shared_ptr; struct MergeTreeDataSelectAnalysisResult; using MergeTreeDataSelectAnalysisResultPtr = std::shared_ptr; +struct SubqueryForSet; +using SubqueriesForSets = std::unordered_map; + struct PrewhereInfo { /// Actions which are executed in order to alias columns are used for prewhere actions. @@ -121,6 +124,7 @@ struct ProjectionCandidate ReadInOrderOptimizerPtr order_optimizer; InputOrderInfoPtr input_order_info; ManyExpressionActions group_by_elements_actions; + std::shared_ptr subqueries_for_sets; MergeTreeDataSelectAnalysisResultPtr merge_tree_projection_select_result_ptr; MergeTreeDataSelectAnalysisResultPtr merge_tree_normal_select_result_ptr; }; diff --git a/tests/queries/0_stateless/01710_projection_in_set.reference b/tests/queries/0_stateless/01710_projection_in_set.reference new file mode 100644 index 00000000000..f6705ccafc4 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_in_set.reference @@ -0,0 +1,2 @@ +2 3 +2 3 diff --git a/tests/queries/0_stateless/01710_projection_in_set.sql b/tests/queries/0_stateless/01710_projection_in_set.sql new file mode 100644 index 00000000000..99fa2cab0c5 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_in_set.sql @@ -0,0 +1,12 @@ +drop table if exists x; +create table x (i UInt64, j UInt64, k UInt64, projection agg (select sum(j), avg(k) group by i), projection norm (select j, k order by i)) engine MergeTree order by tuple(); + +insert into x values (1, 2, 3); + +set allow_experimental_projection_optimization = 1, use_index_for_in_with_subqueries = 0; + +select sum(j), avg(k) from x where i in (select number from numbers(4)); + +select j, k from x where i in (select number from numbers(4)); + +drop table x; From 6f12348282c90a343b8ec9706501d08c88e12b92 Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 29 Oct 2021 12:31:18 +0000 Subject: [PATCH 226/396] enable modify table comment of some table --- src/Storages/StorageBuffer.cpp | 7 +++---- src/Storages/StorageDistributed.cpp | 8 +++----- src/Storages/StorageMerge.cpp | 7 +++---- 3 files changed, 9 insertions(+), 13 deletions(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 12eae0e82a2..6417aa9f72c 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -1027,10 +1027,9 @@ void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, Context auto name_deps = getDependentViewsByColumn(local_context); for (const auto & command : commands) { - if (command.type != AlterCommand::Type::ADD_COLUMN - && command.type != AlterCommand::Type::MODIFY_COLUMN - && command.type != AlterCommand::Type::DROP_COLUMN - && command.type != AlterCommand::Type::COMMENT_COLUMN) + if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN + && command.type != AlterCommand::Type::DROP_COLUMN && command.type != AlterCommand::Type::COMMENT_COLUMN + && command.type != AlterCommand::Type::COMMENT_TABLE) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}", command.type, getName()); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 2c528a4ebf5..91d54f0d24a 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -762,11 +762,9 @@ void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, Co auto name_deps = getDependentViewsByColumn(local_context); for (const auto & command : commands) { - if (command.type != AlterCommand::Type::ADD_COLUMN - && command.type != AlterCommand::Type::MODIFY_COLUMN - && command.type != AlterCommand::Type::DROP_COLUMN - && command.type != AlterCommand::Type::COMMENT_COLUMN - && command.type != AlterCommand::Type::RENAME_COLUMN) + if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN + && command.type != AlterCommand::Type::DROP_COLUMN && command.type != AlterCommand::Type::COMMENT_COLUMN + && command.type != AlterCommand::Type::RENAME_COLUMN && command.type != AlterCommand::Type::COMMENT_TABLE) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}", command.type, getName()); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 979e027d338..06fbfafc9b6 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -629,10 +629,9 @@ void StorageMerge::checkAlterIsPossible(const AlterCommands & commands, ContextP auto name_deps = getDependentViewsByColumn(local_context); for (const auto & command : commands) { - if (command.type != AlterCommand::Type::ADD_COLUMN - && command.type != AlterCommand::Type::MODIFY_COLUMN - && command.type != AlterCommand::Type::DROP_COLUMN - && command.type != AlterCommand::Type::COMMENT_COLUMN) + if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN + && command.type != AlterCommand::Type::DROP_COLUMN && command.type != AlterCommand::Type::COMMENT_COLUMN + && command.type != AlterCommand::Type::COMMENT_TABLE) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}", command.type, getName()); From b49797ad518d35944d77726ba70c12af21c040de Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 29 Oct 2021 15:54:40 +0300 Subject: [PATCH 227/396] Update ReadBufferFromS3.cpp --- src/IO/ReadBufferFromS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 3ccedaba8db..78d296be60e 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -170,7 +170,7 @@ std::unique_ptr ReadBufferFromS3::initialize() req.SetKey(key); /** - * If remote_filesustem_method = 'read_threadpool', then for MergeTree family tables + * If remote_filesystem_read_method = 'read_threadpool', then for MergeTree family tables * exact byte ranges to read are always passed here. */ if (read_until_position) From 2f26b8777a2f62532fd8a50dbc28feee5aeb6da4 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 29 Oct 2021 16:23:29 +0300 Subject: [PATCH 228/396] [website] update logo order (#30855) --- website/templates/company/investors.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/templates/company/investors.html b/website/templates/company/investors.html index cf7b3631060..fbe15ccd8e9 100644 --- a/website/templates/company/investors.html +++ b/website/templates/company/investors.html @@ -55,10 +55,10 @@
+ -
From 821300342268168f7721ecf3a327e91834491852 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 29 Oct 2021 16:51:57 +0300 Subject: [PATCH 229/396] Fix PVS check, mark tests as no-parallel --- .../Serializations/SerializationNullable.cpp | 40 +++++++++---------- ...2103_tsv_csv_custom_null_representation.sh | 1 + .../02104_json_strings_nullable_string.sh | 1 + 3 files changed, 22 insertions(+), 20 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index 0dbb140af8b..7bf7b6e55b0 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -257,7 +257,7 @@ void SerializationNullable::deserializeTextEscaped(IColumn & column, ReadBuffer template ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, - const SerializationPtr & nested) + const SerializationPtr & nested_serialization) { const String & null_representation = settings.tsv.null_representation; @@ -265,9 +265,9 @@ ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, R if (istr.eof() || (!null_representation.empty() && *istr.position() != null_representation[0])) { /// This is not null, surely. - return safeDeserialize(column, *nested, + return safeDeserialize(column, *nested_serialization, [] { return false; }, - [&nested, &istr, &settings] (IColumn & nested_column) { nested->deserializeTextEscaped(nested_column, istr, settings); }); + [&nested_serialization, &istr, &settings] (IColumn & nested_column) { nested_serialization->deserializeTextEscaped(nested_column, istr, settings); }); } /// Check if we have enough data in buffer to check if it's a null. @@ -281,11 +281,11 @@ ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, R istr.position() = pos; return false; }; - auto deserialize_nested = [&nested, &settings, &istr] (IColumn & nested_column) + auto deserialize_nested = [&nested_serialization, &settings, &istr] (IColumn & nested_column) { - nested->deserializeTextEscaped(nested_column, istr, settings); + nested_serialization->deserializeTextEscaped(nested_column, istr, settings); }; - return safeDeserialize(column, *nested, check_for_null, deserialize_nested); + return safeDeserialize(column, *nested_serialization, check_for_null, deserialize_nested); } /// We don't have enough data in buffer to check if it's a null. @@ -303,10 +303,10 @@ ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, R return false; }; - auto deserialize_nested = [&nested, &settings, &buf, &null_representation, &istr] (IColumn & nested_column) + auto deserialize_nested = [&nested_serialization, &settings, &buf, &null_representation, &istr] (IColumn & nested_column) { auto * pos = buf.position(); - nested->deserializeTextEscaped(nested_column, buf, settings); + nested_serialization->deserializeTextEscaped(nested_column, buf, settings); /// Check that we don't have any unread data in PeekableReadBuffer own memory. if (likely(!buf.hasUnreadData())) return; @@ -320,7 +320,7 @@ ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, R "for large input.", ErrorCodes::CANNOT_READ_ALL_DATA); WriteBufferFromOwnString parsed_value; - nested->serializeTextEscaped(nested_column, nested_column.size() - 1, parsed_value, settings); + nested_serialization->serializeTextEscaped(nested_column, nested_column.size() - 1, parsed_value, settings); throw DB::ParsingException("Error while parsing \"" + std::string(pos, buf.buffer().end()) + std::string(istr.position(), std::min(size_t(10), istr.available())) + "\" as Nullable" + " at position " + std::to_string(istr.count()) + ": got \"" + std::string(pos, buf.position() - pos) + "\", which was deserialized as \"" @@ -328,7 +328,7 @@ ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, R ErrorCodes::CANNOT_READ_ALL_DATA); }; - return safeDeserialize(column, *nested, check_for_null, deserialize_nested); + return safeDeserialize(column, *nested_serialization, check_for_null, deserialize_nested); } void SerializationNullable::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -413,15 +413,15 @@ void SerializationNullable::deserializeTextCSV(IColumn & column, ReadBuffer & is template ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, - const SerializationPtr & nested) + const SerializationPtr & nested_serialization) { const String & null_representation = settings.csv.null_representation; if (istr.eof() || (!null_representation.empty() && *istr.position() != null_representation[0])) { /// This is not null, surely. - return safeDeserialize(column, *nested, + return safeDeserialize(column, *nested_serialization, [] { return false; }, - [&nested, &istr, &settings] (IColumn & nested_column) { nested->deserializeTextCSV(nested_column, istr, settings); }); + [&nested_serialization, &istr, &settings] (IColumn & nested_column) { nested_serialization->deserializeTextCSV(nested_column, istr, settings); }); } /// Check if we have enough data in buffer to check if it's a null. @@ -435,11 +435,11 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB istr.position() = pos; return false; }; - auto deserialize_nested = [&nested, &settings, &istr] (IColumn & nested_column) + auto deserialize_nested = [&nested_serialization, &settings, &istr] (IColumn & nested_column) { - nested->deserializeTextCSV(nested_column, istr, settings); + nested_serialization->deserializeTextCSV(nested_column, istr, settings); }; - return safeDeserialize(column, *nested, check_for_null, deserialize_nested); + return safeDeserialize(column, *nested_serialization, check_for_null, deserialize_nested); } /// We don't have enough data in buffer to check if it's a null. @@ -457,10 +457,10 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB return false; }; - auto deserialize_nested = [&nested, &settings, &buf, &null_representation, &istr] (IColumn & nested_column) + auto deserialize_nested = [&nested_serialization, &settings, &buf, &null_representation, &istr] (IColumn & nested_column) { auto * pos = buf.position(); - nested->deserializeTextCSV(nested_column, buf, settings); + nested_serialization->deserializeTextCSV(nested_column, buf, settings); /// Check that we don't have any unread data in PeekableReadBuffer own memory. if (likely(!buf.hasUnreadData())) return; @@ -475,7 +475,7 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB "for large input.", ErrorCodes::CANNOT_READ_ALL_DATA); WriteBufferFromOwnString parsed_value; - nested->serializeTextCSV(nested_column, nested_column.size() - 1, parsed_value, settings); + nested_serialization->serializeTextCSV(nested_column, nested_column.size() - 1, parsed_value, settings); throw DB::ParsingException("Error while parsing \"" + std::string(pos, buf.buffer().end()) + std::string(istr.position(), std::min(size_t(10), istr.available())) + "\" as Nullable" + " at position " + std::to_string(istr.count()) + ": got \"" + std::string(pos, buf.position() - pos) + "\", which was deserialized as \"" @@ -483,7 +483,7 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB ErrorCodes::CANNOT_READ_ALL_DATA); }; - return safeDeserialize(column, *nested, check_for_null, deserialize_nested); + return safeDeserialize(column, *nested_serialization, check_for_null, deserialize_nested); } void SerializationNullable::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const diff --git a/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.sh b/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.sh index 676e8cb867f..4162e046ca4 100755 --- a/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.sh +++ b/tests/queries/0_stateless/02103_tsv_csv_custom_null_representation.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/02104_json_strings_nullable_string.sh b/tests/queries/0_stateless/02104_json_strings_nullable_string.sh index 5385c1282b2..6a5d369e7b6 100755 --- a/tests/queries/0_stateless/02104_json_strings_nullable_string.sh +++ b/tests/queries/0_stateless/02104_json_strings_nullable_string.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 1d9415f68c4d708fd540dbf4a6853b4c0673ecff Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 16:57:47 +0300 Subject: [PATCH 230/396] Fixes in docs --- docker/docs/builder/Dockerfile | 7 +------ docker/docs/check/run.sh | 5 ++++- tests/ci/docs_check.py | 4 +++- tests/ci/pr_info.py | 3 ++- 4 files changed, 10 insertions(+), 9 deletions(-) diff --git a/docker/docs/builder/Dockerfile b/docker/docs/builder/Dockerfile index 0b0f4ef71b5..c47554808cb 100644 --- a/docker/docs/builder/Dockerfile +++ b/docker/docs/builder/Dockerfile @@ -39,9 +39,4 @@ RUN wget 'https://github.com/wkhtmltopdf/packaging/releases/download/0.12.6-1/wk 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 +RUN pip3 install --ignore-installed --upgrade setuptools pip virtualenv diff --git a/docker/docs/check/run.sh b/docker/docs/check/run.sh index 57f39afec27..f70f82aeb4c 100644 --- a/docker/docs/check/run.sh +++ b/docker/docs/check/run.sh @@ -1,6 +1,9 @@ #!/usr/bin/env bash set -euo pipefail - cd $REPO_PATH/docs/tools +mkdir venv +virtualenv -p $(which python3) venv +source venv/bin/activate +python3 -m pip install --ignore-installed -r requirements.txt ./build.py --skip-git-log 2>&1 | tee $OUTPUT_PATH/output.log diff --git a/tests/ci/docs_check.py b/tests/ci/docs_check.py index b86ab9d9871..cf1ded72488 100644 --- a/tests/ci/docs_check.py +++ b/tests/ci/docs_check.py @@ -137,8 +137,10 @@ if __name__ == "__main__": if lines: status = "failure" description = "Found errors during docs check" - else: + elif status != "failure": lines.append(("No errors found", "OK")) + else: + lines.append(("Non zero exit code", "FAIL")) s3_helper = S3Helper('https://s3.amazonaws.com') diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 672043bd7b7..377c03a9f4c 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -1,8 +1,9 @@ #!/usr/bin/env python3 +import os import urllib + import requests from unidiff import PatchSet -import os DIFF_IN_DOCUMENTATION_EXT = [".html", ".md", ".yml", ".txt", ".css", ".js", ".xml", ".ico", ".conf", ".svg", ".png", ".jpg", ".py", ".sh"] From 3ac3f3b03538afd34a611f5b58d70560ffa05539 Mon Sep 17 00:00:00 2001 From: olgarev Date: Fri, 29 Oct 2021 14:57:53 +0000 Subject: [PATCH 231/396] Initial --- docs/en/operations/settings/settings.md | 26 +++++++++++++++++++++++++ docs/ru/operations/settings/settings.md | 26 +++++++++++++++++++++++++ 2 files changed, 52 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 46806d7d101..043ceebff0d 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1397,6 +1397,32 @@ Minimum count of executing same expression before it is get compiled. Default value: `3`. +## compile_aggregate_expressions {#compile_aggregate_expressions} + +Enables or disables JIT-compilation of aggregate functions to native code. Enabling the `compile_aggregate_expressions` setting can improve the performance. + +Possible values: + +- 0 — Aggregation is done without JIT compilation. +- 1 — Aggregation is done using JIT compilation. + +Default value: `0`. + +**See Also** + +- [min_count_to_compile_aggregate_expression](#min_count_to_compile_aggregate_expression) + +## min_count_to_compile_aggregate_expression {#min_count_to_compile_aggregate_expression} + +If the [compile_aggregate_expressions](#compile_aggregate_expressions) setting is enabled and there are more then `min_count_to_compile_aggregate_expression` identical aggregate expressions in a `SELECT` query, then they are compiled JIT. + +Possible values: + +- Positive integer. +- 0 — Identical aggregate expressions are always JIT-compiled (if `compile_aggregate_expressions = 1`). + +Default value: `0`. + ## output_format_json_quote_64bit_integers {#session_settings-output_format_json_quote_64bit_integers} Controls quoting of 64-bit or bigger [integers](../../sql-reference/data-types/int-uint.md) (like `UInt64` or `Int128`) when they are output in a [JSON](../../interfaces/formats.md#json) format. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 70dc0e349e4..e6f683b4991 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1361,6 +1361,32 @@ load_balancing = round_robin Значение по умолчанию: `3`. +## compile_aggregate_expressions {#compile_aggregate_expressions} + +Включает или отключает компиляцию агрегированных функций в нативный код во время выполнения запроса. Включение настройки `compile_aggregate_expressions` может улучшить производительность выполнения запросов. + +Возможные значения: + +- 0 — агрегированные функции не компилируются в нативный код. +- 1 — агрегированные функции компилируются в нативный код в процессе выполнения запроса. + +Значение по умолчанию: `0`. + +**См. также** + +- [min_count_to_compile_aggregate_expression](#min_count_to_compile_aggregate_expression) + +## min_count_to_compile_aggregate_expression {#min_count_to_compile_aggregate_expression} + +Если настройка [compile_aggregate_expressions](#compile_aggregate_expressions) включена и в ходе выполнения одного запроса агрегированная функция выполняется более `min_count_to_compile_aggregate_expression` раз для одного и того же выражения, тогда эта агрегированная функция компилируется в нативный код. + +Возможные значения: + +- Целое положительное число. +- 0 — агрегированные функциии для одинаковых выражений всегда компилируются в ходе выполнения запроса (если `compile_aggregate_expressions = 1`). + +Значение по умолчанию: `0`. + ## input_format_skip_unknown_fields {#input-format-skip-unknown-fields} Если значение равно true, то при выполнении INSERT входные данные из столбцов с неизвестными именами будут пропущены. В противном случае эта ситуация создаст исключение. From 8d779317d1065b1540a6fd9fd8023f9c979f91c8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 18:01:29 +0300 Subject: [PATCH 232/396] Add docs release --- .github/workflows/main.yml | 2 +- .github/workflows/release.yml | 47 ++++++++++ docker/docs/release/Dockerfile | 9 ++ docker/docs/release/run.sh | 9 ++ docker/images.json | 10 ++- tests/ci/docker_images_check.py | 3 +- tests/ci/docs_check.py | 2 +- tests/ci/docs_release.py | 153 ++++++++++++++++++++++++++++++++ tests/ci/pr_info.py | 52 +++++++---- 9 files changed, 263 insertions(+), 24 deletions(-) create mode 100644 .github/workflows/release.yml create mode 100644 docker/docs/release/Dockerfile create mode 100644 docker/docs/release/run.sh create mode 100644 tests/ci/docs_release.py diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index a3ce64e0701..a224f49244d 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -58,7 +58,7 @@ jobs: sudo rm -fr $TEMP_PATH DocsCheck: needs: DockerHubPush - runs-on: [self-hosted, style-checker] + runs-on: [self-hosted, func-tester] steps: - name: Download changed images uses: actions/download-artifact@v2 diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml new file mode 100644 index 00000000000..dbd945ce427 --- /dev/null +++ b/.github/workflows/release.yml @@ -0,0 +1,47 @@ +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) ||: + sudo rm -fr $TEMP_PATH diff --git a/docker/docs/release/Dockerfile b/docker/docs/release/Dockerfile new file mode 100644 index 00000000000..63765180a4c --- /dev/null +++ b/docker/docs/release/Dockerfile @@ -0,0 +1,9 @@ +# docker build -t clickhouse/docs-release . +FROM clickhouse/docs-builder + +COPY run.sh / + +ENV REPO_PATH=/repo_path +ENV OUTPUT_PATH=/output_path + +CMD ["/bin/bash", "/run.sh"] diff --git a/docker/docs/release/run.sh b/docker/docs/release/run.sh new file mode 100644 index 00000000000..3ecfd26cb44 --- /dev/null +++ b/docker/docs/release/run.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +set -euo pipefail + +cd $REPO_PATH/docs/tools +mkdir venv +virtualenv -p $(which python3) venv +source venv/bin/activate +python3 -m pip install --ignore-installed -r requirements.txt +./release.sh 2>&1 | tee tee $OUTPUT_PATH/output.log diff --git a/docker/images.json b/docker/images.json index adbaf794e71..a6cc821108e 100644 --- a/docker/images.json +++ b/docker/images.json @@ -169,11 +169,17 @@ }, "docker/docs/builder": { "name": "clickhouse/docs-builder", - "dependent": ["docker/docs/check"] + "dependent": [ + "docker/docs/check", + "docker/docs/release" + ] }, "docker/docs/check": { "name": "clickhouse/docs-check", "dependent": [] + }, + "docker/docs/release": { + "name": "clickhouse/docs-release", + "dependent": [] } - } diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 470dcc18233..d874ca422c3 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -193,8 +193,9 @@ if __name__ == "__main__": changed_images, dockerhub_repo_name = get_changed_docker_images(pr_info, repo_path, "docker/images.json") logging.info("Has changed images %s", ', '.join([str(image[0]) for image in changed_images])) pr_commit_version = str(pr_info.number) + '-' + pr_info.sha - versions = [str(pr_info.number), pr_commit_version] + if pr_info.number == 0: + versions.append("latest") subprocess.check_output("docker login --username 'robotclickhouse' --password '{}'".format(dockerhub_password), shell=True) diff --git a/tests/ci/docs_check.py b/tests/ci/docs_check.py index cf1ded72488..a99641537c5 100644 --- a/tests/ci/docs_check.py +++ b/tests/ci/docs_check.py @@ -111,7 +111,7 @@ if __name__ == "__main__": retcode = process.wait() if retcode == 0: logging.info("Run successfully") - status = "Success" + status = "success" description = "Run Ok" else: description = "Run failed (non zero exit code)" diff --git a/tests/ci/docs_release.py b/tests/ci/docs_release.py new file mode 100644 index 00000000000..be6915f3cc9 --- /dev/null +++ b/tests/ci/docs_release.py @@ -0,0 +1,153 @@ +#!/usr/bin/env python3 + +#!/usr/bin/env python3 +import logging +import subprocess +import os +import time +import json +import sys +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 + +NAME = "Docs Release (actions)" + +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, additional_files): + s3_path_prefix = f"{pr_number}/{commit_sha}/docs_release" + 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(NAME, test_results, raw_log_url, task_url, branch_url, branch_name, commit_url, additional_urls) + 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 + +if __name__ == "__main__": + logging.basicConfig(level=logging.INFO) + + temp_path = os.path.join(os.getenv("TEMP_PATH")) + repo_path = os.path.join(os.getenv("REPO_COPY")) + + with open(os.getenv('GITHUB_EVENT_PATH'), 'r', encoding='utf-8') as event_file: + event = json.load(event_file) + + pr_info = PRInfo(event, need_changed_files=True) + + gh = Github(get_best_robot_token()) + if not pr_info.has_changes_in_documentation(): + logging.info ("No changes in documentation") + commit = get_commit(gh, pr_info.sha) + commit.create_status(context=NAME, description="No changes in docs", state="success") + sys.exit(0) + + logging.info("Has changes in docs") + + if not os.path.exists(temp_path): + os.makedirs(temp_path) + + images_path = os.path.join(temp_path, 'changed_images.json') + + docker_image = 'clickhouse/docs-release' + if 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 'clickhouse/docs-release' in images: + docker_image += ':' + images['clickhouse/docs-release'] + + logging.info("Got docker image %s", docker_image) + for i in range(10): + try: + subprocess.check_output(f"docker pull {docker_image}", 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_image}") + + test_output = os.path.join(temp_path, 'docs_release_log') + if not os.path.exists(test_output): + os.makedirs(test_output) + + token = get_parameter_from_ssm('cloudflare_token', decrypt=True) + 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: + with subprocess.Popen(cmd, shell=True, stderr=log, stdout=log) as process: + retcode = process.wait() + if retcode == 0: + logging.info("Run successfully") + status = "success" + description = "Run Ok" + else: + description = "Run failed (non zero exit code)" + status = "failure" + logging.info("Run failed") + + subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) + files = os.listdir(test_output) + lines = [] + additional_files = [] + if not files: + logging.error("No output files after docs release") + description = "No output files after docs release" + status = "failure" + else: + for f in files: + path = os.path.join(test_output, f) + additional_files.append(path) + with open(path, 'r', encoding='utf-8') as check_file: + for line in check_file: + if "ERROR" in line: + lines.append((line.split(':')[-1], "FAIL")) + if lines: + status = "failure" + description = "Found errors during docs release" + elif status != "failure": + lines.append(("No errors found", "OK")) + else: + lines.append(("Non zero exit code", "FAIL")) + + s3_helper = S3Helper('https://s3.amazonaws.com') + + 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 377c03a9f4c..14a97e510a2 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -10,27 +10,41 @@ 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): - self.number = github_event['number'] - if 'after' in github_event: + if 'pull_request' in github_event: # pull request and other similar events + self.number = github_event['number'] + if 'after' in github_event: + self.sha = github_event['after'] + else: + self.sha = github_event['pull_request']['head']['sha'] + + self.labels = { l['name'] for l in github_event['pull_request']['labels'] } + self.user_login = github_event['pull_request']['user']['login'] + self.user_orgs = set([]) + if need_orgs: + user_orgs_response = requests.get(github_event['pull_request']['user']['organizations_url']) + if user_orgs_response.ok: + response_json = user_orgs_response.json() + self.user_orgs = set(org['id'] for org in response_json) + + self.changed_files = set([]) + if need_changed_files: + diff_url = github_event['pull_request']['diff_url'] + 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 + self.number = 0 self.sha = github_event['after'] + 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}') + if 'files' in diff: + self.changed_files = [f['filename'] for f in diff['files']] + else: + self.changed_files = set([]) else: - self.sha = github_event['pull_request']['head']['sha'] - - self.labels = { l['name'] for l in github_event['pull_request']['labels'] } - self.user_login = github_event['pull_request']['user']['login'] - self.user_orgs = set([]) - if need_orgs: - user_orgs_response = requests.get(github_event['pull_request']['user']['organizations_url']) - if user_orgs_response.ok: - response_json = user_orgs_response.json() - self.user_orgs = set(org['id'] for org in response_json) - - self.changed_files = set([]) - if need_changed_files: - diff_url = github_event['pull_request']['diff_url'] - 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 } + raise Exception("Unknown event type") def get_dict(self): return { From 833ab0cbf896147aa27cc2ac5b387318aedea130 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Fri, 29 Oct 2021 19:09:19 +0300 Subject: [PATCH 233/396] Update docs/en/operations/system-tables/columns.md Co-authored-by: tavplubix --- docs/en/operations/system-tables/columns.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/columns.md b/docs/en/operations/system-tables/columns.md index d9f1ea18196..5d93aef5859 100644 --- a/docs/en/operations/system-tables/columns.md +++ b/docs/en/operations/system-tables/columns.md @@ -24,7 +24,7 @@ Columns: - `is_in_primary_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the primary key expression. - `is_in_sampling_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the sampling key expression. - `compression_codec` ([String](../../sql-reference/data-types/string.md)) — Compression codec name. -- `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. Otherwise, the `NULL` value is returned. +- `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. In ClickHouse makes sense only for `FixedString` data type. Otherwise, the `NULL` value is returned. - `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. Otherwise, the `NULL` value is returned. - `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. Otherwise, the `NULL` value is returned. - `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. Otherwise, the `NULL` value is returned. From 761a488c482d0831fb11e5caa6f98fdfe2dc368b Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Fri, 29 Oct 2021 19:09:36 +0300 Subject: [PATCH 234/396] Update docs/en/operations/system-tables/columns.md Co-authored-by: tavplubix --- docs/en/operations/system-tables/columns.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/columns.md b/docs/en/operations/system-tables/columns.md index 5d93aef5859..d69a5642636 100644 --- a/docs/en/operations/system-tables/columns.md +++ b/docs/en/operations/system-tables/columns.md @@ -25,7 +25,7 @@ Columns: - `is_in_sampling_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the sampling key expression. - `compression_codec` ([String](../../sql-reference/data-types/string.md)) — Compression codec name. - `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. In ClickHouse makes sense only for `FixedString` data type. Otherwise, the `NULL` value is returned. -- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. Otherwise, the `NULL` value is returned. +- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it's bitness for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned. - `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. Otherwise, the `NULL` value is returned. - `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. Otherwise, the `NULL` value is returned. - `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Subtype code for `DateTime` and `ISO` interval data types. For other data types, the `NULL` value is returned. From 50fc8f4d0da482e12b37080daf46df0b7594db11 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Fri, 29 Oct 2021 19:09:53 +0300 Subject: [PATCH 235/396] Update docs/en/operations/system-tables/columns.md Co-authored-by: tavplubix --- docs/en/operations/system-tables/columns.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/columns.md b/docs/en/operations/system-tables/columns.md index d69a5642636..418641a5311 100644 --- a/docs/en/operations/system-tables/columns.md +++ b/docs/en/operations/system-tables/columns.md @@ -26,7 +26,7 @@ Columns: - `compression_codec` ([String](../../sql-reference/data-types/string.md)) — Compression codec name. - `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. In ClickHouse makes sense only for `FixedString` data type. Otherwise, the `NULL` value is returned. - `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it's bitness for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned. -- `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. Otherwise, the `NULL` value is returned. +- `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. In ClickHouse it's 2 for integer types and 10 for `Decimal` types. Otherwise, the `NULL` value is returned. - `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. Otherwise, the `NULL` value is returned. - `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Subtype code for `DateTime` and `ISO` interval data types. For other data types, the `NULL` value is returned. From 84ea9056fc4bf7f242b0c77701b19b870c0cdfc5 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Fri, 29 Oct 2021 19:10:12 +0300 Subject: [PATCH 236/396] Update docs/en/operations/system-tables/columns.md Co-authored-by: tavplubix --- docs/en/operations/system-tables/columns.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/columns.md b/docs/en/operations/system-tables/columns.md index 418641a5311..e271b2c0b93 100644 --- a/docs/en/operations/system-tables/columns.md +++ b/docs/en/operations/system-tables/columns.md @@ -27,7 +27,7 @@ Columns: - `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. In ClickHouse makes sense only for `FixedString` data type. Otherwise, the `NULL` value is returned. - `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it's bitness for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned. - `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. In ClickHouse it's 2 for integer types and 10 for `Decimal` types. Otherwise, the `NULL` value is returned. -- `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. Otherwise, the `NULL` value is returned. +- `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse makes sense only for `Decimal` types. Otherwise, the `NULL` value is returned. - `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Subtype code for `DateTime` and `ISO` interval data types. For other data types, the `NULL` value is returned. **Example** From cadbb74bab2ee9e8dd5bb93c6a68ded9d894ab78 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Fri, 29 Oct 2021 19:10:28 +0300 Subject: [PATCH 237/396] Update docs/en/operations/system-tables/columns.md Co-authored-by: tavplubix --- docs/en/operations/system-tables/columns.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/columns.md b/docs/en/operations/system-tables/columns.md index e271b2c0b93..8b420eb7894 100644 --- a/docs/en/operations/system-tables/columns.md +++ b/docs/en/operations/system-tables/columns.md @@ -28,7 +28,7 @@ Columns: - `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it's bitness for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned. - `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. In ClickHouse it's 2 for integer types and 10 for `Decimal` types. Otherwise, the `NULL` value is returned. - `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse makes sense only for `Decimal` types. Otherwise, the `NULL` value is returned. -- `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Subtype code for `DateTime` and `ISO` interval data types. For other data types, the `NULL` value is returned. +- `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Decimal precision of `DateTime64` data type. For other data types, the `NULL` value is returned. **Example** From 5b799a7821a773c5cdf9dc06e8c291b43a886723 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Fri, 29 Oct 2021 19:10:37 +0300 Subject: [PATCH 238/396] Update docs/en/operations/system-tables/information_schema.md Co-authored-by: tavplubix --- docs/en/operations/system-tables/information_schema.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/information_schema.md b/docs/en/operations/system-tables/information_schema.md index dfdb9df5dfd..46aec078af9 100644 --- a/docs/en/operations/system-tables/information_schema.md +++ b/docs/en/operations/system-tables/information_schema.md @@ -31,7 +31,7 @@ Contains columns read from the `system.columns` system table and columns that ar Columns: - `table_catalog` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the table is located. -- `table_schema` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the schema is located. +- `table_schema` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the table is located. - `table_name` ([String](../../sql-reference/data-types/string.md)) — Table name. - `column_name` ([String](../../sql-reference/data-types/string.md)) — Column name. - `ordinal_position` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Ordinal position of a column in a table starting with 1. From 69dca0bac031b07b9264968909805cda4ab494e1 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Fri, 29 Oct 2021 19:10:55 +0300 Subject: [PATCH 239/396] Update docs/en/operations/system-tables/information_schema.md Co-authored-by: tavplubix --- docs/en/operations/system-tables/information_schema.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/system-tables/information_schema.md b/docs/en/operations/system-tables/information_schema.md index 46aec078af9..129d1dd1600 100644 --- a/docs/en/operations/system-tables/information_schema.md +++ b/docs/en/operations/system-tables/information_schema.md @@ -98,8 +98,8 @@ Contains columns read from the `system.databases` system table and columns that Columns: -- `catalog_name` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the table is located. -- `schema_name` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the schema is located. +- `catalog_name` ([String](../../sql-reference/data-types/string.md)) — The name of the database. +- `schema_name` ([String](../../sql-reference/data-types/string.md)) — The name of the database. - `schema_owner` ([String](../../sql-reference/data-types/string.md)) — Schema owner name, always `'default'`. - `default_character_set_catalog` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. - `default_character_set_schema` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. From e6056fad8d8ec41277ef9addb071cd6ccbe6b583 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Fri, 29 Oct 2021 19:11:28 +0300 Subject: [PATCH 240/396] Update docs/en/operations/system-tables/information_schema.md Co-authored-by: tavplubix --- docs/en/operations/system-tables/information_schema.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/information_schema.md b/docs/en/operations/system-tables/information_schema.md index 129d1dd1600..49aa2fa0337 100644 --- a/docs/en/operations/system-tables/information_schema.md +++ b/docs/en/operations/system-tables/information_schema.md @@ -170,7 +170,7 @@ Contains columns read from the `system.tables` system table, when the table engi Columns: - `table_catalog` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the table is located. -- `table_schema` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the schema is located. +- `table_schema` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the table is located. - `table_name` ([String](../../sql-reference/data-types/string.md)) — Table name. - `view_definition` ([String](../../sql-reference/data-types/string.md)) — `SELECT` query for view. - `check_option` ([String](../../sql-reference/data-types/string.md)) — `NONE`, no checking. From 41c2c24b61b5bbf991986e6efd596cddd0f6421d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 29 Oct 2021 19:18:36 +0300 Subject: [PATCH 241/396] Fix protocol revision. --- src/Core/ProtocolDefines.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 289917f4bdb..8a31ffd49b1 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -12,7 +12,7 @@ /// 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 54456 +#define DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD 54448 #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 From 05ee60ded044b0453a32c9db0554b9b84a1f53dc Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 29 Oct 2021 19:37:56 +0300 Subject: [PATCH 242/396] [docs] update email (#30865) --- docs/en/operations/performance-test.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/performance-test.md b/docs/en/operations/performance-test.md index 2880793962a..a220575cb3c 100644 --- a/docs/en/operations/performance-test.md +++ b/docs/en/operations/performance-test.md @@ -23,7 +23,7 @@ chmod a+x ./hardware.sh ./hardware.sh ``` -3. Copy the output and send it to clickhouse-feedback@yandex-team.com +3. Copy the output and send it to feedback@clickhouse.com All the results are published here: https://clickhouse.com/benchmark/hardware/ From 75a4556067cca85c233818ee1e1d8877d579223d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 29 Oct 2021 19:41:47 +0300 Subject: [PATCH 243/396] Update adopters.md --- docs/en/introduction/adopters.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 787cd451d96..f04d4102138 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -171,5 +171,6 @@ toc_title: Adopters | Цифровой Рабочий | Industrial IoT, Analytics | — | — | — | [Blog post in Russian, March 2021](https://habr.com/en/company/croc/blog/548018/) | | ООО «МПЗ Богородский» | Agriculture | — | — | — | [Article in Russian, November 2020](https://cloud.yandex.ru/cases/okraina) | | ДомКлик | Real Estate | — | — | — | [Article in Russian, October 2021](https://habr.com/ru/company/domclick/blog/585936/) | +| Deepl | Machine Learning | — | — | — | [Video, October 2021](https://www.youtube.com/watch?v=WIYJiPwxXdM&t=1182s) | [Original article](https://clickhouse.com/docs/en/introduction/adopters/) From 3933cd8259b2babab0360644cee8113f7636c422 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 29 Oct 2021 19:47:57 +0300 Subject: [PATCH 244/396] support alter table for Memory database --- src/Databases/DatabaseMemory.cpp | 12 +++- src/Databases/DatabaseMemory.h | 2 + src/Databases/DatabaseOnDisk.cpp | 62 ------------------ src/Databases/DatabaseOnDisk.h | 2 - src/Databases/DatabasesCommon.cpp | 63 +++++++++++++++++++ src/Databases/DatabasesCommon.h | 2 + .../01164_alter_memory_database.reference | 2 + .../01164_alter_memory_database.sql | 12 ++++ ...021_create_database_with_comment.reference | 8 +-- .../02021_create_database_with_comment.sh | 4 +- 10 files changed, 97 insertions(+), 72 deletions(-) create mode 100644 tests/queries/0_stateless/01164_alter_memory_database.reference create mode 100644 tests/queries/0_stateless/01164_alter_memory_database.sql diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 87e5f86c893..8d1220cc1a1 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -91,7 +91,7 @@ ASTPtr DatabaseMemory::getCreateTableQueryImpl(const String & table_name, Contex if (it == create_queries.end() || !it->second) { if (throw_on_error) - throw Exception("There is no metadata of table " + table_name + " in database " + database_name, ErrorCodes::UNKNOWN_TABLE); + throw Exception(ErrorCodes::UNKNOWN_TABLE, "There is no metadata of table {} in database {}", table_name, database_name); else return {}; } @@ -111,4 +111,14 @@ void DatabaseMemory::drop(ContextPtr local_context) std::filesystem::remove_all(local_context->getPath() + data_path); } +void DatabaseMemory::alterTable(ContextPtr, const StorageID & table_id, const StorageInMemoryMetadata & metadata) +{ + std::lock_guard lock{mutex}; + auto it = create_queries.find(table_id.table_name); + if (it == create_queries.end() || !it->second) + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Cannot alter: There is no metadata of table {}", table_id.getNameForLogs()); + + applyMetadataChangesToCreateQuery(it->second, metadata); +} + } diff --git a/src/Databases/DatabaseMemory.h b/src/Databases/DatabaseMemory.h index 8c9cf86ec19..a661d1c49b0 100644 --- a/src/Databases/DatabaseMemory.h +++ b/src/Databases/DatabaseMemory.h @@ -48,6 +48,8 @@ public: void drop(ContextPtr context) override; + void alterTable(ContextPtr local_context, const StorageID & table_id, const StorageInMemoryMetadata & metadata) override; + private: String data_path; using NameToASTCreate = std::unordered_map; diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 638aef7186c..55e5a3071bc 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -137,68 +137,6 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query) return statement_buf.str(); } -void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemoryMetadata & metadata) -{ - auto & ast_create_query = query->as(); - - bool has_structure = ast_create_query.columns_list && ast_create_query.columns_list->columns; - if (ast_create_query.as_table_function && !has_structure) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot alter table {} because it was created AS table function" - " and doesn't have structure in metadata", backQuote(ast_create_query.table)); - - assert(has_structure); - ASTPtr new_columns = InterpreterCreateQuery::formatColumns(metadata.columns); - ASTPtr new_indices = InterpreterCreateQuery::formatIndices(metadata.secondary_indices); - ASTPtr new_constraints = InterpreterCreateQuery::formatConstraints(metadata.constraints); - ASTPtr new_projections = InterpreterCreateQuery::formatProjections(metadata.projections); - - ast_create_query.columns_list->replace(ast_create_query.columns_list->columns, new_columns); - ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->indices, new_indices); - ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->constraints, new_constraints); - ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->projections, new_projections); - - if (metadata.select.select_query) - { - query->replace(ast_create_query.select, metadata.select.select_query); - } - - /// MaterializedView is one type of CREATE query without storage. - if (ast_create_query.storage) - { - ASTStorage & storage_ast = *ast_create_query.storage; - - bool is_extended_storage_def - = storage_ast.partition_by || storage_ast.primary_key || storage_ast.order_by || storage_ast.sample_by || storage_ast.settings; - - if (is_extended_storage_def) - { - if (metadata.sorting_key.definition_ast) - storage_ast.set(storage_ast.order_by, metadata.sorting_key.definition_ast); - - if (metadata.primary_key.definition_ast) - storage_ast.set(storage_ast.primary_key, metadata.primary_key.definition_ast); - - 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); - else if (storage_ast.ttl_table != nullptr) /// TTL was removed - storage_ast.ttl_table = nullptr; - - if (metadata.settings_changes) - storage_ast.set(storage_ast.settings, metadata.settings_changes); - } - } - - if (metadata.comment.empty()) - ast_create_query.reset(ast_create_query.comment); - else - ast_create_query.set(ast_create_query.comment, std::make_shared(metadata.comment)); -} - DatabaseOnDisk::DatabaseOnDisk( const String & name, diff --git a/src/Databases/DatabaseOnDisk.h b/src/Databases/DatabaseOnDisk.h index dce82c2b441..81d5bd5adcb 100644 --- a/src/Databases/DatabaseOnDisk.h +++ b/src/Databases/DatabaseOnDisk.h @@ -24,8 +24,6 @@ std::pair createTableFromAST( */ String getObjectDefinitionFromCreateQuery(const ASTPtr & query); -void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemoryMetadata & metadata); - /* Class to provide basic operations with tables when metadata is stored on disk in .sql files. */ diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 8c0da37a111..2968ad58dd8 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -21,6 +21,69 @@ namespace ErrorCodes extern const int UNKNOWN_DATABASE; } +void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemoryMetadata & metadata) +{ + auto & ast_create_query = query->as(); + + bool has_structure = ast_create_query.columns_list && ast_create_query.columns_list->columns; + if (ast_create_query.as_table_function && !has_structure) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot alter table {} because it was created AS table function" + " and doesn't have structure in metadata", backQuote(ast_create_query.table)); + + assert(has_structure); + ASTPtr new_columns = InterpreterCreateQuery::formatColumns(metadata.columns); + ASTPtr new_indices = InterpreterCreateQuery::formatIndices(metadata.secondary_indices); + ASTPtr new_constraints = InterpreterCreateQuery::formatConstraints(metadata.constraints); + ASTPtr new_projections = InterpreterCreateQuery::formatProjections(metadata.projections); + + ast_create_query.columns_list->replace(ast_create_query.columns_list->columns, new_columns); + ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->indices, new_indices); + ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->constraints, new_constraints); + ast_create_query.columns_list->setOrReplace(ast_create_query.columns_list->projections, new_projections); + + if (metadata.select.select_query) + { + query->replace(ast_create_query.select, metadata.select.select_query); + } + + /// MaterializedView is one type of CREATE query without storage. + if (ast_create_query.storage) + { + ASTStorage & storage_ast = *ast_create_query.storage; + + bool is_extended_storage_def + = storage_ast.partition_by || storage_ast.primary_key || storage_ast.order_by || storage_ast.sample_by || storage_ast.settings; + + if (is_extended_storage_def) + { + if (metadata.sorting_key.definition_ast) + storage_ast.set(storage_ast.order_by, metadata.sorting_key.definition_ast); + + if (metadata.primary_key.definition_ast) + storage_ast.set(storage_ast.primary_key, metadata.primary_key.definition_ast); + + 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); + else if (storage_ast.ttl_table != nullptr) /// TTL was removed + storage_ast.ttl_table = nullptr; + + if (metadata.settings_changes) + storage_ast.set(storage_ast.settings, metadata.settings_changes); + } + } + + if (metadata.comment.empty()) + ast_create_query.reset(ast_create_query.comment); + else + ast_create_query.set(ast_create_query.comment, std::make_shared(metadata.comment)); +} + + DatabaseWithOwnTablesBase::DatabaseWithOwnTablesBase(const String & name_, const String & logger, ContextPtr context_) : IDatabase(name_), WithContext(context_->getGlobalContext()), log(&Poco::Logger::get(logger)) { diff --git a/src/Databases/DatabasesCommon.h b/src/Databases/DatabasesCommon.h index dfa99b632e3..59a2ddc3c41 100644 --- a/src/Databases/DatabasesCommon.h +++ b/src/Databases/DatabasesCommon.h @@ -13,6 +13,8 @@ namespace DB { +void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemoryMetadata & metadata); + class Context; /// A base class for databases that manage their own list of tables. diff --git a/tests/queries/0_stateless/01164_alter_memory_database.reference b/tests/queries/0_stateless/01164_alter_memory_database.reference new file mode 100644 index 00000000000..146d291f5a9 --- /dev/null +++ b/tests/queries/0_stateless/01164_alter_memory_database.reference @@ -0,0 +1,2 @@ +CREATE TABLE test_1164_memory.r1\n(\n `n` Int32,\n `m` Int32\n)\nENGINE = ReplicatedMergeTree(\'/test/01164/t\', \'1\')\nORDER BY n\nSETTINGS index_granularity = 8192 +CREATE TABLE test_1164_memory.r1\n(\n `n` Int32,\n `m` Int32\n)\nENGINE = ReplicatedMergeTree(\'/test/01164/t\', \'1\')\nORDER BY n\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01164_alter_memory_database.sql b/tests/queries/0_stateless/01164_alter_memory_database.sql new file mode 100644 index 00000000000..392ba6ebd3b --- /dev/null +++ b/tests/queries/0_stateless/01164_alter_memory_database.sql @@ -0,0 +1,12 @@ +-- Tags: zookeeper, no-parallel + +drop database if exists test_1164_memory; +create database test_1164_memory engine=Memory; +create table test_1164_memory.r1 (n int) engine=ReplicatedMergeTree('/test/01164/t', '1') order by n; +create table test_1164_memory.r2 (n int) engine=ReplicatedMergeTree('/test/01164/t', '2') order by n; +alter table test_1164_memory.r1 add column m int; +system sync replica test_1164_memory.r1; +system sync replica test_1164_memory.r2; +show create table test_1164_memory.r1; +show create table test_1164_memory.r1; +drop database test_1164_memory; diff --git a/tests/queries/0_stateless/02021_create_database_with_comment.reference b/tests/queries/0_stateless/02021_create_database_with_comment.reference index ada8f919956..65fc9b6d290 100644 --- a/tests/queries/0_stateless/02021_create_database_with_comment.reference +++ b/tests/queries/0_stateless/02021_create_database_with_comment.reference @@ -1,3 +1,7 @@ +engine : Memory +CREATE DATABASE default\nENGINE = Memory()\nCOMMENT \'Test DB with comment\' +comment= Test DB with comment + engine : Atomic CREATE DATABASE default\nENGINE = Atomic\nCOMMENT \'Test DB with comment\' comment= Test DB with comment @@ -10,7 +14,3 @@ engine : Lazy(1) CREATE DATABASE default\nENGINE = Lazy(1)\nCOMMENT \'Test DB with comment\' comment= Test DB with comment -engine : Memory -CREATE DATABASE default\nENGINE = Memory()\nCOMMENT \'Test DB with comment\' -comment= Test DB with comment - diff --git a/tests/queries/0_stateless/02021_create_database_with_comment.sh b/tests/queries/0_stateless/02021_create_database_with_comment.sh index d8b288eb9ba..ce7a7bef066 100755 --- a/tests/queries/0_stateless/02021_create_database_with_comment.sh +++ b/tests/queries/0_stateless/02021_create_database_with_comment.sh @@ -26,12 +26,10 @@ EOF get_db_comment_info } -# For some reason order seems important, putting Atomic after Memory makes test fail every time -# due to DB metadata SQL file still being present after "DROP DATABASE" +test_db_comments "Memory" test_db_comments "Atomic" test_db_comments "Ordinary" test_db_comments "Lazy(1)" -test_db_comments "Memory" # test_db_comments "MySQL('127.0.0.1:9004', 'default', 'default', '')" # fails due to CH internal reasons # test_db_comments "SQLite('dummy_sqlitedb')" ## needs to be explicitly enabled with `SET allow_experimental_database_replicated=1` From a5901f2901a6a901e7dbbd29e0d824eb29b1cd5e Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 29 Oct 2021 19:58:13 +0300 Subject: [PATCH 245/396] Update DatabasesCommon.cpp --- src/Databases/DatabasesCommon.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 2968ad58dd8..e5e54f287ec 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -19,6 +19,7 @@ namespace ErrorCodes extern const int TABLE_ALREADY_EXISTS; extern const int UNKNOWN_TABLE; extern const int UNKNOWN_DATABASE; + extern const int NOT_IMPLEMENTED; } void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemoryMetadata & metadata) From 138c4fcd9d2cda8d77ec86eabaf2cbef824c043c Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 29 Oct 2021 19:58:42 +0300 Subject: [PATCH 246/396] Update 01164_alter_memory_database.sql --- tests/queries/0_stateless/01164_alter_memory_database.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01164_alter_memory_database.sql b/tests/queries/0_stateless/01164_alter_memory_database.sql index 392ba6ebd3b..f46fc8f9853 100644 --- a/tests/queries/0_stateless/01164_alter_memory_database.sql +++ b/tests/queries/0_stateless/01164_alter_memory_database.sql @@ -2,8 +2,8 @@ drop database if exists test_1164_memory; create database test_1164_memory engine=Memory; -create table test_1164_memory.r1 (n int) engine=ReplicatedMergeTree('/test/01164/t', '1') order by n; -create table test_1164_memory.r2 (n int) engine=ReplicatedMergeTree('/test/01164/t', '2') order by n; +create table test_1164_memory.r1 (n int) engine=ReplicatedMergeTree('/test/01164/{database}/t', '1') order by n; +create table test_1164_memory.r2 (n int) engine=ReplicatedMergeTree('/test/01164/{database}/t', '2') order by n; alter table test_1164_memory.r1 add column m int; system sync replica test_1164_memory.r1; system sync replica test_1164_memory.r2; From f3671ad9e96125fa14e4c672c73cc4850ef22b90 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Fri, 29 Oct 2021 20:32:04 +0300 Subject: [PATCH 247/396] Correction of inaccuracies MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил по комментариям разработчика. --- docs/en/operations/system-tables/columns.md | 2 +- .../system-tables/information_schema.md | 22 +++++++------- docs/ru/operations/system-tables/columns.md | 10 +++---- .../system-tables/information_schema.md | 30 +++++++++---------- 4 files changed, 32 insertions(+), 32 deletions(-) diff --git a/docs/en/operations/system-tables/columns.md b/docs/en/operations/system-tables/columns.md index 8b420eb7894..5ba38ab3e67 100644 --- a/docs/en/operations/system-tables/columns.md +++ b/docs/en/operations/system-tables/columns.md @@ -25,7 +25,7 @@ Columns: - `is_in_sampling_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the sampling key expression. - `compression_codec` ([String](../../sql-reference/data-types/string.md)) — Compression codec name. - `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. In ClickHouse makes sense only for `FixedString` data type. Otherwise, the `NULL` value is returned. -- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it's bitness for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned. +- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it is bitness for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned. - `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. In ClickHouse it's 2 for integer types and 10 for `Decimal` types. Otherwise, the `NULL` value is returned. - `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse makes sense only for `Decimal` types. Otherwise, the `NULL` value is returned. - `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Decimal precision of `DateTime64` data type. For other data types, the `NULL` value is returned. diff --git a/docs/en/operations/system-tables/information_schema.md b/docs/en/operations/system-tables/information_schema.md index 49aa2fa0337..df5b012f2b6 100644 --- a/docs/en/operations/system-tables/information_schema.md +++ b/docs/en/operations/system-tables/information_schema.md @@ -26,7 +26,7 @@ SHOW TABLES FROM INFORMATION_SCHEMA; ## COLUMNS {#columns} -Contains columns read from the `system.columns` system table and columns that are not supported in ClickHouse or do not make sense (always `NULL`), but must be by the standard. +Contains columns read from the [system.columns](../../operations/system-tables/columns.md) system table and columns that are not supported in ClickHouse or do not make sense (always `NULL`), but must be by the standard. Columns: @@ -38,12 +38,12 @@ Columns: - `column_default` ([String](../../sql-reference/data-types/string.md)) — Expression for the default value, or an empty string if it is not defined. - `is_nullable` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column type is `Nullable`. - `data_type` ([String](../../sql-reference/data-types/string.md)) — Column type. -- `character_maximum_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. -- `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. -- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. Otherwise, the `NULL` value is returned. -- `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. Otherwise, the `NULL` value is returned. -- `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. Otherwise, the `NULL` value is returned. -- `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Subtype code for `DateTime` and `ISO` interval data types. For other data types, the `NULL` value is returned. +- `character_maximum_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. In ClickHouse makes sense only for `FixedString` data type. Otherwise, the `NULL` value is returned. +- `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. In ClickHouse makes sense only for `FixedString` data type. Otherwise, the `NULL` value is returned. +- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it is bitness for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned. +- `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. In ClickHouse it's 2 for integer types and 10 for `Decimal` types. Otherwise, the `NULL` value is returned. +- `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse makes sense only for `Decimal` types. Otherwise, the `NULL` value is returned. +- `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Decimal precision of `DateTime64` data type. For other data types, the `NULL` value is returned. - `character_set_catalog` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. - `character_set_schema` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. - `character_set_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, not supported. @@ -94,7 +94,7 @@ domain_name: ᴺᵁᴸᴸ ## SCHEMATA {#schemata} -Contains columns read from the `system.databases` system table and columns that are not supported in ClickHouse or do not make sense (always `NULL`), but must be by the standard. +Contains columns read from the [system.databases](../../operations/system-tables/databases.md) system table and columns that are not supported in ClickHouse or do not make sense (always `NULL`), but must be by the standard. Columns: @@ -130,12 +130,12 @@ sql_path: ᴺᵁᴸᴸ ## TABLES {#tables} -Contains columns read from the `system.tables` system table. +Contains columns read from the [system.tables](../../operations/system-tables/tables.md) system table. Columns: - `table_catalog` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the table is located. -- `table_schema` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the schema is located. +- `table_schema` ([String](../../sql-reference/data-types/string.md)) — The name of the database in which the table is located. - `table_name` ([String](../../sql-reference/data-types/string.md)) — Table name. - `table_type` ([Enum8](../../sql-reference/data-types/enum.md)) — Table type. Possible values: - `BASE TABLE` @@ -165,7 +165,7 @@ table_type: BASE TABLE ## VIEWS {#views} -Contains columns read from the `system.tables` system table, when the table engine [View](../../engines/table-engines/special/view.md) is used. +Contains columns read from the [system.tables](../../operations/system-tables/tables.md) system table, when the table engine [View](../../engines/table-engines/special/view.md) is used. Columns: diff --git a/docs/ru/operations/system-tables/columns.md b/docs/ru/operations/system-tables/columns.md index 5b526372820..b5108386ce8 100644 --- a/docs/ru/operations/system-tables/columns.md +++ b/docs/ru/operations/system-tables/columns.md @@ -24,11 +24,11 @@ Cтолбцы: - `is_in_primary_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — флаг, показывающий включение столбца в первичный ключ. - `is_in_sampling_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — флаг, показывающий включение столбца в ключ выборки. - `compression_codec` ([String](../../sql-reference/data-types/string.md)) — имя кодека сжатия. -- `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Максимальная длина в байтах для двоичных данных, символьных данных или текстовых данных и изображений. Иначе возвращается значение `NULL`. -- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Точность приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. Иначе возвращается значение `NULL`. -- `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Основание системы счисления точности приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. Иначе возвращается значение `NULL`. -- `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Масштаб приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. Иначе возвращается значение `NULL`. -- `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Код подтипа для типов данных интервала `DateTime` и `ISO`. Для других типов данных возвращается значение `NULL`. +- `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — максимальная длина в байтах для двоичных данных, символьных данных или текстовых данных и изображений. В ClickHouse имеет смысл только для типа данных `FixedString`. Иначе возвращается значение `NULL`. +- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — точность приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. В ClickHouse это разрядность для целочисленных типов и десятичная точность для типов `Decimal`. Иначе возвращается значение `NULL`. +- `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — основание системы счисления точности приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. В ClickHouse значение столбца равно 2 для целочисленных типов и 10 — для типов `Decimal`. Иначе возвращается значение `NULL`. +- `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — масштаб приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. В ClickHouse имеет смысл только для типов `Decimal`. Иначе возвращается значение `NULL`. +- `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — десятичная точность для данных типа `DateTime64`. Для других типов данных возвращается значение `NULL`. **Пример** diff --git a/docs/ru/operations/system-tables/information_schema.md b/docs/ru/operations/system-tables/information_schema.md index 1cc970757b3..b61418931bd 100644 --- a/docs/ru/operations/system-tables/information_schema.md +++ b/docs/ru/operations/system-tables/information_schema.md @@ -26,24 +26,24 @@ SHOW TABLES FROM INFORMATION_SCHEMA; ## COLUMNS {#columns} -Содержит столбцы, которые считываются из системной таблицы `system.columns`, и столбцы, которые не поддерживаются в ClickHouse или не имеют смысла (всегда имеют значение `NULL`), но должны быть по стандарту. +Содержит столбцы, которые считываются из системной таблицы [system.columns](../../operations/system-tables/columns.md), и столбцы, которые не поддерживаются в ClickHouse или не имеют смысла (всегда имеют значение `NULL`), но должны быть по стандарту. Столбцы: - `table_catalog` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится таблица. -- `table_schema` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится схема. +- `table_schema` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится таблица. - `table_name` ([String](../../sql-reference/data-types/string.md)) — имя таблицы. - `column_name` ([String](../../sql-reference/data-types/string.md)) — имя столбца. - `ordinal_position` ([UInt64](../../sql-reference/data-types/int-uint.md)) — порядковый номер столбца в таблице (нумерация начинается с 1). - `column_default` ([String](../../sql-reference/data-types/string.md)) — выражение для значения по умолчанию или пустая строка. - `is_nullable` ([UInt8](../../sql-reference/data-types/int-uint.md)) — флаг, показывающий является ли столбец типа `Nullable`. - `data_type` ([String](../../sql-reference/data-types/string.md)) — тип столбца. -- `character_maximum_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — максимальная длина в байтах для двоичных данных, символьных данных или текстовых данных и изображений. Иначе возвращается значение `NULL`. -- `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — максимальная длина в байтах для двоичных данных, символьных данных или текстовых данных и изображений. Иначе возвращается значение `NULL`. -- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — точность приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. Иначе возвращается значение `NULL`. -- `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — основание системы счисления точности приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. Иначе возвращается значение `NULL`. -- `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — масштаб приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. Иначе возвращается значение `NULL`. -- `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — код подтипа для типов данных интервала `DateTime` и `ISO`. Для других типов данных возвращается значение `NULL`. +- `character_maximum_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — максимальная длина в байтах для двоичных данных, символьных данных или текстовых данных и изображений. В ClickHouse имеет смысл только для типа данных `FixedString`. Иначе возвращается значение `NULL`. +- `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — максимальная длина в байтах для двоичных данных, символьных данных или текстовых данных и изображений. В ClickHouse имеет смысл только для типа данных `FixedString`. Иначе возвращается значение `NULL`. +- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — точность приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. В ClickHouse это разрядность для целочисленных типов и десятичная точность для типов `Decimal`. Иначе возвращается значение `NULL`. +- `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — основание системы счисления точности приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. В ClickHouse значение столбца равно 2 для целочисленных типов и 10 — для типов `Decimal`. Иначе возвращается значение `NULL`. +- `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — масштаб приблизительных числовых данных, точных числовых данных, целочисленных данных или денежных данных. В ClickHouse имеет смысл только для типов `Decimal`. Иначе возвращается значение `NULL`. +- `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — десятичная точность для данных типа `DateTime64`. Для других типов данных возвращается значение `NULL`. - `character_set_catalog` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. - `character_set_schema` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. - `character_set_name` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. @@ -94,12 +94,12 @@ domain_name: ᴺᵁᴸᴸ ## SCHEMATA {#schemata} -Содержит столбцы, которые считываются из системной таблицы `system.databases`, и столбцы, которые не поддерживаются в ClickHouse или не имеют смысла (всегда имеют значение `NULL`), но должны быть по стандарту. +Содержит столбцы, которые считываются из системной таблицы [system.databases](../../operations/system-tables/databases.md), и столбцы, которые не поддерживаются в ClickHouse или не имеют смысла (всегда имеют значение `NULL`), но должны быть по стандарту. Столбцы: -- `catalog_name` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится таблица. -- `schema_name` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится схема. +- `catalog_name` ([String](../../sql-reference/data-types/string.md)) — имя базы данных. +- `schema_name` ([String](../../sql-reference/data-types/string.md)) — имя базы данных. - `schema_owner` ([String](../../sql-reference/data-types/string.md)) — имя владельца схемы, всегда `'default'`. - `default_character_set_catalog` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. - `default_character_set_schema` ([Nullable](../../sql-reference/data-types/nullable.md)([String](../../sql-reference/data-types/string.md))) — `NULL`, не поддерживается. @@ -130,12 +130,12 @@ sql_path: ᴺᵁᴸᴸ ## TABLES {#tables} -Содержит столбцы, которые считываются из системной таблицы `system.tables`. +Содержит столбцы, которые считываются из системной таблицы [system.tables](../../operations/system-tables/tables.md). Столбцы: - `table_catalog` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится таблица. -- `table_schema` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится схема. +- `table_schema` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится таблица. - `table_name` ([String](../../sql-reference/data-types/string.md)) — имя таблицы. - `table_type` ([Enum8](../../sql-reference/data-types/enum.md)) — тип таблицы. Возможные значения: - `BASE TABLE` @@ -165,12 +165,12 @@ table_type: BASE TABLE ## VIEWS {#views} -Содержит столбцы, которые считываются из системной таблицы `system.tables`, если использован движок [View](../../engines/table-engines/special/view.md). +Содержит столбцы, которые считываются из системной таблицы [system.tables](../../operations/system-tables/tables.md), если использован движок [View](../../engines/table-engines/special/view.md). Столбцы: - `table_catalog` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится таблица. -- `table_schema` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится схема. +- `table_schema` ([String](../../sql-reference/data-types/string.md)) — имя базы данных, в которой находится таблица. - `table_name` ([String](../../sql-reference/data-types/string.md)) — имя таблицы. - `view_definition` ([String](../../sql-reference/data-types/string.md)) — `SELECT` запрос для представления. - `check_option` ([String](../../sql-reference/data-types/string.md)) — `NONE`, нет проверки. From 7ae8995ff37b75af62430cfd67a72d336a54eb03 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 20:40:28 +0300 Subject: [PATCH 248/396] Better messages --- tests/ci/docs_check.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/docs_check.py b/tests/ci/docs_check.py index a99641537c5..aece781a703 100644 --- a/tests/ci/docs_check.py +++ b/tests/ci/docs_check.py @@ -112,9 +112,9 @@ if __name__ == "__main__": if retcode == 0: logging.info("Run successfully") status = "success" - description = "Run Ok" + description = "Docs check passed" else: - description = "Run failed (non zero exit code)" + description = "Docs check failed (non zero exit code)" status = "failure" logging.info("Run failed") @@ -136,7 +136,7 @@ if __name__ == "__main__": lines.append((line.split(':')[-1], "FAIL")) if lines: status = "failure" - description = "Found errors during docs check" + description = "Found errors in docs" elif status != "failure": lines.append(("No errors found", "OK")) else: From ae64b829610055dc5261d22deabbdf811734d539 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 20:41:43 +0300 Subject: [PATCH 249/396] More checks --- tests/ci/docs_release.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/docs_release.py b/tests/ci/docs_release.py index be6915f3cc9..5a4c9537c06 100644 --- a/tests/ci/docs_release.py +++ b/tests/ci/docs_release.py @@ -115,9 +115,9 @@ if __name__ == "__main__": if retcode == 0: logging.info("Run successfully") status = "success" - description = "Run Ok" + description = "Released successfuly" else: - description = "Run failed (non zero exit code)" + description = "Release failed (non zero exit code)" status = "failure" logging.info("Run failed") @@ -139,7 +139,7 @@ if __name__ == "__main__": lines.append((line.split(':')[-1], "FAIL")) if lines: status = "failure" - description = "Found errors during docs release" + description = "Found errors in docs" elif status != "failure": lines.append(("No errors found", "OK")) else: From 3e36edd5d2b762000866460800ebe18b16e392e4 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Fri, 29 Oct 2021 21:35:11 +0300 Subject: [PATCH 250/396] Document the min_bytes_to_rebalance_partition_over_jbod setting MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Задокументировал настройку min_bytes_to_rebalance_partition_over_jbod. --- .../table-engines/mergetree-family/mergetree.md | 2 ++ docs/en/operations/settings/merge-tree-settings.md | 14 ++++++++++++++ 2 files changed, 16 insertions(+) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index aeaf39e28cb..dd9a18c8dba 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -787,6 +787,8 @@ Moving data does not interfere with data replication. Therefore, different stora After the completion of background merges and mutations, old parts are removed only after a certain amount of time (`old_parts_lifetime`). During this time, they are not moved to other volumes or disks. Therefore, until the parts are finally removed, they are still taken into account for evaluation of the occupied disk space. +User can assign new parts to different disks of a [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures) volume in a balanced way using the [min_bytes_to_rebalance_partition_over_jbod](../../../operations/settings/merge-tree-settings.md#min-bytes-to-rebalance-partition-over-jbod) setting. + ## Using S3 for Data Storage {#table_engine-mergetree-s3} `MergeTree` family table engines can store data to [S3](https://aws.amazon.com/s3/) using a disk with type `s3`. diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index a3a258234e1..1a723f4efb0 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -328,3 +328,17 @@ Possible values: Default value: `true`. By default, the ClickHouse server checks at table creation the data type of a column for sampling or sampling expression. If you already have tables with incorrect sampling expression and do not want the server to raise an exception during startup, set `check_sample_column_is_correct` to `false`. + +## min_bytes_to_rebalance_partition_over_jbod {#min-bytes-to-rebalance-partition-over-jbod} + +Allows assigning new parts to different disks of a [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures) volume in a balanced way. + +Possible values: + +- Any positive integer. + +Default value: `0`. + +**Usage** + +Throw exception if the value of the `min_bytes_to_rebalance_partition_over_jbod` setting should be less than the value of the [max_bytes_to_merge_at_max_space_in_pool](../../operations/settings/merge-tree-settings.md#max-bytes-to-merge-at-max-space-in-pool) setting. From 0dfc1adbe364eccb88e7858a51cece837f609f1c Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 21:42:42 +0300 Subject: [PATCH 251/396] Better --- docker/docs/builder/Dockerfile | 1 + tests/ci/docs_release.py | 4 +- tests/ci/ssh.py | 117 +++++++++++++++++++++++++++++++++ 3 files changed, 120 insertions(+), 2 deletions(-) create mode 100644 tests/ci/ssh.py diff --git a/docker/docs/builder/Dockerfile b/docker/docs/builder/Dockerfile index c47554808cb..8afddefa41a 100644 --- a/docker/docs/builder/Dockerfile +++ b/docker/docs/builder/Dockerfile @@ -30,6 +30,7 @@ RUN apt-get update \ nodejs \ npm \ openjdk-11-jdk \ + ssh-client \ && pip --no-cache-dir install scipy \ && apt-get autoremove --yes \ && apt-get clean \ diff --git a/tests/ci/docs_release.py b/tests/ci/docs_release.py index 5a4c9537c06..dfb3d50f1d8 100644 --- a/tests/ci/docs_release.py +++ b/tests/ci/docs_release.py @@ -12,6 +12,7 @@ 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 ssh import SSHKey NAME = "Docs Release (actions)" @@ -109,7 +110,7 @@ if __name__ == "__main__": run_log_path = os.path.join(test_output, 'runlog.log') - with open(run_log_path, 'w', encoding='utf-8') as log: + with open(run_log_path, 'w', encoding='utf-8') as log, SSHKey("robot-clickhouse-ssh"): with subprocess.Popen(cmd, shell=True, stderr=log, stdout=log) as process: retcode = process.wait() if retcode == 0: @@ -149,5 +150,4 @@ 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/ssh.py b/tests/ci/ssh.py new file mode 100644 index 00000000000..3e0cca31664 --- /dev/null +++ b/tests/ci/ssh.py @@ -0,0 +1,117 @@ +#!/usr/bin/env python3 + +import shutil +import os +import subprocess +import tempfile +import logging +import signal +from get_robot_token import get_parameter_from_ssm + + +class SSHAgent: + def __init__(self): + self._env = {} + self._env_backup = {} + self._keys = {} + self.start() + + @property + def pid(self): + return int(self._env["SSH_AGENT_PID"]) + + def start(self): + if shutil.which("ssh-agent") is None: + raise Exception("ssh-agent binary is not available") + + self._env_backup["SSH_AUTH_SOCK"] = os.environ.get("SSH_AUTH_SOCK") + self._env_backup["SSH_OPTIONS"] = os.environ.get("SSH_OPTIONS") + + # set ENV from stdout of ssh-agent + for line in self._run(['ssh-agent']).splitlines(): + name, _, value = line.partition(b"=") + if _ == b"=": + value = value.split(b";", 1)[0] + self._env[name.decode()] = value.decode() + os.environ[name.decode()] = value.decode() + + ssh_options = "," + os.environ["SSH_OPTIONS"] if os.environ.get("SSH_OPTIONS") else "" + os.environ["SSH_OPTIONS"] = f"{ssh_options}UserKnownHostsFile=/dev/null,StrictHostKeyChecking=no" + + def add(self, key): + key_pub = self._key_pub(key) + + if key_pub in self._keys: + self._keys[key_pub] += 1 + else: + self._run(["ssh-add", "-"], stdin=key.encode()) + self._keys[key_pub] = 1 + + return key_pub + + def remove(self, key_pub): + if key_pub not in self._keys: + raise Exception(f"Private key not found, public part: {key_pub}") + + if self._keys[key_pub] > 1: + self._keys[key_pub] -= 1 + else: + with tempfile.NamedTemporaryFile() as f: + f.write(key_pub) + f.flush() + self._run(["ssh-add", "-d", f.name]) + self._keys.pop(key_pub) + + def print_keys(self): + keys = self._run(["ssh-add", "-l"]).splitlines() + if keys: + logging.info("ssh-agent keys:") + for key in keys: + logging.info("%s", key) + else: + logging.info("ssh-agent (pid %d) is empty", self.pid) + + def kill(self): + for k, v in self._env.items(): + os.environ.pop(k, None) + + for k, v in self._env_backup.items(): + if v is not None: + os.environ[k] = v + + os.kill(self.pid, signal.SIGTERM) + + def _key_pub(self, key): + with tempfile.NamedTemporaryFile() as f: + f.write(key.encode()) + f.flush() + return self._run(["ssh-keygen", "-y", "-f", f.name]) + + @staticmethod + def _run(cmd, stdin=None): + shell = isinstance(cmd, str) + with subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.PIPE, stdin=subprocess.PIPE if stdin else None, shell=shell) as p: + stdout, stderr = p.communicate(stdin) + + if stdout.strip().decode() == "The agent has no identities.": + return "" + + if p.returncode: + message = stderr.strip() + b"\n" + stdout.strip() + raise Exception(message.strip().decode()) + + return stdout + +class SSHKey: + def __init__(self, key_name): + self.key = get_parameter_from_ssm(key_name) + self._key_pub = None + self._ssh_agent = SSHAgent() + + def __enter__(self): + self._key_pub = self._ssh_agent.add(self.key) + self._ssh_agent.print_keys() + + def __exit__(self, exc_type, exc_val, exc_tb): + self._ssh_agent.remove(self._key_pub) + self._ssh_agent.print_keys() From faeb4ef869a88e01d8abf45e9ce25da06c6cd4c0 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 252/396] 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 afcadc95a775ef4665e5edcc7205eb2ee28b20c2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 12:58:25 +0300 Subject: [PATCH 253/396] Add docs check to documentation --- .github/workflows/main.yml | 27 +++++- docker/docs/builder/Dockerfile | 7 +- docker/images.json | 7 +- tests/ci/docs_check.py | 148 +++++++++++++++++++++++++++++++++ tests/ci/pr_info.py | 15 ++++ 5 files changed, 197 insertions(+), 7 deletions(-) create mode 100644 tests/ci/docs_check.py diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 69dc2e94e15..22f6aa775f3 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -21,7 +21,6 @@ jobs: python3 run_check.py DockerHubPush: needs: CheckLabels - if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, style-checker] steps: - name: Check out repository code @@ -57,8 +56,33 @@ jobs: run: | docker kill $(docker ps -q) ||: sudo rm -fr $TEMP_PATH + DocsCheck: + needs: DockerHubPush + runs-on: [self-hosted, style-checker] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/docs_check + - name: Check out repository code + uses: actions/checkout@v2 + - name: Style Check + env: + TEMP_PATH: ${{runner.temp}}/docs_check + REPO_COPY: ${{runner.temp}}/docs_check/ClickHouse + run: | + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 docs_check.py + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + sudo rm -fr $TEMP_PATH BuilderDebDebug: needs: DockerHubPush + if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: - name: Download changed images @@ -179,6 +203,7 @@ jobs: 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') }} runs-on: [self-hosted, builder] steps: - name: Check out repository code diff --git a/docker/docs/builder/Dockerfile b/docker/docs/builder/Dockerfile index 1fc27c5fc18..0b0f4ef71b5 100644 --- a/docker/docs/builder/Dockerfile +++ b/docker/docs/builder/Dockerfile @@ -1,4 +1,4 @@ -# docker build -t yandex/clickhouse-docs-build . +# docker build -t clickhouse/docs-build . FROM ubuntu:20.04 ENV LANG=C.UTF-8 @@ -44,7 +44,4 @@ certifi==2020.4.5.2 chardet==3.0.4 click==7.1.2 closure==20191111 cssmin==0.2.0 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 * / +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 diff --git a/docker/images.json b/docker/images.json index 1cb999bd16f..c49f10d676f 100644 --- a/docker/images.json +++ b/docker/images.json @@ -169,6 +169,11 @@ }, "docker/docs/builder": { "name": "clickhouse/docs-builder", - "dependent": [] + "dependent": ["docker/docs/check"] + }, + "docker/docs/check": { + "name": "clickhouse/docs-check", + "dependent": [] } + } diff --git a/tests/ci/docs_check.py b/tests/ci/docs_check.py new file mode 100644 index 00000000000..d8a68d29cd3 --- /dev/null +++ b/tests/ci/docs_check.py @@ -0,0 +1,148 @@ +#!/usr/bin/env python3 +import logging +import subprocess +import os +import time +import json +import sys +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 + +NAME = "Docs Check (actions)" + +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, additional_files): + s3_path_prefix = f"{pr_number}/{commit_sha}/docs_check" + 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(NAME, test_results, raw_log_url, task_url, branch_url, branch_name, commit_url, additional_urls) + 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 + +if __name__ == "__main__": + logging.basicConfig(level=logging.INFO) + + temp_path = os.path.join(os.getenv("TEMP_PATH")) + repo_path = os.path.join(os.getenv("REPO_COPY")) + + with open(os.getenv('GITHUB_EVENT_PATH'), 'r', encoding='utf-8') as event_file: + event = json.load(event_file) + + pr_info = PRInfo(event, need_changed_files=True) + + gh = Github(get_best_robot_token()) + if not pr_info.has_changes_in_documentation(): + logging.info ("No changes in documentation") + commit = get_commit(gh, pr_info.sha) + commit.create_status(context=NAME, description="No changes in docs", state="success") + sys.exit(0) + + logging.info("Has changes in docs") + + if not os.path.exists(temp_path): + os.makedirs(temp_path) + + images_path = os.path.join(temp_path, 'changed_images.json') + + docker_image = 'clickhouse/docs-check' + if 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 'clickhouse/docs-check' in images: + docker_image += ':' + images['clickhouse/docs-check'] + + logging.info("Got docker image %s", docker_image) + for i in range(10): + try: + subprocess.check_output(f"docker pull {docker_image}", 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_image}") + + test_output = os.path.join(temp_path, 'docs_check_log') + if not os.path.exists(test_output): + os.makedirs(test_output) + + cmd = "docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={test_output}:/test_output {docker_image}" + + run_log_path = os.path.join(test_output, 'runlog.log') + + with open(run_log_path, 'w', encoding='utf-8') as log: + with subprocess.Popen(cmd, shell=True, stderr=log, stdout=log) as process: + retcode = process.wait() + if retcode == 0: + logging.info("Run successfully") + status = "Success" + description = "Run Ok" + else: + description = "Run failed (non zero exit code)" + status = "failure" + logging.info("Run failed") + + subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) + files = os.listdir(test_output) + lines = [] + additional_files = [] + if not files: + logging.error("No output files after docs check") + description = "No output files after docs check" + status = "failure" + else: + for f in files: + path = os.path.join(test_output, f) + additional_files.append(path) + with open(path, 'r', encoding='utf-8') as check_file: + for line in check_file: + if "ERROR" in line: + lines.append((line.split(':')[-1], "FAIL")) + if lines: + status = "failure" + description = "Found errors during docs check" + else: + lines.append(("No errors found", "OK")) + + s3_helper = S3Helper('https://s3.amazonaws.com') + + 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 0b4aeb56699..672043bd7b7 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -2,8 +2,11 @@ import urllib import requests from unidiff import PatchSet +import os +DIFF_IN_DOCUMENTATION_EXT = [".html", ".md", ".yml", ".txt", ".css", ".js", ".xml", ".ico", ".conf", ".svg", ".png", ".jpg", ".py", ".sh"] + class PRInfo: def __init__(self, github_event, need_orgs=False, need_changed_files=False): self.number = github_event['number'] @@ -37,6 +40,18 @@ class PRInfo: 'user_orgs': self.user_orgs, } + def has_changes_in_documentation(self): + # If the list wasn't built yet the best we can do is to + # assume that there were changes. + if self.changed_files is None or not self.changed_files: + return True + + for f in self.changed_files: + _, ext = os.path.splitext(f) + if ext in DIFF_IN_DOCUMENTATION_EXT or 'Dockerfile' in f: + return True + return False + class FakePRInfo: def __init__(self): From 11f71fffe1856f1d9ff24d30f93d04e70aa8652c Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 12:59:50 +0300 Subject: [PATCH 254/396] Tabs to spaces --- docker/images.json | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/images.json b/docker/images.json index c49f10d676f..adbaf794e71 100644 --- a/docker/images.json +++ b/docker/images.json @@ -169,11 +169,11 @@ }, "docker/docs/builder": { "name": "clickhouse/docs-builder", - "dependent": ["docker/docs/check"] + "dependent": ["docker/docs/check"] }, "docker/docs/check": { "name": "clickhouse/docs-check", - "dependent": [] + "dependent": [] } } From 6c8ba21c52662f26613c695aff13c8ad7f4aa766 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 13:00:27 +0300 Subject: [PATCH 255/396] Add docs check image --- docker/docs/check/Dockerfile | 9 +++++++++ docker/docs/check/run.sh | 6 ++++++ 2 files changed, 15 insertions(+) create mode 100644 docker/docs/check/Dockerfile create mode 100644 docker/docs/check/run.sh diff --git a/docker/docs/check/Dockerfile b/docker/docs/check/Dockerfile new file mode 100644 index 00000000000..55647df5c3e --- /dev/null +++ b/docker/docs/check/Dockerfile @@ -0,0 +1,9 @@ +# docker build -t clickhouse/docs-check . +FROM clickhouse/docs-builder + +COPY run.sh / + +ENV REPO_PATH=/repo_path +ENV OUTPUT_PATH=/output_path + +CMD ["/bin/bash", "/run.sh"] diff --git a/docker/docs/check/run.sh b/docker/docs/check/run.sh new file mode 100644 index 00000000000..57f39afec27 --- /dev/null +++ b/docker/docs/check/run.sh @@ -0,0 +1,6 @@ +#!/usr/bin/env bash +set -euo pipefail + + +cd $REPO_PATH/docs/tools +./build.py --skip-git-log 2>&1 | tee $OUTPUT_PATH/output.log From 05ba8c8009c8e6767481f2dc05815231e13f792b Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 13:01:33 +0300 Subject: [PATCH 256/396] Fix yml --- .github/workflows/main.yml | 48 +++++++++++++++++++------------------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 22f6aa775f3..36da6c4e40f 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -56,30 +56,30 @@ jobs: run: | docker kill $(docker ps -q) ||: sudo rm -fr $TEMP_PATH - DocsCheck: - needs: DockerHubPush - runs-on: [self-hosted, style-checker] - steps: - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ runner.temp }}/docs_check - - name: Check out repository code - uses: actions/checkout@v2 - - name: Style Check - env: - TEMP_PATH: ${{runner.temp}}/docs_check - REPO_COPY: ${{runner.temp}}/docs_check/ClickHouse - run: | - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 docs_check.py - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - sudo rm -fr $TEMP_PATH + DocsCheck: + needs: DockerHubPush + runs-on: [self-hosted, style-checker] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/docs_check + - name: Check out repository code + uses: actions/checkout@v2 + - name: Style Check + env: + TEMP_PATH: ${{runner.temp}}/docs_check + REPO_COPY: ${{runner.temp}}/docs_check/ClickHouse + run: | + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 docs_check.py + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + sudo rm -fr $TEMP_PATH BuilderDebDebug: needs: DockerHubPush if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} From 23a39572097b202dbdb51c557e6bba6e6462cd15 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 14:39:05 +0300 Subject: [PATCH 257/396] Fix docs --- .github/workflows/main.yml | 2 +- tests/ci/docs_check.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 36da6c4e40f..a3ce64e0701 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -67,7 +67,7 @@ jobs: path: ${{ runner.temp }}/docs_check - name: Check out repository code uses: actions/checkout@v2 - - name: Style Check + - name: Docs Check env: TEMP_PATH: ${{runner.temp}}/docs_check REPO_COPY: ${{runner.temp}}/docs_check/ClickHouse diff --git a/tests/ci/docs_check.py b/tests/ci/docs_check.py index d8a68d29cd3..10bb2f3a396 100644 --- a/tests/ci/docs_check.py +++ b/tests/ci/docs_check.py @@ -102,7 +102,7 @@ if __name__ == "__main__": if not os.path.exists(test_output): os.makedirs(test_output) - cmd = "docker run --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={test_output}:/test_output {docker_image}" + cmd = f"docker run --cap-add=SYS_PTRACE --volume={repo_path}:/repo_path --volume={test_output}:/output_path {docker_image}" run_log_path = os.path.join(test_output, 'runlog.log') From 6425d2fb49e31b0e557b15f37af964fe9967fda5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 14:40:16 +0300 Subject: [PATCH 258/396] Fix padding: --- tests/ci/docs_check.py | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/ci/docs_check.py b/tests/ci/docs_check.py index 10bb2f3a396..b86ab9d9871 100644 --- a/tests/ci/docs_check.py +++ b/tests/ci/docs_check.py @@ -109,14 +109,14 @@ if __name__ == "__main__": with open(run_log_path, 'w', encoding='utf-8') as log: with subprocess.Popen(cmd, shell=True, stderr=log, stdout=log) as process: retcode = process.wait() - if retcode == 0: - logging.info("Run successfully") - status = "Success" - description = "Run Ok" - else: - description = "Run failed (non zero exit code)" - status = "failure" - logging.info("Run failed") + if retcode == 0: + logging.info("Run successfully") + status = "Success" + description = "Run Ok" + else: + description = "Run failed (non zero exit code)" + status = "failure" + logging.info("Run failed") subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) files = os.listdir(test_output) From 1a13a741f95c9efb654f2a0a1e63a7bd345cad9f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 16:57:47 +0300 Subject: [PATCH 259/396] Fixes in docs --- docker/docs/builder/Dockerfile | 7 +------ docker/docs/check/run.sh | 5 ++++- tests/ci/docs_check.py | 4 +++- tests/ci/pr_info.py | 3 ++- 4 files changed, 10 insertions(+), 9 deletions(-) diff --git a/docker/docs/builder/Dockerfile b/docker/docs/builder/Dockerfile index 0b0f4ef71b5..c47554808cb 100644 --- a/docker/docs/builder/Dockerfile +++ b/docker/docs/builder/Dockerfile @@ -39,9 +39,4 @@ RUN wget 'https://github.com/wkhtmltopdf/packaging/releases/download/0.12.6-1/wk 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 +RUN pip3 install --ignore-installed --upgrade setuptools pip virtualenv diff --git a/docker/docs/check/run.sh b/docker/docs/check/run.sh index 57f39afec27..f70f82aeb4c 100644 --- a/docker/docs/check/run.sh +++ b/docker/docs/check/run.sh @@ -1,6 +1,9 @@ #!/usr/bin/env bash set -euo pipefail - cd $REPO_PATH/docs/tools +mkdir venv +virtualenv -p $(which python3) venv +source venv/bin/activate +python3 -m pip install --ignore-installed -r requirements.txt ./build.py --skip-git-log 2>&1 | tee $OUTPUT_PATH/output.log diff --git a/tests/ci/docs_check.py b/tests/ci/docs_check.py index b86ab9d9871..cf1ded72488 100644 --- a/tests/ci/docs_check.py +++ b/tests/ci/docs_check.py @@ -137,8 +137,10 @@ if __name__ == "__main__": if lines: status = "failure" description = "Found errors during docs check" - else: + elif status != "failure": lines.append(("No errors found", "OK")) + else: + lines.append(("Non zero exit code", "FAIL")) s3_helper = S3Helper('https://s3.amazonaws.com') diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 672043bd7b7..377c03a9f4c 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -1,8 +1,9 @@ #!/usr/bin/env python3 +import os import urllib + import requests from unidiff import PatchSet -import os DIFF_IN_DOCUMENTATION_EXT = [".html", ".md", ".yml", ".txt", ".css", ".js", ".xml", ".ico", ".conf", ".svg", ".png", ".jpg", ".py", ".sh"] From 4c645f3ef66c345b0fc583fd8ca5def5b4d541bd Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 18:01:29 +0300 Subject: [PATCH 260/396] Add docs release --- .github/workflows/main.yml | 2 +- .github/workflows/release.yml | 47 ++++++++++ docker/docs/release/Dockerfile | 9 ++ docker/docs/release/run.sh | 9 ++ docker/images.json | 10 ++- tests/ci/docker_images_check.py | 3 +- tests/ci/docs_check.py | 2 +- tests/ci/docs_release.py | 153 ++++++++++++++++++++++++++++++++ tests/ci/pr_info.py | 52 +++++++---- 9 files changed, 263 insertions(+), 24 deletions(-) create mode 100644 .github/workflows/release.yml create mode 100644 docker/docs/release/Dockerfile create mode 100644 docker/docs/release/run.sh create mode 100644 tests/ci/docs_release.py diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index a3ce64e0701..a224f49244d 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -58,7 +58,7 @@ jobs: sudo rm -fr $TEMP_PATH DocsCheck: needs: DockerHubPush - runs-on: [self-hosted, style-checker] + runs-on: [self-hosted, func-tester] steps: - name: Download changed images uses: actions/download-artifact@v2 diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml new file mode 100644 index 00000000000..dbd945ce427 --- /dev/null +++ b/.github/workflows/release.yml @@ -0,0 +1,47 @@ +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) ||: + sudo rm -fr $TEMP_PATH diff --git a/docker/docs/release/Dockerfile b/docker/docs/release/Dockerfile new file mode 100644 index 00000000000..63765180a4c --- /dev/null +++ b/docker/docs/release/Dockerfile @@ -0,0 +1,9 @@ +# docker build -t clickhouse/docs-release . +FROM clickhouse/docs-builder + +COPY run.sh / + +ENV REPO_PATH=/repo_path +ENV OUTPUT_PATH=/output_path + +CMD ["/bin/bash", "/run.sh"] diff --git a/docker/docs/release/run.sh b/docker/docs/release/run.sh new file mode 100644 index 00000000000..3ecfd26cb44 --- /dev/null +++ b/docker/docs/release/run.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +set -euo pipefail + +cd $REPO_PATH/docs/tools +mkdir venv +virtualenv -p $(which python3) venv +source venv/bin/activate +python3 -m pip install --ignore-installed -r requirements.txt +./release.sh 2>&1 | tee tee $OUTPUT_PATH/output.log diff --git a/docker/images.json b/docker/images.json index adbaf794e71..a6cc821108e 100644 --- a/docker/images.json +++ b/docker/images.json @@ -169,11 +169,17 @@ }, "docker/docs/builder": { "name": "clickhouse/docs-builder", - "dependent": ["docker/docs/check"] + "dependent": [ + "docker/docs/check", + "docker/docs/release" + ] }, "docker/docs/check": { "name": "clickhouse/docs-check", "dependent": [] + }, + "docker/docs/release": { + "name": "clickhouse/docs-release", + "dependent": [] } - } diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 470dcc18233..d874ca422c3 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -193,8 +193,9 @@ if __name__ == "__main__": changed_images, dockerhub_repo_name = get_changed_docker_images(pr_info, repo_path, "docker/images.json") logging.info("Has changed images %s", ', '.join([str(image[0]) for image in changed_images])) pr_commit_version = str(pr_info.number) + '-' + pr_info.sha - versions = [str(pr_info.number), pr_commit_version] + if pr_info.number == 0: + versions.append("latest") subprocess.check_output("docker login --username 'robotclickhouse' --password '{}'".format(dockerhub_password), shell=True) diff --git a/tests/ci/docs_check.py b/tests/ci/docs_check.py index cf1ded72488..a99641537c5 100644 --- a/tests/ci/docs_check.py +++ b/tests/ci/docs_check.py @@ -111,7 +111,7 @@ if __name__ == "__main__": retcode = process.wait() if retcode == 0: logging.info("Run successfully") - status = "Success" + status = "success" description = "Run Ok" else: description = "Run failed (non zero exit code)" diff --git a/tests/ci/docs_release.py b/tests/ci/docs_release.py new file mode 100644 index 00000000000..be6915f3cc9 --- /dev/null +++ b/tests/ci/docs_release.py @@ -0,0 +1,153 @@ +#!/usr/bin/env python3 + +#!/usr/bin/env python3 +import logging +import subprocess +import os +import time +import json +import sys +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 + +NAME = "Docs Release (actions)" + +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, additional_files): + s3_path_prefix = f"{pr_number}/{commit_sha}/docs_release" + 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(NAME, test_results, raw_log_url, task_url, branch_url, branch_name, commit_url, additional_urls) + 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 + +if __name__ == "__main__": + logging.basicConfig(level=logging.INFO) + + temp_path = os.path.join(os.getenv("TEMP_PATH")) + repo_path = os.path.join(os.getenv("REPO_COPY")) + + with open(os.getenv('GITHUB_EVENT_PATH'), 'r', encoding='utf-8') as event_file: + event = json.load(event_file) + + pr_info = PRInfo(event, need_changed_files=True) + + gh = Github(get_best_robot_token()) + if not pr_info.has_changes_in_documentation(): + logging.info ("No changes in documentation") + commit = get_commit(gh, pr_info.sha) + commit.create_status(context=NAME, description="No changes in docs", state="success") + sys.exit(0) + + logging.info("Has changes in docs") + + if not os.path.exists(temp_path): + os.makedirs(temp_path) + + images_path = os.path.join(temp_path, 'changed_images.json') + + docker_image = 'clickhouse/docs-release' + if 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 'clickhouse/docs-release' in images: + docker_image += ':' + images['clickhouse/docs-release'] + + logging.info("Got docker image %s", docker_image) + for i in range(10): + try: + subprocess.check_output(f"docker pull {docker_image}", 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_image}") + + test_output = os.path.join(temp_path, 'docs_release_log') + if not os.path.exists(test_output): + os.makedirs(test_output) + + token = get_parameter_from_ssm('cloudflare_token', decrypt=True) + 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: + with subprocess.Popen(cmd, shell=True, stderr=log, stdout=log) as process: + retcode = process.wait() + if retcode == 0: + logging.info("Run successfully") + status = "success" + description = "Run Ok" + else: + description = "Run failed (non zero exit code)" + status = "failure" + logging.info("Run failed") + + subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) + files = os.listdir(test_output) + lines = [] + additional_files = [] + if not files: + logging.error("No output files after docs release") + description = "No output files after docs release" + status = "failure" + else: + for f in files: + path = os.path.join(test_output, f) + additional_files.append(path) + with open(path, 'r', encoding='utf-8') as check_file: + for line in check_file: + if "ERROR" in line: + lines.append((line.split(':')[-1], "FAIL")) + if lines: + status = "failure" + description = "Found errors during docs release" + elif status != "failure": + lines.append(("No errors found", "OK")) + else: + lines.append(("Non zero exit code", "FAIL")) + + s3_helper = S3Helper('https://s3.amazonaws.com') + + 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 377c03a9f4c..14a97e510a2 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -10,27 +10,41 @@ 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): - self.number = github_event['number'] - if 'after' in github_event: + if 'pull_request' in github_event: # pull request and other similar events + self.number = github_event['number'] + if 'after' in github_event: + self.sha = github_event['after'] + else: + self.sha = github_event['pull_request']['head']['sha'] + + self.labels = { l['name'] for l in github_event['pull_request']['labels'] } + self.user_login = github_event['pull_request']['user']['login'] + self.user_orgs = set([]) + if need_orgs: + user_orgs_response = requests.get(github_event['pull_request']['user']['organizations_url']) + if user_orgs_response.ok: + response_json = user_orgs_response.json() + self.user_orgs = set(org['id'] for org in response_json) + + self.changed_files = set([]) + if need_changed_files: + diff_url = github_event['pull_request']['diff_url'] + 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 + self.number = 0 self.sha = github_event['after'] + 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}') + if 'files' in diff: + self.changed_files = [f['filename'] for f in diff['files']] + else: + self.changed_files = set([]) else: - self.sha = github_event['pull_request']['head']['sha'] - - self.labels = { l['name'] for l in github_event['pull_request']['labels'] } - self.user_login = github_event['pull_request']['user']['login'] - self.user_orgs = set([]) - if need_orgs: - user_orgs_response = requests.get(github_event['pull_request']['user']['organizations_url']) - if user_orgs_response.ok: - response_json = user_orgs_response.json() - self.user_orgs = set(org['id'] for org in response_json) - - self.changed_files = set([]) - if need_changed_files: - diff_url = github_event['pull_request']['diff_url'] - 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 } + raise Exception("Unknown event type") def get_dict(self): return { From 683a8cf79940d0e604b3ce95a09f7f9cdf57f063 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 20:40:28 +0300 Subject: [PATCH 261/396] Better messages --- tests/ci/docs_check.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/docs_check.py b/tests/ci/docs_check.py index a99641537c5..aece781a703 100644 --- a/tests/ci/docs_check.py +++ b/tests/ci/docs_check.py @@ -112,9 +112,9 @@ if __name__ == "__main__": if retcode == 0: logging.info("Run successfully") status = "success" - description = "Run Ok" + description = "Docs check passed" else: - description = "Run failed (non zero exit code)" + description = "Docs check failed (non zero exit code)" status = "failure" logging.info("Run failed") @@ -136,7 +136,7 @@ if __name__ == "__main__": lines.append((line.split(':')[-1], "FAIL")) if lines: status = "failure" - description = "Found errors during docs check" + description = "Found errors in docs" elif status != "failure": lines.append(("No errors found", "OK")) else: From e6084ab893c8ec02033146e5023921082304a300 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 20:41:43 +0300 Subject: [PATCH 262/396] More checks --- tests/ci/docs_release.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/docs_release.py b/tests/ci/docs_release.py index be6915f3cc9..5a4c9537c06 100644 --- a/tests/ci/docs_release.py +++ b/tests/ci/docs_release.py @@ -115,9 +115,9 @@ if __name__ == "__main__": if retcode == 0: logging.info("Run successfully") status = "success" - description = "Run Ok" + description = "Released successfuly" else: - description = "Run failed (non zero exit code)" + description = "Release failed (non zero exit code)" status = "failure" logging.info("Run failed") @@ -139,7 +139,7 @@ if __name__ == "__main__": lines.append((line.split(':')[-1], "FAIL")) if lines: status = "failure" - description = "Found errors during docs release" + description = "Found errors in docs" elif status != "failure": lines.append(("No errors found", "OK")) else: From d872eae809228a680c37c443ed479f3f99569374 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 21:42:42 +0300 Subject: [PATCH 263/396] Better --- docker/docs/builder/Dockerfile | 1 + tests/ci/docs_release.py | 4 +- tests/ci/ssh.py | 117 +++++++++++++++++++++++++++++++++ 3 files changed, 120 insertions(+), 2 deletions(-) create mode 100644 tests/ci/ssh.py diff --git a/docker/docs/builder/Dockerfile b/docker/docs/builder/Dockerfile index c47554808cb..8afddefa41a 100644 --- a/docker/docs/builder/Dockerfile +++ b/docker/docs/builder/Dockerfile @@ -30,6 +30,7 @@ RUN apt-get update \ nodejs \ npm \ openjdk-11-jdk \ + ssh-client \ && pip --no-cache-dir install scipy \ && apt-get autoremove --yes \ && apt-get clean \ diff --git a/tests/ci/docs_release.py b/tests/ci/docs_release.py index 5a4c9537c06..dfb3d50f1d8 100644 --- a/tests/ci/docs_release.py +++ b/tests/ci/docs_release.py @@ -12,6 +12,7 @@ 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 ssh import SSHKey NAME = "Docs Release (actions)" @@ -109,7 +110,7 @@ if __name__ == "__main__": run_log_path = os.path.join(test_output, 'runlog.log') - with open(run_log_path, 'w', encoding='utf-8') as log: + with open(run_log_path, 'w', encoding='utf-8') as log, SSHKey("robot-clickhouse-ssh"): with subprocess.Popen(cmd, shell=True, stderr=log, stdout=log) as process: retcode = process.wait() if retcode == 0: @@ -149,5 +150,4 @@ 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/ssh.py b/tests/ci/ssh.py new file mode 100644 index 00000000000..3e0cca31664 --- /dev/null +++ b/tests/ci/ssh.py @@ -0,0 +1,117 @@ +#!/usr/bin/env python3 + +import shutil +import os +import subprocess +import tempfile +import logging +import signal +from get_robot_token import get_parameter_from_ssm + + +class SSHAgent: + def __init__(self): + self._env = {} + self._env_backup = {} + self._keys = {} + self.start() + + @property + def pid(self): + return int(self._env["SSH_AGENT_PID"]) + + def start(self): + if shutil.which("ssh-agent") is None: + raise Exception("ssh-agent binary is not available") + + self._env_backup["SSH_AUTH_SOCK"] = os.environ.get("SSH_AUTH_SOCK") + self._env_backup["SSH_OPTIONS"] = os.environ.get("SSH_OPTIONS") + + # set ENV from stdout of ssh-agent + for line in self._run(['ssh-agent']).splitlines(): + name, _, value = line.partition(b"=") + if _ == b"=": + value = value.split(b";", 1)[0] + self._env[name.decode()] = value.decode() + os.environ[name.decode()] = value.decode() + + ssh_options = "," + os.environ["SSH_OPTIONS"] if os.environ.get("SSH_OPTIONS") else "" + os.environ["SSH_OPTIONS"] = f"{ssh_options}UserKnownHostsFile=/dev/null,StrictHostKeyChecking=no" + + def add(self, key): + key_pub = self._key_pub(key) + + if key_pub in self._keys: + self._keys[key_pub] += 1 + else: + self._run(["ssh-add", "-"], stdin=key.encode()) + self._keys[key_pub] = 1 + + return key_pub + + def remove(self, key_pub): + if key_pub not in self._keys: + raise Exception(f"Private key not found, public part: {key_pub}") + + if self._keys[key_pub] > 1: + self._keys[key_pub] -= 1 + else: + with tempfile.NamedTemporaryFile() as f: + f.write(key_pub) + f.flush() + self._run(["ssh-add", "-d", f.name]) + self._keys.pop(key_pub) + + def print_keys(self): + keys = self._run(["ssh-add", "-l"]).splitlines() + if keys: + logging.info("ssh-agent keys:") + for key in keys: + logging.info("%s", key) + else: + logging.info("ssh-agent (pid %d) is empty", self.pid) + + def kill(self): + for k, v in self._env.items(): + os.environ.pop(k, None) + + for k, v in self._env_backup.items(): + if v is not None: + os.environ[k] = v + + os.kill(self.pid, signal.SIGTERM) + + def _key_pub(self, key): + with tempfile.NamedTemporaryFile() as f: + f.write(key.encode()) + f.flush() + return self._run(["ssh-keygen", "-y", "-f", f.name]) + + @staticmethod + def _run(cmd, stdin=None): + shell = isinstance(cmd, str) + with subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.PIPE, stdin=subprocess.PIPE if stdin else None, shell=shell) as p: + stdout, stderr = p.communicate(stdin) + + if stdout.strip().decode() == "The agent has no identities.": + return "" + + if p.returncode: + message = stderr.strip() + b"\n" + stdout.strip() + raise Exception(message.strip().decode()) + + return stdout + +class SSHKey: + def __init__(self, key_name): + self.key = get_parameter_from_ssm(key_name) + self._key_pub = None + self._ssh_agent = SSHAgent() + + def __enter__(self): + self._key_pub = self._ssh_agent.add(self.key) + self._ssh_agent.print_keys() + + def __exit__(self, exc_type, exc_val, exc_tb): + self._ssh_agent.remove(self._key_pub) + self._ssh_agent.print_keys() From 6bd53e9a4f09371fb9d87ee7eef877523aead2ab Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 21:44:36 +0300 Subject: [PATCH 264/396] Additional cleanup after checks --- .github/workflows/main.yml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index a224f49244d..3d1c9730f99 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -55,6 +55,7 @@ jobs: if: always() run: | docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH DocsCheck: needs: DockerHubPush @@ -79,6 +80,7 @@ jobs: if: always() run: | docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH BuilderDebDebug: needs: DockerHubPush @@ -117,6 +119,7 @@ jobs: if: always() run: | docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH BuilderReport: needs: [BuilderDebDebug] @@ -142,6 +145,7 @@ jobs: if: always() run: | docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH FunctionalStatelessTestDebug: needs: [BuilderDebDebug] @@ -171,6 +175,7 @@ jobs: if: always() run: | docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH FunctionalStatefulTestDebug: needs: [BuilderDebDebug] @@ -200,6 +205,7 @@ jobs: if: always() run: | docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH FastTest: needs: DockerHubPush @@ -222,6 +228,7 @@ jobs: if: always() run: | docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH FinishCheck: needs: [StyleCheck, DockerHubPush, CheckLabels, BuilderReport, FastTest, FunctionalStatelessTestDebug, FunctionalStatefulTestDebug] From e83f47c4b3750be67e031aa83350fa130128f7d0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 21:46:50 +0300 Subject: [PATCH 265/396] Add cleanup of docker images --- .github/workflows/main.yml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 69dc2e94e15..b738cf96b46 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -56,6 +56,7 @@ jobs: if: always() run: | docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH BuilderDebDebug: needs: DockerHubPush @@ -93,6 +94,7 @@ jobs: if: always() run: | docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH BuilderReport: needs: [BuilderDebDebug] @@ -118,6 +120,7 @@ jobs: if: always() run: | docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH FunctionalStatelessTestDebug: needs: [BuilderDebDebug] @@ -147,6 +150,7 @@ jobs: if: always() run: | docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH FunctionalStatefulTestDebug: needs: [BuilderDebDebug] @@ -176,6 +180,7 @@ jobs: if: always() run: | docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH FastTest: needs: DockerHubPush @@ -197,6 +202,7 @@ jobs: if: always() run: | docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH FinishCheck: needs: [StyleCheck, DockerHubPush, CheckLabels, BuilderReport, FastTest, FunctionalStatelessTestDebug, FunctionalStatefulTestDebug] From 4437d686406b5eecaa2859b0a60f6498acdef571 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Fri, 29 Oct 2021 21:48:39 +0300 Subject: [PATCH 266/396] Apply suggestions from code review Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/en/operations/settings/settings.md | 4 ++-- docs/ru/operations/settings/settings.md | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 043ceebff0d..81406b080d3 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1399,7 +1399,7 @@ Default value: `3`. ## compile_aggregate_expressions {#compile_aggregate_expressions} -Enables or disables JIT-compilation of aggregate functions to native code. Enabling the `compile_aggregate_expressions` setting can improve the performance. +Enables or disables JIT-compilation of aggregate functions to native code. Enabling this setting can improve the performance. Possible values: @@ -1414,7 +1414,7 @@ Default value: `0`. ## min_count_to_compile_aggregate_expression {#min_count_to_compile_aggregate_expression} -If the [compile_aggregate_expressions](#compile_aggregate_expressions) setting is enabled and there are more then `min_count_to_compile_aggregate_expression` identical aggregate expressions in a `SELECT` query, then they are compiled JIT. +If the [compile_aggregate_expressions](#compile_aggregate_expressions) setting is enabled and there are more than `min_count_to_compile_aggregate_expression` identical aggregate expressions in a `SELECT` query, then they are JIT-compiled. Possible values: diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index e6f683b4991..f0c9211c34d 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1363,7 +1363,7 @@ load_balancing = round_robin ## compile_aggregate_expressions {#compile_aggregate_expressions} -Включает или отключает компиляцию агрегированных функций в нативный код во время выполнения запроса. Включение настройки `compile_aggregate_expressions` может улучшить производительность выполнения запросов. +Включает или отключает компиляцию агрегированных функций в нативный код во время выполнения запроса. Включение этой настройки может улучшить производительность выполнения запросов. Возможные значения: From ef57f6d36347c20f92fb3b001fb95c8346610e4b Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Fri, 29 Oct 2021 19:08:17 +0300 Subject: [PATCH 267/396] Fixed `--disable-net-host`. --- tests/integration/runner | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index 10c940afb46..86a254b26f5 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -150,8 +150,7 @@ if __name__ == "__main__": parser.add_argument( "--network", - default='host', - help="Set network driver for runnner container") + help="Set network driver for runnner container (defaults to `host`)") parser.add_argument( "--docker-image-version", From dcbef00cd0ee8fd9b4e6086ad0a96eea0e704019 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 21:55:32 +0300 Subject: [PATCH 268/396] Merged correctly --- .github/workflows/main.yml | 24 ------------------------ 1 file changed, 24 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 9c19ca19ea2..3d1c9730f99 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -82,30 +82,6 @@ jobs: docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH - DocsCheck: - needs: DockerHubPush - runs-on: [self-hosted, func-tester] - steps: - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ runner.temp }}/docs_check - - name: Check out repository code - uses: actions/checkout@v2 - - name: Docs Check - env: - TEMP_PATH: ${{runner.temp}}/docs_check - REPO_COPY: ${{runner.temp}}/docs_check/ClickHouse - run: | - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 docs_check.py - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - sudo rm -fr $TEMP_PATH BuilderDebDebug: needs: DockerHubPush if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} From 40616a9cbb2e3ee176fe1106eed47fb03ac317d2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 Oct 2021 21:57:17 +0300 Subject: [PATCH 269/396] More angry kill --- .github/workflows/release.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index dbd945ce427..f1b100b7795 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -44,4 +44,5 @@ jobs: if: always() run: | docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH From 7fb7de54fde5595bbdf12206a0c1eeff999bf675 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Fri, 29 Oct 2021 21:59:54 +0300 Subject: [PATCH 270/396] Apply suggestions from code review --- 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 f0c9211c34d..b3815e68283 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1378,7 +1378,7 @@ load_balancing = round_robin ## min_count_to_compile_aggregate_expression {#min_count_to_compile_aggregate_expression} -Если настройка [compile_aggregate_expressions](#compile_aggregate_expressions) включена и в ходе выполнения одного запроса агрегированная функция выполняется более `min_count_to_compile_aggregate_expression` раз для одного и того же выражения, тогда эта агрегированная функция компилируется в нативный код. +Минимальное количество вызовов агрегатной функции с одинаковым выражением, при котором функция будет компилироваться в нативный код в ходе выполнения запроса. Работает только если включена настройка [compile_aggregate_expressions](#compile_aggregate_expressions). Возможные значения: From e22bf9ef0614d0ca52885754f66d2724b436b4e3 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Fri, 29 Oct 2021 22:00:04 +0300 Subject: [PATCH 271/396] Apply suggestions from code review --- 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 b3815e68283..c26ce7a2939 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1367,8 +1367,8 @@ load_balancing = round_robin Возможные значения: -- 0 — агрегированные функции не компилируются в нативный код. -- 1 — агрегированные функции компилируются в нативный код в процессе выполнения запроса. +- 0 — агрегатные функции не компилируются в нативный код. +- 1 — агрегатные функции компилируются в нативный код в процессе выполнения запроса. Значение по умолчанию: `0`. From 00692b59d3bacbdbbfdf1aacf6e32125087cf31a Mon Sep 17 00:00:00 2001 From: olgarev Date: Fri, 29 Oct 2021 19:07:00 +0000 Subject: [PATCH 272/396] Corrections --- docs/en/operations/settings/settings.md | 4 ++-- docs/ru/operations/settings/settings.md | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 81406b080d3..9df1951972f 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1414,12 +1414,12 @@ Default value: `0`. ## min_count_to_compile_aggregate_expression {#min_count_to_compile_aggregate_expression} -If the [compile_aggregate_expressions](#compile_aggregate_expressions) setting is enabled and there are more than `min_count_to_compile_aggregate_expression` identical aggregate expressions in a `SELECT` query, then they are JIT-compiled. +The minimum number of identical aggregate expressions to start JIT-compilation. Works only if the [compile_aggregate_expressions](#compile_aggregate_expressions) setting is enabled. Possible values: - Positive integer. -- 0 — Identical aggregate expressions are always JIT-compiled (if `compile_aggregate_expressions = 1`). +- 0 — Identical aggregate expressions are always JIT-compiled. Default value: `0`. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index c26ce7a2939..d5b102e53ab 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1363,7 +1363,7 @@ load_balancing = round_robin ## compile_aggregate_expressions {#compile_aggregate_expressions} -Включает или отключает компиляцию агрегированных функций в нативный код во время выполнения запроса. Включение этой настройки может улучшить производительность выполнения запросов. +Включает или отключает компиляцию агрегатных функций в нативный код во время выполнения запроса. Включение этой настройки может улучшить производительность выполнения запросов. Возможные значения: @@ -1383,7 +1383,7 @@ load_balancing = round_robin Возможные значения: - Целое положительное число. -- 0 — агрегированные функциии для одинаковых выражений всегда компилируются в ходе выполнения запроса (если `compile_aggregate_expressions = 1`). +- 0 — агрегатные функциии всегда компилируются в ходе выполнения запроса. Значение по умолчанию: `0`. From a5cd1aa29496480ff4032bba2a20f93961d45956 Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Fri, 29 Oct 2021 22:24:14 +0300 Subject: [PATCH 273/396] Apply suggestions from code review Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/tuple-functions.md | 2 +- docs/ru/sql-reference/functions/tuple-functions.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index faa7b91f960..b72b75d6de6 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -178,7 +178,7 @@ tupleToNameValuePairs(tuple) **Arguments** -- `tuple` — Named tuple. [Tuple](../../sql-reference/data-types/tuple.md) with any type of values. +- `tuple` — Named tuple. [Tuple](../../sql-reference/data-types/tuple.md) with any types of values. **Returned value** diff --git a/docs/ru/sql-reference/functions/tuple-functions.md b/docs/ru/sql-reference/functions/tuple-functions.md index 68649500116..006557d859f 100644 --- a/docs/ru/sql-reference/functions/tuple-functions.md +++ b/docs/ru/sql-reference/functions/tuple-functions.md @@ -205,7 +205,7 @@ SELECT tupleToNameValuePairs(col) FROM tupletest; └───────────────────────────────────────┘ ``` -С помощью этой функции возможно приводить столбцы к строкам: +С помощью этой функции можно выводить столбцы в виде строк: ``` sql CREATE TABLE tupletest (`col` Tuple(CPU Float64, Memory Float64, Disk Float64)) ENGINE = Memory; @@ -225,7 +225,7 @@ SELECT arrayJoin(tupleToNameValuePairs(col))FROM tupletest; └───────────────────────────────────────┘ ``` -При передаче функции обычный кортеж ClickHouse использует индексы значений как их имена: +Если в функцию передается обычный кортеж, ClickHouse использует индексы значений в качестве имен: ``` sql SELECT tupleToNameValuePairs(tuple(3, 2, 1)); From 6dd605cc13885251df277096727f23201eaa96ca Mon Sep 17 00:00:00 2001 From: Teja Date: Fri, 29 Oct 2021 21:26:44 +0200 Subject: [PATCH 274/396] update out statement --- programs/install/Install.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 57e56c8b9ea..d3a9792a95b 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -814,7 +814,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) "\nClickHouse has been successfully installed.\n" "\nRestart clickhouse-server with:\n" " sudo clickhouse restart\n" - "\nStart clickhouse-client with:\n" + "\nRestart clickhouse-client with:\n" " clickhouse-client{}\n\n", maybe_password); } else { From 68e771f849f675a58fdca070ce1c852c4f14b959 Mon Sep 17 00:00:00 2001 From: Teja Date: Fri, 29 Oct 2021 21:30:55 +0200 Subject: [PATCH 275/396] Update Install.cpp --- programs/install/Install.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index d3a9792a95b..57e56c8b9ea 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -814,7 +814,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) "\nClickHouse has been successfully installed.\n" "\nRestart clickhouse-server with:\n" " sudo clickhouse restart\n" - "\nRestart clickhouse-client with:\n" + "\nStart clickhouse-client with:\n" " clickhouse-client{}\n\n", maybe_password); } else { From 89bfb1cb4fe7743cb1eedf15e8fb46933db94a5e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 Oct 2021 22:42:58 +0300 Subject: [PATCH 276/396] Rename Mac architecture on Quick Start page --- website/templates/index/quickstart.html | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/website/templates/index/quickstart.html b/website/templates/index/quickstart.html index 256340b731a..28c2a28f17b 100644 --- a/website/templates/index/quickstart.html +++ b/website/templates/index/quickstart.html @@ -20,10 +20,10 @@ Linux (ARM) From 00be07b9af976c44a97eb7c47fb84d555f410c27 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 30 Oct 2021 01:09:07 +0300 Subject: [PATCH 282/396] 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 283/396] 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 284/396] 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 285/396] 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 286/396] 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 287/396] 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 288/396] 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 289/396] 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 290/396] 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 291/396] 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 723df5c04a3105a68270644632138ff3c7e01cf5 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sat, 30 Oct 2021 16:00:57 +0300 Subject: [PATCH 292/396] Update merge-tree-settings.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил описание настройки. --- docs/en/operations/settings/merge-tree-settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 1a723f4efb0..7159ae7f697 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -331,7 +331,7 @@ By default, the ClickHouse server checks at table creation the data type of a co ## min_bytes_to_rebalance_partition_over_jbod {#min-bytes-to-rebalance-partition-over-jbod} -Allows assigning new parts to different disks of a [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures) volume in a balanced way. +Sets minimal amount of bytes to enable parts rebalance over [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures) array. Allows assigning new parts to different disks of a `JBOD` volume in a balanced way. Possible values: @@ -341,4 +341,4 @@ Default value: `0`. **Usage** -Throw exception if the value of the `min_bytes_to_rebalance_partition_over_jbod` setting should be less than the value of the [max_bytes_to_merge_at_max_space_in_pool](../../operations/settings/merge-tree-settings.md#max-bytes-to-merge-at-max-space-in-pool) setting. +The value of the `min_bytes_to_rebalance_partition_over_jbod` setting should be less than the value of the [max_bytes_to_merge_at_max_space_in_pool](../../operations/settings/merge-tree-settings.md#max-bytes-to-merge-at-max-space-in-pool) setting. Otherwise, ClickHouse throws an exception. From 30ab4b3a7fd9593cc1ea6ccb0b99d05e2f24758a Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 30 Oct 2021 16:02:05 +0300 Subject: [PATCH 293/396] 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 294/396] 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 295/396] 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 296/396] 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 297/396] 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 298/396] 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 299/396] 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 300/396] 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 301/396] 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 302/396] 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 303/396] 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 304/396] 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 305/396] 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 306/396] 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 307/396] 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 308/396] 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 309/396] 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 310/396] 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 311/396] 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 312/396] 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 313/396] 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 314/396] 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 315/396] 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 316/396] 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 317/396] 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 318/396] 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 319/396] 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 320/396] 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 aff2711f1a682e9a461f91fe70fc9874293a1184 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sun, 31 Oct 2021 19:22:20 +0300 Subject: [PATCH 321/396] support INTERVAL type in WITH FILL --- src/Common/IntervalKind.cpp | 1 - src/Common/IntervalKind.h | 11 ++ src/Core/SortDescription.h | 7 +- src/Interpreters/FillingRow.cpp | 4 +- src/Interpreters/InterpreterSelectQuery.cpp | 19 ++- .../Transforms/FillingTransform.cpp | 152 +++++++++++++----- .../02112_with_fill_interval.reference | 88 ++++++++++ .../0_stateless/02112_with_fill_interval.sql | 61 +++++++ 8 files changed, 296 insertions(+), 47 deletions(-) create mode 100644 tests/queries/0_stateless/02112_with_fill_interval.reference create mode 100644 tests/queries/0_stateless/02112_with_fill_interval.sql diff --git a/src/Common/IntervalKind.cpp b/src/Common/IntervalKind.cpp index ee4b67469ce..69b56be48ac 100644 --- a/src/Common/IntervalKind.cpp +++ b/src/Common/IntervalKind.cpp @@ -25,7 +25,6 @@ Int32 IntervalKind::toAvgSeconds() const __builtin_unreachable(); } - IntervalKind IntervalKind::fromAvgSeconds(Int64 num_seconds) { if (num_seconds) diff --git a/src/Common/IntervalKind.h b/src/Common/IntervalKind.h index 85372ffa9ab..f1b5dce0792 100644 --- a/src/Common/IntervalKind.h +++ b/src/Common/IntervalKind.h @@ -58,4 +58,15 @@ struct IntervalKind /// For example, `IntervalKind::tryParseString('second', result)` returns `result` equals `IntervalKind::Kind::Second`. static bool tryParseString(const std::string & kind, IntervalKind::Kind & result); }; + +#define FOR_EACH_INTERVAL_KIND(M) \ + M(Second) \ + M(Minute) \ + M(Hour) \ + M(Day) \ + M(Week) \ + M(Month) \ + M(Quarter) \ + M(Year) + } diff --git a/src/Core/SortDescription.h b/src/Core/SortDescription.h index e1653b9102b..db15f3a54db 100644 --- a/src/Core/SortDescription.h +++ b/src/Core/SortDescription.h @@ -6,6 +6,7 @@ #include #include #include +#include class Collator; @@ -27,7 +28,11 @@ struct FillColumnDescription /// Range [FROM, TO) respects sorting direction Field fill_from; /// Fill value >= FILL_FROM Field fill_to; /// Fill value + STEP < FILL_TO - Field fill_step; /// Default = 1 or -1 according to direction + Field fill_step; /// Default = +1 or -1 according to direction + std::optional step_kind; + + using StepFunction = std::function; + StepFunction step_func; }; /// Description of the sorting rule by one column. diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 4bbb8974fe9..89194a9738a 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -64,7 +64,7 @@ bool FillingRow::next(const FillingRow & to_row) continue; auto next_value = row[i]; - applyVisitor(FieldVisitorSum(getFillDescription(i).fill_step), next_value); + getFillDescription(pos).step_func(next_value); if (less(next_value, getFillDescription(i).fill_to, getDirection(i))) { row[i] = next_value; @@ -74,7 +74,7 @@ bool FillingRow::next(const FillingRow & to_row) } auto next_value = row[pos]; - applyVisitor(FieldVisitorSum(getFillDescription(pos).fill_step), next_value); + getFillDescription(pos).step_func(next_value); if (less(to_row[pos], next_value, getDirection(pos))) return false; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 0ed7df2d18f..3af4955e017 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -705,7 +706,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() static Field getWithFillFieldValue(const ASTPtr & node, ContextPtr context) { - const auto & [field, type] = evaluateConstantExpression(node, context); + auto [field, type] = evaluateConstantExpression(node, context); if (!isColumnedAsNumber(type)) throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); @@ -713,6 +714,19 @@ static Field getWithFillFieldValue(const ASTPtr & node, ContextPtr context) return field; } +static std::pair> getWithFillStep(const ASTPtr & node, ContextPtr context) +{ + auto [field, type] = evaluateConstantExpression(node, context); + + if (const auto * type_interval = typeid_cast(type.get())) + return std::make_pair(std::move(field), type_interval->getKind()); + + if (isColumnedAsNumber(type)) + return std::make_pair(std::move(field), std::nullopt); + + throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); +} + static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, ContextPtr context) { FillColumnDescription descr; @@ -720,8 +734,9 @@ static FillColumnDescription getWithFillDescription(const ASTOrderByElement & or descr.fill_from = getWithFillFieldValue(order_by_elem.fill_from, context); if (order_by_elem.fill_to) descr.fill_to = getWithFillFieldValue(order_by_elem.fill_to, context); + if (order_by_elem.fill_step) - descr.fill_step = getWithFillFieldValue(order_by_elem.fill_step, context); + std::tie(descr.fill_step, descr.step_kind) = getWithFillStep(order_by_elem.fill_step, context); else descr.fill_step = order_by_elem.direction; diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 3ff89c302ff..ae1717b9e21 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -5,6 +5,9 @@ #include #include #include +#include +#include +#include namespace DB @@ -29,6 +32,113 @@ Block FillingTransform::transformHeader(Block header, const SortDescription & so return header; } +template +static FillColumnDescription::StepFunction getStepFunction( + IntervalKind kind, Int64 step, const DateLUTImpl & date_lut) +{ + switch (kind) + { + #define DECLARE_CASE(NAME) \ + case IntervalKind::NAME: \ + return [step, &date_lut](Field & field) { field = Add##NAME##sImpl::execute(get(field), step, date_lut); }; + + FOR_EACH_INTERVAL_KIND(DECLARE_CASE) + #undef DECLARE_CASE + } +} + +static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & type) +{ + auto max_type = Field::Types::Null; + WhichDataType which(type); + DataTypePtr to_type; + + /// TODO Wrong results for big integers. + if (isInteger(type) || which.isDate() || which.isDate32() || which.isDateTime()) + { + max_type = Field::Types::Int64; + to_type = std::make_shared(); + } + else if (which.isDateTime64()) + { + max_type = Field::Types::Decimal64; + const auto & date_type = static_cast(*type); + size_t precision = date_type.getPrecision(); + size_t scale = date_type.getScale(); + to_type = std::make_shared>(precision, scale); + } + else if (which.isFloat()) + { + max_type = Field::Types::Float64; + to_type = std::make_shared(); + } + else + return false; + + if (descr.fill_from.getType() > max_type + || descr.fill_to.getType() > max_type + || descr.fill_step.getType() > max_type) + return false; + + descr.fill_from = convertFieldToType(descr.fill_from, *to_type); + descr.fill_to = convertFieldToType(descr.fill_to, *to_type); + descr.fill_step = convertFieldToType(descr.fill_step, *to_type); + + if (descr.step_kind) + { + if (which.isDate() || which.isDate32()) + { + Int64 avg_seconds = get(descr.fill_step) * descr.step_kind->toAvgSeconds(); + if (avg_seconds < 86400) + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "Value of step is to low ({} seconds). Must be >= 1 day", avg_seconds); + } + + if (which.isDate()) + descr.step_func = getStepFunction(*descr.step_kind, get(descr.fill_step), DateLUT::instance()); + else if (which.isDate32()) + descr.step_func = getStepFunction(*descr.step_kind, get(descr.fill_step), DateLUT::instance()); + else if (const auto * date_time = checkAndGetDataType(type.get())) + descr.step_func = getStepFunction(*descr.step_kind, get(descr.fill_step), date_time->getTimeZone()); + else if (const auto * date_time64 = checkAndGetDataType(type.get())) + { + const auto & time_zone = date_time64->getTimeZone(); + const auto & step_dec = get &>(descr.fill_step); + Int64 step = DecimalUtils::convertTo(step_dec.getValue(), step_dec.getScale()); + + switch (*descr.step_kind) + { + #define DECLARE_CASE(NAME) \ + case IntervalKind::NAME: \ + descr.step_func = [step, &time_zone](Field & field) \ + { \ + auto field_decimal = get>(field); \ + auto components = DecimalUtils::splitWithScaleMultiplier(field_decimal.getValue(), field_decimal.getScaleMultiplier()); \ + auto res = Add##NAME##sImpl::execute(components, step, time_zone); \ + auto res_decimal = decimalFromComponentsWithMultiplier(res, field_decimal.getScaleMultiplier()); \ + field = DecimalField(res_decimal, field_decimal.getScale()); \ + }; \ + break; + + FOR_EACH_INTERVAL_KIND(DECLARE_CASE) + #undef DECLARE_CASE + } + } + else + throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION, + "STEP of Interval type can be used only with Date/DateTime types, but got {}", type->getName()); + } + else + { + descr.step_func = [step = descr.fill_step](Field & field) + { + applyVisitor(FieldVisitorSum(step), field); + }; + } + + return true; +} + FillingTransform::FillingTransform( const Block & header_, const SortDescription & sort_description_, bool on_totals_) : ISimpleTransform(header_, transformHeader(header_, sort_description_), true) @@ -40,46 +150,6 @@ FillingTransform::FillingTransform( if (on_totals) return; - auto try_convert_fields = [](auto & descr, const auto & type) - { - auto max_type = Field::Types::Null; - WhichDataType which(type); - DataTypePtr to_type; - - /// TODO Wrong results for big integers. - if (isInteger(type) || which.isDate() || which.isDate32() || which.isDateTime()) - { - max_type = Field::Types::Int64; - to_type = std::make_shared(); - } - else if (which.isDateTime64()) - { - max_type = Field::Types::Decimal64; - const auto & date_type = static_cast(*type); - size_t precision = date_type.getPrecision(); - size_t scale = date_type.getScale(); - to_type = std::make_shared>(precision, scale); - } - else if (which.isFloat()) - { - max_type = Field::Types::Float64; - to_type = std::make_shared(); - } - else - return false; - - if (descr.fill_from.getType() > max_type - || descr.fill_to.getType() > max_type - || descr.fill_step.getType() > max_type) - return false; - - descr.fill_from = convertFieldToType(descr.fill_from, *to_type); - descr.fill_to = convertFieldToType(descr.fill_to, *to_type); - descr.fill_step = convertFieldToType(descr.fill_step, *to_type); - - return true; - }; - std::vector is_fill_column(header_.columns()); for (size_t i = 0, size = sort_description.size(); i < size; ++i) { @@ -90,7 +160,7 @@ FillingTransform::FillingTransform( auto & descr = filling_row.getFillDescription(i); const auto & type = header_.getByPosition(block_position).type; - if (!try_convert_fields(descr, type)) + if (!tryConvertFields(descr, type)) throw Exception("Incompatible types of WITH FILL expression values with column type " + type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); diff --git a/tests/queries/0_stateless/02112_with_fill_interval.reference b/tests/queries/0_stateless/02112_with_fill_interval.reference new file mode 100644 index 00000000000..b65d91814d1 --- /dev/null +++ b/tests/queries/0_stateless/02112_with_fill_interval.reference @@ -0,0 +1,88 @@ +1 DAY +2020-02-05 1 +2020-02-06 0 +2020-02-07 0 +2020-02-08 0 +2020-02-09 0 +1 WEEK +2020-02-02 1 +2020-02-09 0 +2020-02-16 1 +2020-02-23 0 +2020-03-01 1 +1 MONTH +2020-02-01 2 +2020-03-01 1 +2020-04-01 0 +2020-05-01 0 +2020-06-01 1 +3 MONTH +2020-01-01 0 +2020-02-01 2 +2020-03-01 1 +2020-04-01 0 +2020-06-01 1 +2020-07-01 0 +2020-10-01 0 +1 DAY +2020-02-05 1 +2020-02-06 0 +2020-02-07 0 +2020-02-08 0 +2020-02-09 0 +1 WEEK +2020-02-02 1 +2020-02-09 0 +2020-02-16 1 +2020-02-23 0 +2020-03-01 1 +1 MONTH +2020-02-01 2 +2020-03-01 1 +2020-04-01 0 +2020-05-01 0 +2020-06-01 1 +3 MONTH +2020-01-01 0 +2020-02-01 2 +2020-03-01 1 +2020-04-01 0 +2020-06-01 1 +2020-07-01 0 +2020-10-01 0 +15 MINUTE +2020-02-05 10:20:00 1 +2020-02-05 10:35:00 0 +2020-02-05 10:50:00 0 +2020-02-05 11:05:00 0 +2020-02-05 11:20:00 0 +6 HOUR +2020-02-05 10:00:00 1 +2020-02-05 16:00:00 0 +2020-02-05 22:00:00 0 +2020-02-06 04:00:00 0 +2020-02-06 10:00:00 0 +10 DAY +2020-02-05 00:00:00 1 +2020-02-15 00:00:00 0 +2020-02-25 00:00:00 0 +2020-03-06 00:00:00 0 +2020-03-08 00:00:00 1 +15 MINUTE +2020-02-05 10:20:00.000 1 +2020-02-05 10:35:00.000 0 +2020-02-05 10:50:00.000 0 +2020-02-05 11:05:00.000 0 +2020-02-05 11:20:00.000 0 +6 HOUR +2020-02-05 10:00:00 1 +2020-02-05 16:00:00 0 +2020-02-05 22:00:00 0 +2020-02-06 04:00:00 0 +2020-02-06 10:00:00 0 +10 DAY +2020-02-05 00:00:00 1 +2020-02-15 00:00:00 0 +2020-02-25 00:00:00 0 +2020-03-06 00:00:00 0 +2020-03-08 00:00:00 1 diff --git a/tests/queries/0_stateless/02112_with_fill_interval.sql b/tests/queries/0_stateless/02112_with_fill_interval.sql new file mode 100644 index 00000000000..324677ca143 --- /dev/null +++ b/tests/queries/0_stateless/02112_with_fill_interval.sql @@ -0,0 +1,61 @@ +DROP TABLE IF EXISTS with_fill_date; +CREATE TABLE with_fill_date (d Date, d32 Date32) ENGINE = Memory; + +INSERT INTO with_fill_date VALUES (toDate('2020-02-05'), toDate32('2020-02-05')); +INSERT INTO with_fill_date VALUES (toDate('2020-02-16'), toDate32('2020-02-16')); +INSERT INTO with_fill_date VALUES (toDate('2020-03-03'), toDate32('2020-03-03')); +INSERT INTO with_fill_date VALUES (toDate('2020-06-10'), toDate32('2020-06-10')); + +SELECT '1 DAY'; +SELECT d, count() FROM with_fill_date GROUP BY d ORDER BY d WITH FILL STEP INTERVAL 1 DAY LIMIT 5; +SELECT '1 WEEK'; +SELECT toStartOfWeek(d) as d, count() FROM with_fill_date GROUP BY d ORDER BY d WITH FILL STEP INTERVAL 1 WEEK LIMIT 5; +SELECT '1 MONTH'; +SELECT toStartOfMonth(d) as d, count() FROM with_fill_date GROUP BY d ORDER BY d WITH FILL STEP INTERVAL 1 MONTH LIMIT 5; +SELECT '3 MONTH'; +SELECT toStartOfMonth(d) as d, count() FROM with_fill_date GROUP BY d ORDER BY d WITH FILL + FROM toDate('2020-01-01') + TO toDate('2021-01-01') + STEP INTERVAL 3 MONTH; + +SELECT d, count() FROM with_fill_date GROUP BY d ORDER BY d WITH FILL STEP INTERVAL 1 HOUR LIMIT 5; -- { serverError 475 } + +SELECT '1 DAY'; +SELECT d32, count() FROM with_fill_date GROUP BY d32 ORDER BY d32 WITH FILL STEP INTERVAL 1 DAY LIMIT 5; +SELECT '1 WEEK'; +SELECT toStartOfWeek(d32) as d32, count() FROM with_fill_date GROUP BY d32 ORDER BY d32 WITH FILL STEP INTERVAL 1 WEEK LIMIT 5; +SELECT '1 MONTH'; +SELECT toStartOfMonth(d32) as d32, count() FROM with_fill_date GROUP BY d32 ORDER BY d32 WITH FILL STEP INTERVAL 1 MONTH LIMIT 5; +SELECT '3 MONTH'; +SELECT toStartOfMonth(d32) as d32, count() FROM with_fill_date GROUP BY d32 ORDER BY d32 WITH FILL + FROM toDate('2020-01-01') + TO toDate('2021-01-01') + STEP INTERVAL 3 MONTH; + +SELECT d, count() FROM with_fill_date GROUP BY d ORDER BY d WITH FILL STEP INTERVAL 1 HOUR LIMIT 5; -- { serverError 475 } + +DROP TABLE with_fill_date; + +DROP TABLE IF EXISTS with_fill_date; +CREATE TABLE with_fill_date (d DateTime, d64 DateTime64) ENGINE = Memory; + +INSERT INTO with_fill_date VALUES (toDateTime('2020-02-05 10:20:00'), toDateTime64('2020-02-05 10:20:00', 3)); +INSERT INTO with_fill_date VALUES (toDateTime('2020-03-08 11:01:00'), toDateTime64('2020-03-08 11:01:00', 3)); + +SELECT '15 MINUTE'; +SELECT d, count() FROM with_fill_date GROUP BY d ORDER BY d WITH FILL STEP INTERVAL 15 MINUTE LIMIT 5; +SELECT '6 HOUR'; +SELECT toStartOfHour(d) as d, count() FROM with_fill_date GROUP BY d ORDER BY d WITH FILL STEP INTERVAL 6 HOUR LIMIT 5; +SELECT '10 DAY'; +SELECT toStartOfDay(d) as d, count() FROM with_fill_date GROUP BY d ORDER BY d WITH FILL STEP INTERVAL 10 DAY LIMIT 5; + +SELECT '15 MINUTE'; +SELECT d64, count() FROM with_fill_date GROUP BY d64 ORDER BY d64 WITH FILL STEP INTERVAL 15 MINUTE LIMIT 5; +SELECT '6 HOUR'; +SELECT toStartOfHour(d64) as d64, count() FROM with_fill_date GROUP BY d64 ORDER BY d64 WITH FILL STEP INTERVAL 6 HOUR LIMIT 5; +SELECT '10 DAY'; +SELECT toStartOfDay(d64) as d64, count() FROM with_fill_date GROUP BY d64 ORDER BY d64 WITH FILL STEP INTERVAL 10 DAY LIMIT 5; + +DROP TABLE with_fill_date; + +SELECT number FROM numbers(100) ORDER BY number WITH FILL STEP INTERVAL 1 HOUR; -- { serverError 475 } From b9887273b181cb87307de7d6a110bfb385afdbf9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 31 Oct 2021 20:29:54 +0300 Subject: [PATCH 322/396] 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 323/396] 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 324/396] 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 325/396] 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 326/396] 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 327/396] 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 328/396] 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 07d6013bc69823004f5847617090d0ecd3a20f0f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sun, 31 Oct 2021 22:48:12 +0300 Subject: [PATCH 329/396] fix with fill --- src/Interpreters/FillingRow.cpp | 2 +- .../02112_with_fill_interval.reference | 21 +++++++++++++++++++ .../0_stateless/02112_with_fill_interval.sql | 20 ++++++++++++++++++ 3 files changed, 42 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/FillingRow.cpp b/src/Interpreters/FillingRow.cpp index 89194a9738a..df99c0d11ed 100644 --- a/src/Interpreters/FillingRow.cpp +++ b/src/Interpreters/FillingRow.cpp @@ -64,7 +64,7 @@ bool FillingRow::next(const FillingRow & to_row) continue; auto next_value = row[i]; - getFillDescription(pos).step_func(next_value); + getFillDescription(i).step_func(next_value); if (less(next_value, getFillDescription(i).fill_to, getDirection(i))) { row[i] = next_value; diff --git a/tests/queries/0_stateless/02112_with_fill_interval.reference b/tests/queries/0_stateless/02112_with_fill_interval.reference index b65d91814d1..fc6f9378bfa 100644 --- a/tests/queries/0_stateless/02112_with_fill_interval.reference +++ b/tests/queries/0_stateless/02112_with_fill_interval.reference @@ -86,3 +86,24 @@ 2020-02-25 00:00:00 0 2020-03-06 00:00:00 0 2020-03-08 00:00:00 1 +1 MONTH +2020-01-01 1 0 +2020-01-01 2 0 +2020-01-01 3 0 +2020-01-01 4 0 +2020-02-01 1 1 +2020-02-01 2 0 +2020-02-01 3 1 +2020-02-01 4 0 +2020-03-01 1 0 +2020-03-01 2 1 +2020-03-01 3 1 +2020-03-01 4 0 +2020-04-01 1 0 +2020-04-01 2 0 +2020-04-01 3 0 +2020-04-01 4 0 +2020-05-01 1 0 +2020-05-01 2 0 +2020-05-01 3 0 +2020-05-01 4 0 diff --git a/tests/queries/0_stateless/02112_with_fill_interval.sql b/tests/queries/0_stateless/02112_with_fill_interval.sql index 324677ca143..e93a7664de0 100644 --- a/tests/queries/0_stateless/02112_with_fill_interval.sql +++ b/tests/queries/0_stateless/02112_with_fill_interval.sql @@ -59,3 +59,23 @@ SELECT toStartOfDay(d64) as d64, count() FROM with_fill_date GROUP BY d64 ORDER DROP TABLE with_fill_date; SELECT number FROM numbers(100) ORDER BY number WITH FILL STEP INTERVAL 1 HOUR; -- { serverError 475 } + +CREATE TABLE with_fill_date (d Date, id UInt32) ENGINE = Memory; + +INSERT INTO with_fill_date VALUES (toDate('2020-02-05'), 1); +INSERT INTO with_fill_date VALUES (toDate('2020-02-16'), 3); +INSERT INTO with_fill_date VALUES (toDate('2020-03-10'), 2); +INSERT INTO with_fill_date VALUES (toDate('2020-03-03'), 3); + +SELECT '1 MONTH'; + +SELECT toStartOfMonth(d) as d, id, count() FROM with_fill_date +GROUP BY d, id +ORDER BY +d WITH FILL + FROM toDate('2020-01-01') + TO toDate('2020-05-01') + STEP INTERVAL 1 MONTH, +id WITH FILL FROM 1 TO 5; + +DROP TABLE with_fill_date; From 0da1c343a0e19eb2bfbfe36cda3b7ace719267a7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 31 Oct 2021 23:47:11 +0300 Subject: [PATCH 330/396] 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 331/396] 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 332/396] 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 333/396] 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 334/396] 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 335/396] 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 336/396] 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 337/396] 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 338/396] 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 339/396] 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 340/396] 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 341/396] 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 342/396] 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 343/396] 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 344/396] 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 345/396] 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 346/396] 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 7c7c90f1b492d2c62ae10b263b5b892e42b118f9 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 1 Nov 2021 11:45:10 +0300 Subject: [PATCH 347/396] try --- tests/integration/parallel_skip.json | 261 +-------------------------- 1 file changed, 1 insertion(+), 260 deletions(-) diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index b4f368abb8e..b56264fb570 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -3,264 +3,5 @@ "test_host_ip_change/test.py::test_ip_change_drop_dns_cache", "test_host_ip_change/test.py::test_ip_change_update_dns_cache", "test_host_ip_change/test.py::test_user_access_ip_change[node0]", - "test_host_ip_change/test.py::test_user_access_ip_change[node1]", - "test_atomic_drop_table/test.py::test_atomic_delete_with_stopped_zookeeper", - "test_attach_without_fetching/test.py::test_attach_without_fetching", - "test_broken_part_during_merge/test.py::test_merge_and_part_corruption", - "test_cleanup_dir_after_bad_zk_conn/test.py::test_attach_without_zk", - "test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_bad_zk_conn", - "test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_wrong_replica_name", - "test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_wrong_zk_path", - "test_consistent_parts_after_clone_replica/test.py::test_inconsistent_parts_if_drop_while_replica_not_active", - "test_cross_replication/test.py::test", - "test_ddl_worker_non_leader/test.py::test_non_leader_replica", - "test_delayed_replica_failover/test.py::test", - "test_dictionaries_update_field/test.py::test_update_field[complex_key_hashed_update_field_dictionary-HASHED]", - "test_dictionaries_update_field/test.py::test_update_field[flat_update_field_dictionary-FLAT]", - "test_dictionaries_update_field/test.py::test_update_field[simple_key_hashed_update_field_dictionary-HASHED]", - "test_dictionary_allow_read_expired_keys/test_default_reading.py::test_default_reading", - "test_dictionary_allow_read_expired_keys/test_default_string.py::test_return_real_values", - "test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py::test_simple_dict_get_or_default", - "test_dictionary_allow_read_expired_keys/test_dict_get.py::test_simple_dict_get", - "test_disabled_mysql_server/test.py::test_disabled_mysql_server", - "test_distributed_ddl_on_cross_replication/test.py::test_alter_ddl", - "test_distributed_ddl_on_cross_replication/test.py::test_atomic_database", - "test_distributed_ddl_parallel/test.py::test_all_in_parallel", - "test_distributed_ddl_parallel/test.py::test_slow_dict_load_7", - "test_distributed_ddl_parallel/test.py::test_smoke", - "test_distributed_ddl_parallel/test.py::test_smoke_parallel", - "test_distributed_ddl_parallel/test.py::test_smoke_parallel_dict_reload", - "test_distributed_ddl_parallel/test.py::test_two_in_parallel_two_queued", - "test_distributed_ddl_password/test.py::test_alter", - "test_distributed_ddl_password/test.py::test_truncate", - "test_distributed_ddl/test.py::test_allowed_databases[configs]", - "test_distributed_ddl/test.py::test_allowed_databases[configs_secure]", - "test_distributed_ddl/test.py::test_create_as_select[configs]", - "test_distributed_ddl/test.py::test_create_as_select[configs_secure]", - "test_distributed_ddl/test.py::test_create_reserved[configs]", - "test_distributed_ddl/test.py::test_create_reserved[configs_secure]", - "test_distributed_ddl/test.py::test_create_view[configs]", - "test_distributed_ddl/test.py::test_create_view[configs_secure]", - "test_distributed_ddl/test.py::test_default_database[configs]", - "test_distributed_ddl/test.py::test_default_database[configs_secure]", - "test_distributed_ddl/test.py::test_detach_query[configs]", - "test_distributed_ddl/test.py::test_detach_query[configs_secure]", - "test_distributed_ddl/test.py::test_implicit_macros[configs]", - "test_distributed_ddl/test.py::test_implicit_macros[configs_secure]", - "test_distributed_ddl/test.py::test_kill_query[configs]", - "test_distributed_ddl/test.py::test_kill_query[configs_secure]", - "test_distributed_ddl/test.py::test_macro[configs]", - "test_distributed_ddl/test.py::test_macro[configs_secure]", - "test_distributed_ddl/test.py::test_on_connection_loss[configs]", - "test_distributed_ddl/test.py::test_on_connection_loss[configs_secure]", - "test_distributed_ddl/test.py::test_on_server_fail[configs]", - "test_distributed_ddl/test.py::test_on_server_fail[configs_secure]", - "test_distributed_ddl/test.py::test_on_session_expired[configs]", - "test_distributed_ddl/test.py::test_on_session_expired[configs_secure]", - "test_distributed_ddl/test.py::test_optimize_query[configs]", - "test_distributed_ddl/test.py::test_optimize_query[configs_secure]", - "test_distributed_ddl/test.py::test_rename[configs]", - "test_distributed_ddl/test.py::test_rename[configs_secure]", - "test_distributed_ddl/test.py::test_replicated_without_arguments[configs]", - "test_distributed_ddl/test.py::test_replicated_without_arguments[configs_secure]", - "test_distributed_ddl/test.py::test_simple_alters[configs]", - "test_distributed_ddl/test.py::test_simple_alters[configs_secure]", - "test_distributed_ddl/test.py::test_socket_timeout[configs]", - "test_distributed_ddl/test.py::test_socket_timeout[configs_secure]", - "test_distributed_ddl/test_replicated_alter.py::test_replicated_alters[configs]", - "test_distributed_ddl/test_replicated_alter.py::test_replicated_alters[configs_secure]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node1-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node1-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node2-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node2-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node1-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node1-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node2-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node2-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node1-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node1-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node2-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node2-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node1-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node1-remote]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node2-distributed]", - "test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node2-remote]", - "test_drop_replica/test.py::test_drop_replica", - "test_hedged_requests_parallel/test.py::test_combination1", - "test_hedged_requests_parallel/test.py::test_combination2", - "test_hedged_requests_parallel/test.py::test_query_with_no_data_to_sample", - "test_hedged_requests_parallel/test.py::test_send_data", - "test_hedged_requests_parallel/test.py::test_send_table_status_sleep", - "test_hedged_requests/test.py::test_combination1", - "test_hedged_requests/test.py::test_combination2", - "test_hedged_requests/test.py::test_combination3", - "test_hedged_requests/test.py::test_combination4", - "test_hedged_requests/test.py::test_long_query", - "test_hedged_requests/test.py::test_receive_timeout1", - "test_hedged_requests/test.py::test_receive_timeout2", - "test_hedged_requests/test.py::test_send_data", - "test_hedged_requests/test.py::test_send_data2", - "test_hedged_requests/test.py::test_send_table_status_sleep", - "test_hedged_requests/test.py::test_send_table_status_sleep2", - "test_hedged_requests/test.py::test_stuck_replica", - "test_https_replication/test.py::test_both_http", - "test_https_replication/test.py::test_both_https", - "test_https_replication/test.py::test_mixed_protocol", - "test_https_replication/test.py::test_replication_after_partition", - "test_insert_into_distributed_sync_async/test.py::test_async_inserts_into_local_shard", - "test_insert_into_distributed_sync_async/test.py::test_insertion_sync", - "test_insert_into_distributed_sync_async/test.py::test_insertion_sync_fails_with_timeout", - "test_insert_into_distributed_sync_async/test.py::test_insertion_sync_with_disabled_timeout", - "test_insert_into_distributed_sync_async/test.py::test_insertion_without_sync_ignores_timeout", - "test_insert_into_distributed/test.py::test_inserts_batching", - "test_insert_into_distributed/test.py::test_inserts_local", - "test_insert_into_distributed/test.py::test_inserts_low_cardinality", - "test_insert_into_distributed/test.py::test_inserts_single_replica_internal_replication", - "test_insert_into_distributed/test.py::test_inserts_single_replica_local_internal_replication", - "test_insert_into_distributed/test.py::test_inserts_single_replica_no_internal_replication", - "test_insert_into_distributed/test.py::test_prefer_localhost_replica", - "test_insert_into_distributed/test.py::test_reconnect", - "test_insert_into_distributed/test.py::test_table_function", - "test_insert_into_distributed_through_materialized_view/test.py::test_inserts_local", - "test_insert_into_distributed_through_materialized_view/test.py::test_reconnect", - "test_keeper_multinode_blocade_leader/test.py::test_blocade_leader", - "test_keeper_multinode_blocade_leader/test.py::test_blocade_leader_twice", - "test_keeper_multinode_simple/test.py::test_follower_restart", - "test_keeper_multinode_simple/test.py::test_read_write_multinode", - "test_keeper_multinode_simple/test.py::test_session_expiration", - "test_keeper_multinode_simple/test.py::test_simple_replicated_table", - "test_keeper_multinode_simple/test.py::test_watch_on_follower", - "test_limited_replicated_fetches/test.py::test_limited_fetches", - "test_materialized_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_multi_table_update[clickhouse_node0]", - "test_materialized_mysql_database/test.py::test_multi_table_update[clickhouse_node1]", - "test_materialized_mysql_database/test.py::test_mysql_killed_while_insert_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_mysql_killed_while_insert_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_mysql_killed_while_insert_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_mysql_killed_while_insert_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_mysql_settings[clickhouse_node0]", - "test_materialized_mysql_database/test.py::test_mysql_settings[clickhouse_node1]", - "test_materialized_mysql_database/test.py::test_network_partition_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_network_partition_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_network_partition_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_network_partition_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_select_without_columns_5_7[atomic]", - "test_materialized_mysql_database/test.py::test_select_without_columns_5_7[ordinary]", - "test_materialized_mysql_database/test.py::test_select_without_columns_8_0[atomic]", - "test_materialized_mysql_database/test.py::test_select_without_columns_8_0[ordinary]", - "test_materialized_mysql_database/test.py::test_system_parts_table[clickhouse_node0]", - "test_materialized_mysql_database/test.py::test_system_parts_table[clickhouse_node1]", - "test_materialized_mysql_database/test.py::test_system_tables_table[clickhouse_node0]", - "test_materialized_mysql_database/test.py::test_system_tables_table[clickhouse_node1]", - "test_materialized_mysql_database/test.py::test_materialize_with_column_comments[clickhouse_node0]", - "test_materialized_mysql_database/test.py::test_materialize_with_column_comments[clickhouse_node1]", - "test_materialized_mysql_database/test.py::test_materialize_with_enum[clickhouse_node0]", - "test_materialized_mysql_database/test.py::test_materialize_with_enum[clickhouse_node1]", - "test_materialized_mysql_database/test.py::test_utf8mb4[clickhouse_node0]", - "test_materialized_mysql_database/test.py::test_utf8mb4[clickhouse_node1]", - "test_parts_delete_zookeeper/test.py::test_merge_doesnt_work_without_zookeeper", - "test_polymorphic_parts/test.py::test_compact_parts_only", - "test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_compact-Compact]", - "test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_wide-Wide]", - "test_polymorphic_parts/test.py::test_in_memory", - "test_polymorphic_parts/test.py::test_in_memory_alters", - "test_polymorphic_parts/test.py::test_in_memory_deduplication", - "test_polymorphic_parts/test.py::test_in_memory_wal_rotate", - "test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node0-second_node0]", - "test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node1-second_node1]", - "test_polymorphic_parts/test.py::test_polymorphic_parts_index", - "test_polymorphic_parts/test.py::test_polymorphic_parts_non_adaptive", - "test_quorum_inserts_parallel/test.py::test_parallel_quorum_actually_parallel", - "test_quorum_inserts_parallel/test.py::test_parallel_quorum_actually_quorum", - "test_random_inserts/test.py::test_insert_multithreaded", - "test_random_inserts/test.py::test_random_inserts", - "test_reload_clusters_config/test.py::test_add_cluster", - "test_reload_clusters_config/test.py::test_delete_cluster", - "test_reload_clusters_config/test.py::test_simple_reload", - "test_reload_clusters_config/test.py::test_update_one_cluster", - "test_replace_partition/test.py::test_drop_failover", - "test_replace_partition/test.py::test_normal_work", - "test_replace_partition/test.py::test_replace_after_replace_failover", - "test_replicated_database/test.py::test_alters_from_different_replicas", - "test_replicated_database/test.py::test_create_replicated_table", - "test_replicated_database/test.py::test_recover_staled_replica", - "test_replicated_database/test.py::test_simple_alter_table[MergeTree]", - "test_replicated_database/test.py::test_simple_alter_table[ReplicatedMergeTree]", - "test_replicated_database/test.py::test_startup_without_zk", - "test_replicated_fetches_timeouts/test.py::test_no_stall", - "test_storage_kafka/test.py::test_bad_reschedule", - "test_storage_kafka/test.py::test_commits_of_unprocessed_messages_on_drop", - "test_storage_kafka/test.py::test_exception_from_destructor", - "test_storage_kafka/test.py::test_kafka_commit_on_block_write", - "test_storage_kafka/test.py::test_kafka_consumer_hang", - "test_storage_kafka/test.py::test_kafka_consumer_hang2", - "test_storage_kafka/test.py::test_kafka_csv_with_delimiter", - "test_storage_kafka/test.py::test_kafka_csv_with_thread_per_consumer", - "test_storage_kafka/test.py::test_kafka_duplicates_when_commit_failed", - "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream", - "test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream_with_random_malformed_json", - "test_storage_kafka/test.py::test_kafka_flush_by_block_size", - "test_storage_kafka/test.py::test_kafka_flush_by_time", - "test_storage_kafka/test.py::test_kafka_flush_on_big_message", - "test_storage_kafka/test.py::test_kafka_formats", - "test_storage_kafka/test.py::test_kafka_formats_with_broken_message", - "test_storage_kafka/test.py::test_kafka_insert", - "test_storage_kafka/test.py::test_kafka_issue11308", - "test_storage_kafka/test.py::test_kafka_issue14202", - "test_storage_kafka/test.py::test_kafka_issue4116", - "test_storage_kafka/test.py::test_kafka_json_as_string", - "test_storage_kafka/test.py::test_kafka_json_without_delimiter", - "test_storage_kafka/test.py::test_kafka_lot_of_partitions_partial_commit_of_bulk", - "test_storage_kafka/test.py::test_kafka_many_materialized_views", - "test_storage_kafka/test.py::test_kafka_materialized_view", - "test_storage_kafka/test.py::test_kafka_materialized_view_with_subquery", - "test_storage_kafka/test.py::test_kafka_no_holes_when_write_suffix_failed", - "test_storage_kafka/test.py::test_kafka_produce_consume", - "test_storage_kafka/test.py::test_kafka_produce_key_timestamp", - "test_storage_kafka/test.py::test_kafka_protobuf", - "test_storage_kafka/test.py::test_kafka_protobuf_no_delimiter", - "test_storage_kafka/test.py::test_kafka_rebalance", - "test_storage_kafka/test.py::test_kafka_select_empty", - "test_storage_kafka/test.py::test_kafka_settings_new_syntax", - "test_storage_kafka/test.py::test_kafka_settings_old_syntax", - "test_storage_kafka/test.py::test_kafka_string_field_on_first_position_in_protobuf", - "test_storage_kafka/test.py::test_kafka_tsv_with_delimiter", - "test_storage_kafka/test.py::test_kafka_unavailable", - "test_storage_kafka/test.py::test_kafka_virtual_columns", - "test_storage_kafka/test.py::test_kafka_virtual_columns2", - "test_storage_kafka/test.py::test_kafka_virtual_columns_with_materialized_view", - "test_storage_kafka/test.py::test_librdkafka_compression", - "test_storage_kafka/test.py::test_premature_flush_on_eof", - "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string", - "test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc", - "test_system_clusters_actual_information/test.py::test", - "test_system_metrics/test.py::test_readonly_metrics", - "test_system_replicated_fetches/test.py::test_system_replicated_fetches" + "test_host_ip_change/test.py::test_user_access_ip_change[node1]" ] From 9f6af8501eeb91c1c68ce871ebd1add033df3a3a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 1 Nov 2021 11:58:49 +0300 Subject: [PATCH 348/396] Fix build --- src/AggregateFunctions/AggregateFunctionMap.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index 63974091a01..e5367ac4f9f 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -17,7 +17,7 @@ #include #include #include -#include "common/types.h" +#include "base/types.h" #include #include "AggregateFunctions/AggregateFunctionFactory.h" From a1f29d31ea081cf055238be9d4a253327068db1d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 26 Oct 2021 12:48:31 +0300 Subject: [PATCH 349/396] 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 350/396] 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 351/396] 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 8efa1743cf6f56845195488b1f2094717c99b33e Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 29 Oct 2021 16:47:49 +0300 Subject: [PATCH 352/396] output_format_avro_rows_in_file fix and test for _timestamp --- .../Formats/Impl/AvroRowOutputFormat.cpp | 7 ++- tests/integration/test_storage_kafka/test.py | 53 +++++++++++++++++-- 2 files changed, 55 insertions(+), 5 deletions(-) diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp index e5845003ca2..7838da157b1 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp @@ -443,6 +443,10 @@ void AvroRowOutputFormat::consumeImplWithCallback(DB::Chunk chunk) for (size_t row = 0; row < num_rows;) { + size_t current_row = row; + /// used by WriteBufferToKafkaProducer to obtain auxiliary data + /// from the starting row of a file + writePrefix(); for (size_t row_in_file = 0; row_in_file < settings.avro.output_rows_in_file && row < num_rows; @@ -451,11 +455,10 @@ void AvroRowOutputFormat::consumeImplWithCallback(DB::Chunk chunk) write(columns, row); } - file_writer_ptr->flush(); writeSuffix(); - params.callback(columns, num_rows); + params.callback(columns, current_row); } } diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index e01f4bf146e..c2b61b26ba1 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -120,17 +120,20 @@ def kafka_produce(kafka_cluster, topic, messages, timestamp=None, retries=15): def kafka_producer_send_heartbeat_msg(max_retries=50): kafka_produce(kafka_cluster, 'test_heartbeat_topic', ['test'], retries=max_retries) -def kafka_consume(kafka_cluster, topic): +def kafka_consume(kafka_cluster, topic, needDecode = True, timestamp = 0): consumer = KafkaConsumer(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port), auto_offset_reset="earliest") consumer.subscribe(topics=(topic)) for toppar, messages in list(consumer.poll(5000).items()): if toppar.topic == topic: for message in messages: - yield message.value.decode() + assert timestamp == 0 or message.timestamp / 1000 == timestamp + if needDecode: + yield message.value.decode() + else: + yield message.value consumer.unsubscribe() consumer.close() - def kafka_produce_protobuf_messages(kafka_cluster, topic, start_index, num_messages): data = b'' for i in range(start_index, start_index + num_messages): @@ -682,6 +685,16 @@ def kafka_check_result(result, check=False, ref_file='test_kafka_json.reference' return TSV(result) == TSV(reference) +def decode_avro(message): + b = io.BytesIO(message) + ret = avro.datafile.DataFileReader(b, avro.io.DatumReader()) + + output = io.StringIO() + for record in ret: + print(record, file=output) + return output.getvalue() + + # https://stackoverflow.com/a/57692111/1555175 def describe_consumer_group(kafka_cluster, name): client = BrokerConnection('localhost', kafka_cluster.kafka_port, socket.AF_INET) @@ -1830,6 +1843,40 @@ def test_kafka_produce_key_timestamp(kafka_cluster): kafka_delete_topic(admin_client, topic_name) +def test_kafka_insert_avro(kafka_cluster): + instance.query(''' + DROP TABLE IF EXISTS test.kafka; + CREATE TABLE test.kafka (key UInt64, value UInt64, _timestamp DateTime('UTC')) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'avro1', + kafka_group_name = 'avro1', + kafka_format = 'Avro'; + ''') + + + instance.query("INSERT INTO test.kafka select number*10 as key, number*100 as value, 1636505534 as _timestamp from numbers(4) SETTINGS output_format_avro_rows_in_file = 2, output_format_avro_codec = 'deflate'") + + messages = [] + while True: + messages.extend(kafka_consume(kafka_cluster, 'avro1', needDecode = False, timestamp = 1636505534)) + if len(messages) == 2: + break + + result = '' + for a_message in messages: + result += decode_avro(a_message) + '\n' + + expected_result = """{'key': 0, 'value': 0, '_timestamp': 1636505534} +{'key': 10, 'value': 100, '_timestamp': 1636505534} + +{'key': 20, 'value': 200, '_timestamp': 1636505534} +{'key': 30, 'value': 300, '_timestamp': 1636505534} + +""" + assert (result == expected_result) + + def test_kafka_produce_consume_avro(kafka_cluster): admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) From 47b717ceff0a18c2d6408da180bf3691578e8597 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Nov 2021 13:27:46 +0300 Subject: [PATCH 353/396] 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 354/396] 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 355/396] 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 356/396] 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 357/396] 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 358/396] 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 359/396] 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 360/396] 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 53c0e1ae615fd9a79494fa56fc6b6e779af0f18f Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 1 Nov 2021 18:09:16 +0300 Subject: [PATCH 361/396] Add setting for remote reads --- src/Core/Settings.h | 2 ++ src/Processors/QueryPlan/ReadFromMergeTree.cpp | 17 +++++++++++++++-- 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e6f685a0650..8d809a778d4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -511,6 +511,8 @@ class IColumn; 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) \ + M(UInt64, merge_tree_min_rows_for_concurrent_read_for_remote_filesystem, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized, when reading from remote filesystem.", 0) \ + M(UInt64, merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized, when reading from remote filesystem.", 0) \ \ M(UInt64, async_insert_threads, 16, "Maximum number of threads to actually parse and insert data in background. Zero means asynchronous mode is disabled", 0) \ M(Bool, async_insert, false, "If true, data from INSERT query is stored in queue and later flushed to table in background. Makes sense only for inserts via HTTP protocol. If wait_for_async_insert is false, INSERT query is processed almost instantly, otherwise client will wait until data will be flushed to table", 0) \ diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 519de724f9e..f358db3aa1c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -244,6 +244,16 @@ struct PartRangesReadInfo bool use_uncompressed_cache = false; + static bool checkAllPartsOnRemoteFS(const RangesInDataParts & parts) + { + for (const auto & part : parts) + { + if (!part.data_part->isStoredOnRemoteDisk()) + return false; + } + return true; + } + PartRangesReadInfo( const RangesInDataParts & parts, const Settings & settings, @@ -270,9 +280,12 @@ struct PartRangesReadInfo data_settings.index_granularity, index_granularity_bytes); + auto all_parts_on_remote_disk = checkAllPartsOnRemoteFS(parts); min_marks_for_concurrent_read = MergeTreeDataSelectExecutor::minMarksForConcurrentRead( - settings.merge_tree_min_rows_for_concurrent_read, - settings.merge_tree_min_bytes_for_concurrent_read, + all_parts_on_remote_disk ? settings.merge_tree_min_rows_for_concurrent_read_for_remote_filesystem + : settings.merge_tree_min_rows_for_concurrent_read, + all_parts_on_remote_disk ? settings.merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem + : settings.merge_tree_min_bytes_for_concurrent_read, data_settings.index_granularity, index_granularity_bytes, sum_marks); 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 362/396] 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 363/396] 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 364/396] 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 365/396] 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 4bf30957e1d613c8865a8f191bfc4f757d0f1de6 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Mon, 1 Nov 2021 19:44:19 +0300 Subject: [PATCH 366/396] Update docs/en/operations/settings/merge-tree-settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/operations/settings/merge-tree-settings.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 7159ae7f697..e9eed111b4f 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -335,7 +335,8 @@ Sets minimal amount of bytes to enable parts rebalance over [JBOD](https://en.wi Possible values: -- Any positive integer. +- Positive integer. +- 0 - Rebalancing is disabled. Default value: `0`. From 3b501ffb3e13e797599028c31fa3cbf56efff4a8 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Mon, 1 Nov 2021 20:15:46 +0300 Subject: [PATCH 367/396] Translate to Russian MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Выполнил перевод на русский язык. --- .../table-engines/mergetree-family/mergetree.md | 2 +- .../en/operations/settings/merge-tree-settings.md | 4 ++-- .../table-engines/mergetree-family/mergetree.md | 2 ++ .../ru/operations/settings/merge-tree-settings.md | 15 +++++++++++++++ 4 files changed, 20 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index dd9a18c8dba..d08de080e6b 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -787,7 +787,7 @@ Moving data does not interfere with data replication. Therefore, different stora After the completion of background merges and mutations, old parts are removed only after a certain amount of time (`old_parts_lifetime`). During this time, they are not moved to other volumes or disks. Therefore, until the parts are finally removed, they are still taken into account for evaluation of the occupied disk space. -User can assign new parts to different disks of a [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures) volume in a balanced way using the [min_bytes_to_rebalance_partition_over_jbod](../../../operations/settings/merge-tree-settings.md#min-bytes-to-rebalance-partition-over-jbod) setting. +User can assign new big parts to different disks of a [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures) volume in a balanced way using the [min_bytes_to_rebalance_partition_over_jbod](../../../operations/settings/merge-tree-settings.md#min-bytes-to-rebalance-partition-over-jbod) setting. ## Using S3 for Data Storage {#table_engine-mergetree-s3} diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index e9eed111b4f..16ec55f026a 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -331,12 +331,12 @@ By default, the ClickHouse server checks at table creation the data type of a co ## min_bytes_to_rebalance_partition_over_jbod {#min-bytes-to-rebalance-partition-over-jbod} -Sets minimal amount of bytes to enable parts rebalance over [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures) array. Allows assigning new parts to different disks of a `JBOD` volume in a balanced way. +Sets minimal amount of bytes to enable balancing when distributing new big parts over volume disks [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures). Possible values: - Positive integer. -- 0 - Rebalancing is disabled. +- 0 — Balancing is disabled. Default value: `0`. diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 07e67ad1b85..4448372c522 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -775,6 +775,8 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' После выполнения фоновых слияний или мутаций старые куски не удаляются сразу, а через некоторое время (табличная настройка `old_parts_lifetime`). Также они не перемещаются на другие тома или диски, поэтому до момента удаления они продолжают учитываться при подсчёте занятого дискового пространства. +Пользователь может сбалансированно распределять новые большие куски данных по разным дискам тома [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures), используя настройку [min_bytes_to_rebalance_partition_over_jbod](../../../operations/settings/merge-tree-settings.md#min-bytes-to-rebalance-partition-over-jbod). + ## Использование сервиса S3 для хранения данных {#table_engine-mergetree-s3} Таблицы семейства `MergeTree` могут хранить данные в сервисе [S3](https://aws.amazon.com/s3/) при использовании диска типа `s3`. diff --git a/docs/ru/operations/settings/merge-tree-settings.md b/docs/ru/operations/settings/merge-tree-settings.md index 08ea9979426..117ed7d633b 100644 --- a/docs/ru/operations/settings/merge-tree-settings.md +++ b/docs/ru/operations/settings/merge-tree-settings.md @@ -327,3 +327,18 @@ Eсли суммарное число активных кусков во все Значение по умолчанию: `true`. По умолчанию сервер ClickHouse при создании таблицы проверяет тип данных столбца для сэмплирования или выражения сэмплирования. Если уже существуют таблицы с некорректным выражением сэмплирования, то чтобы не возникало исключение при запуске сервера, установите `check_sample_column_is_correct` в значение `false`. + +## min_bytes_to_rebalance_partition_over_jbod {#min-bytes-to-rebalance-partition-over-jbod} + +Устанавливает минимальное количество байтов для обеспечения балансировки при распределении новых больших кусков данных по дискам тома [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures). + +Возможные значения: + +- Положительное целое число. +- 0 — балансировка отключена. + +Значение по умолчанию: `0`. + +**Использование** + +Значение настройки `min_bytes_to_rebalance_partition_over_jbod` должно быть меньше значения настройки [max_bytes_to_merge_at_max_space_in_pool](../../operations/settings/merge-tree-settings.md#max-bytes-to-merge-at-max-space-in-pool). Иначе ClickHouse сгенерирует исключение. From 38f7d1ebc27b6ca0b6a10882b6d3c9df9e310abe Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 1 Nov 2021 22:59:57 +0300 Subject: [PATCH 368/396] 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 369/396] 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 370/396] 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)", From 9080d2ea528e0c30f96af4e49ebb8206cecb7552 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 1 Nov 2021 09:52:27 +0000 Subject: [PATCH 371/396] Fix tests --- src/Storages/ExternalDataSourceConfiguration.cpp | 2 +- src/Storages/StorageURL.cpp | 7 ++++--- src/Storages/StorageXDBC.cpp | 6 +++--- src/Storages/StorageXDBC.h | 2 -- src/TableFunctions/TableFunctionURL.cpp | 2 +- 5 files changed, 9 insertions(+), 10 deletions(-) diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index dcdcb2cbefb..26a00b7cbed 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -332,7 +332,7 @@ std::optional getURLBasedDataSourceConfiguration(const if (arg_name == "url") configuration.url = arg_value.safeGet(); - if (arg_name == "method") + else if (arg_name == "method") configuration.http_method = arg_value.safeGet(); else if (arg_name == "format") configuration.format = arg_value.safeGet(); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index ab1e9713701..fa55ddbcc7b 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -325,7 +325,7 @@ Pipe IStorageURLBase::read( { auto params = getReadURIParams(column_names, metadata_snapshot, query_info, local_context, processed_stage, max_block_size); bool with_globs = (uri.find('{') != std::string::npos && uri.find('}') != std::string::npos) - || uri.find('|') == std::string::npos; + || uri.find('|') != std::string::npos; if (with_globs) { @@ -390,6 +390,7 @@ Pipe StorageURLWithFailover::read( unsigned /*num_streams*/) { auto params = getReadURIParams(column_names, metadata_snapshot, query_info, local_context, processed_stage, max_block_size); + auto pipe = Pipe(std::make_shared( uri_options, getReadMethod(), @@ -473,8 +474,8 @@ StorageURLWithFailover::StorageURLWithFailover( { Poco::URI poco_uri(uri_option); context_->getRemoteHostFilter().checkURL(poco_uri); - uri_options.emplace_back(std::move(uri_option)); LOG_DEBUG(&Poco::Logger::get("StorageURLDistributed"), "Adding URL option: {}", uri_option); + uri_options.emplace_back(std::move(uri_option)); } } @@ -539,7 +540,7 @@ URLBasedDataSourceConfiguration StorageURL::getConfiguration(ASTs & args, Contex illegal_args += ", "; illegal_args += arg.first; } - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown arguments {} for table function URL", illegal_args); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown argument `{}` for storage URL", illegal_args); } } else diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index 3ccde3a3597..90ac04ed250 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -47,7 +47,7 @@ StorageXDBC::StorageXDBC( , remote_table_name(remote_table_name_) , log(&Poco::Logger::get("Storage" + bridge_helper->getName())) { - poco_uri = bridge_helper->getMainURI(); + uri = bridge_helper->getMainURI().toString(); } std::string StorageXDBC::getReadMethod() const @@ -118,7 +118,7 @@ SinkToStoragePtr StorageXDBC::write(const ASTPtr & /*query*/, const StorageMetad { bridge_helper->startBridgeSync(); - Poco::URI request_uri = poco_uri; + auto request_uri = Poco::URI(uri); request_uri.setPath("/write"); auto url_params = bridge_helper->getURLParams(65536); @@ -137,7 +137,7 @@ SinkToStoragePtr StorageXDBC::write(const ASTPtr & /*query*/, const StorageMetad metadata_snapshot->getSampleBlock(), local_context, ConnectionTimeouts::getHTTPTimeouts(local_context), - chooseCompressionMethod(poco_uri.toString(), compression_method)); + chooseCompressionMethod(uri, compression_method)); } Block StorageXDBC::getHeaderBlock(const Names & column_names, const StorageMetadataPtr & metadata_snapshot) const diff --git a/src/Storages/StorageXDBC.h b/src/Storages/StorageXDBC.h index 34207dcbb4c..4438e1c4737 100644 --- a/src/Storages/StorageXDBC.h +++ b/src/Storages/StorageXDBC.h @@ -41,8 +41,6 @@ public: std::string getName() const override; private: - Poco::URI poco_uri; - BridgeHelperPtr bridge_helper; std::string remote_database_name; std::string remote_table_name; diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index 11c2d99de50..94e893466f6 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -45,7 +45,7 @@ void TableFunctionURL::parseArguments(const ASTPtr & ast_function, ContextPtr co illegal_args += ", "; illegal_args += arg.first; } - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown arguments {} for table function URL", illegal_args); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown argument `{}` for table function URL", illegal_args); } filename = configuration.url; From a0c38a3dfd71268d82f6dc5c3e6292f3561ba5f6 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 2 Nov 2021 12:02:22 +0300 Subject: [PATCH 372/396] Rename AuthenticationType::MAX_TYPE. --- src/Access/Authentication.cpp | 8 ++++---- src/Access/Common/AuthenticationData.cpp | 8 ++++---- src/Access/Common/AuthenticationData.h | 15 +++++++-------- src/Interpreters/SessionLog.cpp | 2 +- src/Parsers/Access/ASTCreateUserQuery.cpp | 2 +- src/Parsers/Access/ParserCreateUserQuery.cpp | 2 +- src/Storages/System/StorageSystemUsers.cpp | 2 +- 7 files changed, 19 insertions(+), 20 deletions(-) diff --git a/src/Access/Authentication.cpp b/src/Access/Authentication.cpp index d0aef37c15b..794c0a0d5d5 100644 --- a/src/Access/Authentication.cpp +++ b/src/Access/Authentication.cpp @@ -23,7 +23,7 @@ namespace bool checkPasswordPlainText(const String & password, const Digest & password_plaintext) { - return (Util::encodePlainText(password) == password_plaintext); + return (Util::stringToDigest(password) == password_plaintext); } bool checkPasswordDoubleSHA1(const std::string_view & password, const Digest & password_double_sha1) @@ -87,7 +87,7 @@ bool Authentication::areCredentialsValid(const Credentials & credentials, const case AuthenticationType::KERBEROS: return external_authenticators.checkKerberosCredentials(auth_data.getKerberosRealm(), *gss_acceptor_context); - case AuthenticationType::MAX_TYPE: + case AuthenticationType::MAX: break; } } @@ -110,7 +110,7 @@ bool Authentication::areCredentialsValid(const Credentials & credentials, const case AuthenticationType::KERBEROS: throw Authentication::Require("ClickHouse Basic Authentication"); - case AuthenticationType::MAX_TYPE: + case AuthenticationType::MAX: break; } } @@ -137,7 +137,7 @@ bool Authentication::areCredentialsValid(const Credentials & credentials, const case AuthenticationType::KERBEROS: throw Authentication::Require(auth_data.getKerberosRealm()); - case AuthenticationType::MAX_TYPE: + case AuthenticationType::MAX: break; } } diff --git a/src/Access/Common/AuthenticationData.cpp b/src/Access/Common/AuthenticationData.cpp index e460d1dcc4d..012e7546270 100644 --- a/src/Access/Common/AuthenticationData.cpp +++ b/src/Access/Common/AuthenticationData.cpp @@ -59,7 +59,7 @@ const AuthenticationTypeInfo & AuthenticationTypeInfo::get(AuthenticationType ty static const auto info = make_info("KERBEROS"); return info; } - case AuthenticationType::MAX_TYPE: + case AuthenticationType::MAX: break; } throw Exception("Unknown authentication type: " + std::to_string(static_cast(type_)), ErrorCodes::LOGICAL_ERROR); @@ -101,7 +101,7 @@ void AuthenticationData::setPassword(const String & password_) switch (type) { case AuthenticationType::PLAINTEXT_PASSWORD: - return setPasswordHashBinary(Util::encodePlainText(password_)); + return setPasswordHashBinary(Util::stringToDigest(password_)); case AuthenticationType::SHA256_PASSWORD: return setPasswordHashBinary(Util::encodeSHA256(password_)); @@ -114,7 +114,7 @@ void AuthenticationData::setPassword(const String & password_) case AuthenticationType::KERBEROS: throw Exception("Cannot specify password for authentication type " + toString(type), ErrorCodes::LOGICAL_ERROR); - case AuthenticationType::MAX_TYPE: + case AuthenticationType::MAX: break; } throw Exception("setPassword(): authentication type " + toString(type) + " not supported", ErrorCodes::NOT_IMPLEMENTED); @@ -187,7 +187,7 @@ void AuthenticationData::setPasswordHashBinary(const Digest & hash) case AuthenticationType::KERBEROS: throw Exception("Cannot specify password binary hash for authentication type " + toString(type), ErrorCodes::LOGICAL_ERROR); - case AuthenticationType::MAX_TYPE: + case AuthenticationType::MAX: 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 index 5e47fbb9e15..8b390fd0900 100644 --- a/src/Access/Common/AuthenticationData.h +++ b/src/Access/Common/AuthenticationData.h @@ -27,7 +27,7 @@ enum class AuthenticationType /// Kerberos authentication performed through GSS-API negotiation loop. KERBEROS, - MAX_TYPE, + MAX, }; struct AuthenticationTypeInfo @@ -37,6 +37,11 @@ struct AuthenticationTypeInfo static const AuthenticationTypeInfo & get(AuthenticationType type_); }; +inline String toString(AuthenticationType type_) +{ + return AuthenticationTypeInfo::get(type_).raw_name; +} + /// Stores data for checking password when a user logins. class AuthenticationData @@ -79,7 +84,7 @@ public: struct Util { - static Digest encodePlainText(const std::string_view & text) { return Digest(text.data(), text.data() + text.size()); } + static Digest stringToDigest(const std::string_view & text) { return Digest(text.data(), text.data() + text.size()); } static Digest encodeSHA256(const std::string_view & text); static Digest encodeSHA1(const std::string_view & text); static Digest encodeSHA1(const Digest & text) { return encodeSHA1(std::string_view{reinterpret_cast(text.data()), text.size()}); } @@ -94,10 +99,4 @@ private: String kerberos_realm; }; - -inline String toString(AuthenticationType type_) -{ - return AuthenticationTypeInfo::get(type_).raw_name; -} - } diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index 6dac7ace662..f9419088df8 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -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 >= AuthenticationType::NO_PASSWORD && user_identified_with <= AuthenticationType::MAX_TYPE); + assert(user_identified_with >= AuthenticationType::NO_PASSWORD && user_identified_with <= AuthenticationType::MAX); size_t i = 0; diff --git a/src/Parsers/Access/ASTCreateUserQuery.cpp b/src/Parsers/Access/ASTCreateUserQuery.cpp index 116deef43d9..18030a5ed80 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.cpp +++ b/src/Parsers/Access/ASTCreateUserQuery.cpp @@ -78,7 +78,7 @@ namespace } case AuthenticationType::NO_PASSWORD: [[fallthrough]]; - case AuthenticationType::MAX_TYPE: + case AuthenticationType::MAX: throw Exception("AST: Unexpected authentication type " + toString(auth_type), ErrorCodes::LOGICAL_ERROR); } } diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index 8dcbfb46692..c5b8c9e37b3 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -55,7 +55,7 @@ namespace if (ParserKeyword{"WITH"}.ignore(pos, expected)) { - for (auto check_type : collections::range(AuthenticationType::MAX_TYPE)) + for (auto check_type : collections::range(AuthenticationType::MAX)) { if (ParserKeyword{AuthenticationTypeInfo::get(check_type).raw_name}.ignore(pos, expected)) { diff --git a/src/Storages/System/StorageSystemUsers.cpp b/src/Storages/System/StorageSystemUsers.cpp index e72c1a95385..135bde2a3ff 100644 --- a/src/Storages/System/StorageSystemUsers.cpp +++ b/src/Storages/System/StorageSystemUsers.cpp @@ -25,7 +25,7 @@ namespace DataTypeEnum8::Values getAuthenticationTypeEnumValues() { DataTypeEnum8::Values enum_values; - for (auto type : collections::range(AuthenticationType::MAX_TYPE)) + for (auto type : collections::range(AuthenticationType::MAX)) enum_values.emplace_back(AuthenticationTypeInfo::get(type).name, static_cast(type)); return enum_values; } From 1a228e8071a949b28755994069dd2befecc55715 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 2 Nov 2021 12:14:00 +0300 Subject: [PATCH 373/396] Update 01852_map_combinator.reference --- .../0_stateless/01852_map_combinator.reference | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01852_map_combinator.reference b/tests/queries/0_stateless/01852_map_combinator.reference index 7c0648ccb65..4ad4ea8071e 100644 --- a/tests/queries/0_stateless/01852_map_combinator.reference +++ b/tests/queries/0_stateless/01852_map_combinator.reference @@ -6,12 +6,12 @@ 4 {3:10,4:10,5:10} 5 {4:10,5:10,6:10} 5 {6:10,7:10,8:10} -Map(UInt16,UInt64) {1:20,2:20,3:40,4:40,5:40,6:40,7:20,8:20} -Map(UInt16,UInt32) {1:20,2:20,3:40,4:40,5:40,6:40,7:20,8:20} -Map(UInt16,UInt64) {1:20,2:20,3:40,4:40,5:40,6:40,7:20,8:20} +Map(UInt16, UInt64) {1:20,2:20,3:40,4:40,5:40,6:40,7:20,8:20} +Map(UInt16, UInt32) {1:20,2:20,3:40,4:40,5:40,6:40,7:20,8:20} +Map(UInt16, UInt64) {1:20,2:20,3:40,4:40,5:40,6:40,7:20,8:20} {1:10,2:10,3:10,4:10,5:10,6:10,7:10,8:10} {1:10,2:10,3:10,4:10,5:10,6:10,7:10,8:10} -Map(UInt16,Float64) {1:10,2:10,3:10,4:10,5:10,6:10,7:10,8:10} +Map(UInt16, Float64) {1:10,2:10,3:10,4:10,5:10,6:10,7:10,8:10} {1:2,2:2,3:4,4:4,5:4,6:4,7:2,8:2} 1 {1:10,2:10,3:20,4:10,5:10} 2 {4:10,5:10,6:20,7:10,8:10} @@ -30,5 +30,5 @@ Map(UInt16,Float64) {1:10,2:10,3:10,4:10,5:10,6:10,7:10,8:10} {1:1} {1:1} {1:1} -{1:1.00000,2:2.00000,3:6.00000,4:8.00000,5:10.00000,6:12.00000,7:7.00000,8:8.00000} -{1:1.00000,2:2.00000,3:6.00000,4:8.00000,5:10.00000,6:12.00000,7:7.00000,8:8.00000} +{1:1,2:2,3:6,4:8,5:10,6:12,7:7,8:8} +{1:1,2:2,3:6,4:8,5:10,6:12,7:7,8:8} From 42ab57f0e54f28db7198154b97326309900dc9b6 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 2 Nov 2021 14:06:10 +0300 Subject: [PATCH 374/396] Set output_format_avro_rows_in_file default to 1 --- src/Core/Settings.h | 2 +- src/Formats/FormatSettings.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0d852959302..46b5e275bce 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -594,7 +594,7 @@ class IColumn; M(String, output_format_avro_codec, "", "Compression codec used for output. Possible values: 'null', 'deflate', 'snappy'.", 0) \ M(UInt64, output_format_avro_sync_interval, 16 * 1024, "Sync interval in bytes.", 0) \ M(String, output_format_avro_string_column_pattern, "", "For Avro format: regexp of String columns to select as AVRO string.", 0) \ - M(UInt64, output_format_avro_rows_in_file, 1000000, "Max rows in a file (if permitted by storage)", 0) \ + M(UInt64, output_format_avro_rows_in_file, 1, "Max rows in a file (if permitted by storage)", 0) \ M(Bool, output_format_tsv_crlf_end_of_line, false, "If it is set true, end of line in TSV format will be \\r\\n instead of \\n.", 0) \ M(String, output_format_csv_null_representation, "\\N", "Custom NULL representation in CSV format", 0) \ M(String, output_format_tsv_null_representation, "\\N", "Custom NULL representation in TSV format", 0) \ diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 2123542ad90..6d422e30bc3 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -66,7 +66,7 @@ struct FormatSettings UInt64 output_sync_interval = 16 * 1024; bool allow_missing_fields = false; String string_column_pattern; - UInt64 output_rows_in_file = 1000000; + UInt64 output_rows_in_file = 1; } avro; struct CSV From afe2c9c0402054951d840907108f886bd73224aa Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 2 Nov 2021 14:06:20 +0300 Subject: [PATCH 375/396] Rename AccessControlManager -> AccessControl. --- programs/server/Server.cpp | 4 +- ...ssControlManager.cpp => AccessControl.cpp} | 76 +++++++++---------- ...AccessControlManager.h => AccessControl.h} | 6 +- src/Access/ContextAccess.cpp | 24 +++--- src/Access/ContextAccess.h | 8 +- src/Access/LDAPAccessStorage.cpp | 14 ++-- src/Access/LDAPAccessStorage.h | 8 +- src/Access/QuotaCache.cpp | 12 +-- src/Access/QuotaCache.h | 6 +- src/Access/RoleCache.cpp | 10 +-- src/Access/RoleCache.h | 6 +- src/Access/RolesOrUsersSet.cpp | 42 +++++----- src/Access/RolesOrUsersSet.h | 18 ++--- src/Access/RowPolicyCache.cpp | 12 +-- src/Access/RowPolicyCache.h | 6 +- src/Access/SettingsConstraints.cpp | 8 +- src/Access/SettingsConstraints.h | 6 +- src/Access/SettingsConstraintsAndProfileIDs.h | 2 +- src/Access/SettingsProfileElement.cpp | 34 ++++----- src/Access/SettingsProfileElement.h | 18 ++--- src/Access/SettingsProfilesCache.cpp | 20 ++--- src/Access/SettingsProfilesCache.h | 6 +- src/Access/SettingsProfilesInfo.cpp | 2 +- src/Access/SettingsProfilesInfo.h | 4 +- src/Functions/currentProfiles.cpp | 4 +- src/Functions/currentRoles.cpp | 4 +- .../Access/InterpreterCreateQuotaQuery.cpp | 4 +- .../Access/InterpreterCreateRoleQuery.cpp | 4 +- .../InterpreterCreateRowPolicyQuery.cpp | 4 +- .../InterpreterCreateSettingsProfileQuery.cpp | 4 +- .../Access/InterpreterCreateUserQuery.cpp | 4 +- .../InterpreterDropAccessEntityQuery.cpp | 4 +- .../Access/InterpreterGrantQuery.cpp | 12 +-- .../Access/InterpreterSetRoleQuery.cpp | 6 +- .../Access/InterpreterShowAccessQuery.cpp | 6 +- ...InterpreterShowCreateAccessEntityQuery.cpp | 46 +++++------ .../InterpreterShowCreateAccessEntityQuery.h | 4 +- .../Access/InterpreterShowGrantsQuery.cpp | 18 ++--- .../Access/InterpreterShowGrantsQuery.h | 4 +- src/Interpreters/Context.cpp | 40 +++++----- src/Interpreters/Context.h | 6 +- src/Interpreters/Session.cpp | 6 +- src/Storages/System/StorageSystemGrants.cpp | 4 +- .../System/StorageSystemPrivileges.cpp | 2 +- .../System/StorageSystemQuotaLimits.cpp | 4 +- .../System/StorageSystemQuotaUsage.cpp | 2 +- src/Storages/System/StorageSystemQuotas.cpp | 4 +- .../System/StorageSystemQuotasUsage.cpp | 4 +- .../System/StorageSystemRoleGrants.cpp | 4 +- src/Storages/System/StorageSystemRoles.cpp | 4 +- .../System/StorageSystemRowPolicies.cpp | 4 +- .../StorageSystemSettingsProfileElements.cpp | 4 +- .../System/StorageSystemSettingsProfiles.cpp | 4 +- .../System/StorageSystemUserDirectories.cpp | 4 +- src/Storages/System/StorageSystemUsers.cpp | 4 +- 55 files changed, 290 insertions(+), 290 deletions(-) rename src/Access/{AccessControlManager.cpp => AccessControl.cpp} (84%) rename src/Access/{AccessControlManager.h => AccessControl.h} (98%) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ce7d6973f68..0c4b1e47132 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -53,7 +53,7 @@ #include #include #include -#include +#include #include #include #include @@ -883,7 +883,7 @@ if (ThreadFuzzer::instance().isEffective()) }, /* already_loaded = */ false); /// Reload it right now (initial loading) - auto & access_control = global_context->getAccessControlManager(); + auto & access_control = global_context->getAccessControl(); if (config().has("custom_settings_prefixes")) access_control.setCustomSettingsPrefixes(config().getString("custom_settings_prefixes")); diff --git a/src/Access/AccessControlManager.cpp b/src/Access/AccessControl.cpp similarity index 84% rename from src/Access/AccessControlManager.cpp rename to src/Access/AccessControl.cpp index ff3ebdea9f8..378d8e2c264 100644 --- a/src/Access/AccessControlManager.cpp +++ b/src/Access/AccessControl.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -55,10 +55,10 @@ namespace } -class AccessControlManager::ContextAccessCache +class AccessControl::ContextAccessCache { public: - explicit ContextAccessCache(const AccessControlManager & manager_) : manager(manager_) {} + explicit ContextAccessCache(const AccessControl & access_control_) : access_control(access_control_) {} std::shared_ptr getContextAccess(const ContextAccessParams & params) { @@ -71,19 +71,19 @@ public: /// No user, probably the user has been dropped while it was in the cache. cache.remove(params); } - auto res = std::shared_ptr(new ContextAccess(manager, params)); + auto res = std::shared_ptr(new ContextAccess(access_control, params)); cache.add(params, res); return res; } private: - const AccessControlManager & manager; + const AccessControl & access_control; Poco::ExpireCache> cache; std::mutex mutex; }; -class AccessControlManager::CustomSettingsPrefixes +class AccessControl::CustomSettingsPrefixes { public: void registerPrefixes(const Strings & prefixes_) @@ -130,7 +130,7 @@ private: }; -AccessControlManager::AccessControlManager() +AccessControl::AccessControl() : MultipleAccessStorage("user directories"), context_access_cache(std::make_unique(*this)), role_cache(std::make_unique(*this)), @@ -143,9 +143,9 @@ AccessControlManager::AccessControlManager() } -AccessControlManager::~AccessControlManager() = default; +AccessControl::~AccessControl() = default; -void AccessControlManager::setUsersConfig(const Poco::Util::AbstractConfiguration & users_config_) +void AccessControl::setUsersConfig(const Poco::Util::AbstractConfiguration & users_config_) { auto storages = getStoragesPtr(); for (const auto & storage : *storages) @@ -159,12 +159,12 @@ void AccessControlManager::setUsersConfig(const Poco::Util::AbstractConfiguratio addUsersConfigStorage(users_config_); } -void AccessControlManager::addUsersConfigStorage(const Poco::Util::AbstractConfiguration & users_config_) +void AccessControl::addUsersConfigStorage(const Poco::Util::AbstractConfiguration & users_config_) { addUsersConfigStorage(UsersConfigAccessStorage::STORAGE_TYPE, users_config_); } -void AccessControlManager::addUsersConfigStorage(const String & storage_name_, const Poco::Util::AbstractConfiguration & users_config_) +void AccessControl::addUsersConfigStorage(const String & storage_name_, const Poco::Util::AbstractConfiguration & users_config_) { auto check_setting_name_function = [this](const std::string_view & setting_name) { checkSettingNameIsAllowed(setting_name); }; auto new_storage = std::make_shared(storage_name_, check_setting_name_function); @@ -173,7 +173,7 @@ void AccessControlManager::addUsersConfigStorage(const String & storage_name_, c LOG_DEBUG(getLogger(), "Added {} access storage '{}', path: {}", String(new_storage->getStorageType()), new_storage->getStorageName(), new_storage->getPath()); } -void AccessControlManager::addUsersConfigStorage( +void AccessControl::addUsersConfigStorage( const String & users_config_path_, const String & include_from_path_, const String & preprocessed_dir_, @@ -183,7 +183,7 @@ void AccessControlManager::addUsersConfigStorage( UsersConfigAccessStorage::STORAGE_TYPE, users_config_path_, include_from_path_, preprocessed_dir_, get_zookeeper_function_); } -void AccessControlManager::addUsersConfigStorage( +void AccessControl::addUsersConfigStorage( const String & storage_name_, const String & users_config_path_, const String & include_from_path_, @@ -206,7 +206,7 @@ void AccessControlManager::addUsersConfigStorage( LOG_DEBUG(getLogger(), "Added {} access storage '{}', path: {}", String(new_storage->getStorageType()), new_storage->getStorageName(), new_storage->getPath()); } -void AccessControlManager::reloadUsersConfigs() +void AccessControl::reloadUsersConfigs() { auto storages = getStoragesPtr(); for (const auto & storage : *storages) @@ -216,7 +216,7 @@ void AccessControlManager::reloadUsersConfigs() } } -void AccessControlManager::startPeriodicReloadingUsersConfigs() +void AccessControl::startPeriodicReloadingUsersConfigs() { auto storages = getStoragesPtr(); for (const auto & storage : *storages) @@ -226,7 +226,7 @@ void AccessControlManager::startPeriodicReloadingUsersConfigs() } } -void AccessControlManager::addReplicatedStorage( +void AccessControl::addReplicatedStorage( const String & storage_name_, const String & zookeeper_path_, const zkutil::GetZooKeeper & get_zookeeper_function_) @@ -243,12 +243,12 @@ void AccessControlManager::addReplicatedStorage( new_storage->startup(); } -void AccessControlManager::addDiskStorage(const String & directory_, bool readonly_) +void AccessControl::addDiskStorage(const String & directory_, bool readonly_) { addDiskStorage(DiskAccessStorage::STORAGE_TYPE, directory_, readonly_); } -void AccessControlManager::addDiskStorage(const String & storage_name_, const String & directory_, bool readonly_) +void AccessControl::addDiskStorage(const String & storage_name_, const String & directory_, bool readonly_) { auto storages = getStoragesPtr(); for (const auto & storage : *storages) @@ -269,7 +269,7 @@ void AccessControlManager::addDiskStorage(const String & storage_name_, const St } -void AccessControlManager::addMemoryStorage(const String & storage_name_) +void AccessControl::addMemoryStorage(const String & storage_name_) { auto storages = getStoragesPtr(); for (const auto & storage : *storages) @@ -283,7 +283,7 @@ void AccessControlManager::addMemoryStorage(const String & storage_name_) } -void AccessControlManager::addLDAPStorage(const String & storage_name_, const Poco::Util::AbstractConfiguration & config_, const String & prefix_) +void AccessControl::addLDAPStorage(const String & storage_name_, const Poco::Util::AbstractConfiguration & config_, const String & prefix_) { auto new_storage = std::make_shared(storage_name_, this, config_, prefix_); addStorage(new_storage); @@ -291,7 +291,7 @@ void AccessControlManager::addLDAPStorage(const String & storage_name_, const Po } -void AccessControlManager::addStoragesFromUserDirectoriesConfig( +void AccessControl::addStoragesFromUserDirectoriesConfig( const Poco::Util::AbstractConfiguration & config, const String & key, const String & config_dir, @@ -350,7 +350,7 @@ void AccessControlManager::addStoragesFromUserDirectoriesConfig( } -void AccessControlManager::addStoragesFromMainConfig( +void AccessControl::addStoragesFromMainConfig( const Poco::Util::AbstractConfiguration & config, const String & config_path, const zkutil::GetZooKeeper & get_zookeeper_function) @@ -388,47 +388,47 @@ void AccessControlManager::addStoragesFromMainConfig( } -UUID AccessControlManager::login(const Credentials & credentials, const Poco::Net::IPAddress & address) const +UUID AccessControl::login(const Credentials & credentials, const Poco::Net::IPAddress & address) const { return MultipleAccessStorage::login(credentials, address, *external_authenticators); } -void AccessControlManager::setExternalAuthenticatorsConfig(const Poco::Util::AbstractConfiguration & config) +void AccessControl::setExternalAuthenticatorsConfig(const Poco::Util::AbstractConfiguration & config) { external_authenticators->setConfiguration(config, getLogger()); } -void AccessControlManager::setDefaultProfileName(const String & default_profile_name) +void AccessControl::setDefaultProfileName(const String & default_profile_name) { settings_profiles_cache->setDefaultProfileName(default_profile_name); } -void AccessControlManager::setCustomSettingsPrefixes(const Strings & prefixes) +void AccessControl::setCustomSettingsPrefixes(const Strings & prefixes) { custom_settings_prefixes->registerPrefixes(prefixes); } -void AccessControlManager::setCustomSettingsPrefixes(const String & comma_separated_prefixes) +void AccessControl::setCustomSettingsPrefixes(const String & comma_separated_prefixes) { Strings prefixes; splitInto<','>(prefixes, comma_separated_prefixes); setCustomSettingsPrefixes(prefixes); } -bool AccessControlManager::isSettingNameAllowed(const std::string_view & setting_name) const +bool AccessControl::isSettingNameAllowed(const std::string_view & setting_name) const { return custom_settings_prefixes->isSettingNameAllowed(setting_name); } -void AccessControlManager::checkSettingNameIsAllowed(const std::string_view & setting_name) const +void AccessControl::checkSettingNameIsAllowed(const std::string_view & setting_name) const { custom_settings_prefixes->checkSettingNameIsAllowed(setting_name); } -std::shared_ptr AccessControlManager::getContextAccess( +std::shared_ptr AccessControl::getContextAccess( const UUID & user_id, const std::vector & current_roles, bool use_default_roles, @@ -464,13 +464,13 @@ std::shared_ptr AccessControlManager::getContextAccess( } -std::shared_ptr AccessControlManager::getContextAccess(const ContextAccessParams & params) const +std::shared_ptr AccessControl::getContextAccess(const ContextAccessParams & params) const { return context_access_cache->getContextAccess(params); } -std::shared_ptr AccessControlManager::getEnabledRoles( +std::shared_ptr AccessControl::getEnabledRoles( const std::vector & current_roles, const std::vector & current_roles_with_admin_option) const { @@ -478,13 +478,13 @@ std::shared_ptr AccessControlManager::getEnabledRoles( } -std::shared_ptr AccessControlManager::getEnabledRowPolicies(const UUID & user_id, const boost::container::flat_set & enabled_roles) const +std::shared_ptr AccessControl::getEnabledRowPolicies(const UUID & user_id, const boost::container::flat_set & enabled_roles) const { return row_policy_cache->getEnabledRowPolicies(user_id, enabled_roles); } -std::shared_ptr AccessControlManager::getEnabledQuota( +std::shared_ptr AccessControl::getEnabledQuota( const UUID & user_id, const String & user_name, const boost::container::flat_set & enabled_roles, @@ -496,13 +496,13 @@ std::shared_ptr AccessControlManager::getEnabledQuota( } -std::vector AccessControlManager::getAllQuotasUsage() const +std::vector AccessControl::getAllQuotasUsage() const { return quota_cache->getAllQuotasUsage(); } -std::shared_ptr AccessControlManager::getEnabledSettings( +std::shared_ptr AccessControl::getEnabledSettings( const UUID & user_id, const SettingsProfileElements & settings_from_user, const boost::container::flat_set & enabled_roles, @@ -511,13 +511,13 @@ std::shared_ptr AccessControlManager::getEnabledSettings( return settings_profiles_cache->getEnabledSettings(user_id, settings_from_user, enabled_roles, settings_from_enabled_roles); } -std::shared_ptr AccessControlManager::getSettingsProfileInfo(const UUID & profile_id) +std::shared_ptr AccessControl::getSettingsProfileInfo(const UUID & profile_id) { return settings_profiles_cache->getSettingsProfileInfo(profile_id); } -const ExternalAuthenticators & AccessControlManager::getExternalAuthenticators() const +const ExternalAuthenticators & AccessControl::getExternalAuthenticators() const { return *external_authenticators; } diff --git a/src/Access/AccessControlManager.h b/src/Access/AccessControl.h similarity index 98% rename from src/Access/AccessControlManager.h rename to src/Access/AccessControl.h index 79f7073ef69..779334f2a45 100644 --- a/src/Access/AccessControlManager.h +++ b/src/Access/AccessControl.h @@ -42,11 +42,11 @@ struct Settings; /// Manages access control entities. -class AccessControlManager : public MultipleAccessStorage +class AccessControl : public MultipleAccessStorage { public: - AccessControlManager(); - ~AccessControlManager() override; + AccessControl(); + ~AccessControl() override; /// Parses access entities from a configuration loaded from users.xml. /// This function add UsersConfigAccessStorage if it wasn't added before. diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index a186ffb3495..e9164b4ae44 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -142,13 +142,13 @@ namespace } -ContextAccess::ContextAccess(const AccessControlManager & manager_, const Params & params_) - : manager(&manager_) +ContextAccess::ContextAccess(const AccessControl & access_control_, const Params & params_) + : access_control(&access_control_) , params(params_) { std::lock_guard lock{mutex}; - subscription_for_user_change = manager->subscribeForChanges( + subscription_for_user_change = access_control->subscribeForChanges( *params.user_id, [this](const UUID &, const AccessEntityPtr & entity) { UserPtr changed_user = entity ? typeid_cast(entity) : nullptr; @@ -156,7 +156,7 @@ ContextAccess::ContextAccess(const AccessControlManager & manager_, const Params setUser(changed_user); }); - setUser(manager->read(*params.user_id)); + setUser(access_control->read(*params.user_id)); } @@ -194,7 +194,7 @@ void ContextAccess::setUser(const UserPtr & user_) const } subscription_for_roles_changes.reset(); - enabled_roles = manager->getEnabledRoles(current_roles, current_roles_with_admin_option); + enabled_roles = access_control->getEnabledRoles(current_roles, current_roles_with_admin_option); subscription_for_roles_changes = enabled_roles->subscribeForChanges([this](const std::shared_ptr & roles_info_) { std::lock_guard lock{mutex}; @@ -209,11 +209,11 @@ void ContextAccess::setRolesInfo(const std::shared_ptr & { assert(roles_info_); roles_info = roles_info_; - enabled_row_policies = manager->getEnabledRowPolicies( + enabled_row_policies = access_control->getEnabledRowPolicies( *params.user_id, roles_info->enabled_roles); - enabled_quota = manager->getEnabledQuota( + enabled_quota = access_control->getEnabledQuota( *params.user_id, user_name, roles_info->enabled_roles, params.address, params.forwarded_address, params.quota_key); - enabled_settings = manager->getEnabledSettings( + enabled_settings = access_control->getEnabledSettings( *params.user_id, user->settings, roles_info->enabled_roles, roles_info->settings_from_enabled_roles); calculateAccessRights(); } @@ -327,7 +327,7 @@ std::shared_ptr ContextAccess::getDefaultProfileInfo std::lock_guard lock{mutex}; if (enabled_settings) return enabled_settings->getInfo(); - static const auto everything_by_default = std::make_shared(*manager); + static const auto everything_by_default = std::make_shared(*access_control); return everything_by_default; } @@ -609,7 +609,7 @@ bool ContextAccess::checkAdminOptionImplHelper(const Container & role_ids, const template bool ContextAccess::checkAdminOptionImpl(const UUID & role_id) const { - return checkAdminOptionImplHelper(to_array(role_id), [this](const UUID & id, size_t) { return manager->tryReadName(id); }); + return checkAdminOptionImplHelper(to_array(role_id), [this](const UUID & id, size_t) { return access_control->tryReadName(id); }); } template @@ -627,7 +627,7 @@ bool ContextAccess::checkAdminOptionImpl(const UUID & role_id, const std::unorde template bool ContextAccess::checkAdminOptionImpl(const std::vector & role_ids) const { - return checkAdminOptionImplHelper(role_ids, [this](const UUID & id, size_t) { return manager->tryReadName(id); }); + return checkAdminOptionImplHelper(role_ids, [this](const UUID & id, size_t) { return access_control->tryReadName(id); }); } template diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index 8f5f5a096e7..14cac3a828f 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -25,7 +25,7 @@ struct QuotaUsage; struct Settings; struct SettingsProfilesInfo; class SettingsChanges; -class AccessControlManager; +class AccessControl; class IAST; using ASTPtr = std::shared_ptr; @@ -155,9 +155,9 @@ public: static std::shared_ptr getFullAccess(); private: - friend class AccessControlManager; + friend class AccessControl; ContextAccess() {} - ContextAccess(const AccessControlManager & manager_, const Params & params_); + ContextAccess(const AccessControl & access_control_, const Params & params_); void setUser(const UserPtr & user_) const; void setRolesInfo(const std::shared_ptr & roles_info_) const; @@ -203,7 +203,7 @@ private: template bool checkAdminOptionImplHelper(const Container & role_ids, const GetNameFunction & get_name_function) const; - const AccessControlManager * manager = nullptr; + const AccessControl * access_control = nullptr; const Params params; bool is_full_access = false; mutable Poco::Logger * trace_log = nullptr; diff --git a/src/Access/LDAPAccessStorage.cpp b/src/Access/LDAPAccessStorage.cpp index 182e391cc77..92c9b15612b 100644 --- a/src/Access/LDAPAccessStorage.cpp +++ b/src/Access/LDAPAccessStorage.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -27,10 +27,10 @@ namespace ErrorCodes } -LDAPAccessStorage::LDAPAccessStorage(const String & storage_name_, AccessControlManager * access_control_manager_, const Poco::Util::AbstractConfiguration & config, const String & prefix) +LDAPAccessStorage::LDAPAccessStorage(const String & storage_name_, AccessControl * access_control_, const Poco::Util::AbstractConfiguration & config, const String & prefix) : IAccessStorage(storage_name_) { - setConfiguration(access_control_manager_, config, prefix); + setConfiguration(access_control_, config, prefix); } @@ -40,7 +40,7 @@ String LDAPAccessStorage::getLDAPServerName() const } -void LDAPAccessStorage::setConfiguration(AccessControlManager * access_control_manager_, const Poco::Util::AbstractConfiguration & config, const String & prefix) +void LDAPAccessStorage::setConfiguration(AccessControl * access_control_, const Poco::Util::AbstractConfiguration & config, const String & prefix) { std::scoped_lock lock(mutex); @@ -80,7 +80,7 @@ void LDAPAccessStorage::setConfiguration(AccessControlManager * access_control_m } } - access_control_manager = access_control_manager_; + access_control = access_control_; ldap_server_name = ldap_server_name_cfg; role_search_params.swap(role_search_params_cfg); common_role_names.swap(common_roles_cfg); @@ -91,7 +91,7 @@ void LDAPAccessStorage::setConfiguration(AccessControlManager * access_control_m granted_role_names.clear(); granted_role_ids.clear(); - role_change_subscription = access_control_manager->subscribeForChanges( + role_change_subscription = access_control->subscribeForChanges( [this] (const UUID & id, const AccessEntityPtr & entity) { return this->processRoleChange(id, entity); @@ -215,7 +215,7 @@ void LDAPAccessStorage::assignRolesNoLock(User & user, const LDAPClient::SearchR auto it = granted_role_ids.find(role_name); if (it == granted_role_ids.end()) { - if (const auto role_id = access_control_manager->find(role_name)) + if (const auto role_id = access_control->find(role_name)) { granted_role_names.insert_or_assign(*role_id, role_name); it = granted_role_ids.insert_or_assign(role_name, *role_id).first; diff --git a/src/Access/LDAPAccessStorage.h b/src/Access/LDAPAccessStorage.h index 6cfff07b436..c1512117186 100644 --- a/src/Access/LDAPAccessStorage.h +++ b/src/Access/LDAPAccessStorage.h @@ -22,7 +22,7 @@ namespace Poco namespace DB { -class AccessControlManager; +class AccessControl; /// Implementation of IAccessStorage which allows attaching users from a remote LDAP server. /// Currently, any user name will be treated as a name of an existing remote user, @@ -32,7 +32,7 @@ class LDAPAccessStorage : public IAccessStorage public: static constexpr char STORAGE_TYPE[] = "ldap"; - explicit LDAPAccessStorage(const String & storage_name_, AccessControlManager * access_control_manager_, const Poco::Util::AbstractConfiguration & config, const String & prefix); + explicit LDAPAccessStorage(const String & storage_name_, AccessControl * access_control_, const Poco::Util::AbstractConfiguration & config, const String & prefix); virtual ~LDAPAccessStorage() override = default; String getLDAPServerName() const; @@ -59,7 +59,7 @@ private: // IAccessStorage implementations. virtual UUID getIDOfLoggedUserImpl(const String & user_name) const override; private: - void setConfiguration(AccessControlManager * access_control_manager_, const Poco::Util::AbstractConfiguration & config, const String & prefix); + void setConfiguration(AccessControl * access_control_, const Poco::Util::AbstractConfiguration & config, const String & prefix); void processRoleChange(const UUID & id, const AccessEntityPtr & entity); void applyRoleChangeNoLock(bool grant, const UUID & role_id, const String & role_name); @@ -71,7 +71,7 @@ private: const ExternalAuthenticators & external_authenticators, LDAPClient::SearchResultsList & role_search_results) const; mutable std::recursive_mutex mutex; - AccessControlManager * access_control_manager = nullptr; + AccessControl * access_control = nullptr; String ldap_server_name; LDAPClient::RoleSearchParamsList role_search_params; std::set common_role_names; // role name that should be granted to all users at all times diff --git a/src/Access/QuotaCache.cpp b/src/Access/QuotaCache.cpp index f742751d5fd..e5fa9114a51 100644 --- a/src/Access/QuotaCache.cpp +++ b/src/Access/QuotaCache.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include @@ -172,8 +172,8 @@ boost::shared_ptr QuotaCache::QuotaInfo::rebuildI } -QuotaCache::QuotaCache(const AccessControlManager & access_control_manager_) - : access_control_manager(access_control_manager_) +QuotaCache::QuotaCache(const AccessControl & access_control_) + : access_control(access_control_) { } @@ -215,7 +215,7 @@ void QuotaCache::ensureAllQuotasRead() return; all_quotas_read = true; - subscription = access_control_manager.subscribeForChanges( + subscription = access_control.subscribeForChanges( [&](const UUID & id, const AccessEntityPtr & entity) { if (entity) @@ -224,9 +224,9 @@ void QuotaCache::ensureAllQuotasRead() quotaRemoved(id); }); - for (const UUID & quota_id : access_control_manager.findAll()) + for (const UUID & quota_id : access_control.findAll()) { - auto quota = access_control_manager.tryRead(quota_id); + auto quota = access_control.tryRead(quota_id); if (quota) all_quotas.emplace(quota_id, QuotaInfo(quota, quota_id)); } diff --git a/src/Access/QuotaCache.h b/src/Access/QuotaCache.h index e991399d2e9..487c7a26487 100644 --- a/src/Access/QuotaCache.h +++ b/src/Access/QuotaCache.h @@ -10,14 +10,14 @@ namespace DB { -class AccessControlManager; +class AccessControl; /// Stores information how much amount of resources have been consumed and how much are left. class QuotaCache { public: - QuotaCache(const AccessControlManager & access_control_manager_); + QuotaCache(const AccessControl & access_control_); ~QuotaCache(); std::shared_ptr getEnabledQuota( @@ -56,7 +56,7 @@ private: void chooseQuotaToConsume(); void chooseQuotaToConsumeFor(EnabledQuota & enabled_quota); - const AccessControlManager & access_control_manager; + const AccessControl & access_control; mutable std::mutex mutex; std::unordered_map all_quotas; bool all_quotas_read = false; diff --git a/src/Access/RoleCache.cpp b/src/Access/RoleCache.cpp index e9c88868e8c..f0e1435e299 100644 --- a/src/Access/RoleCache.cpp +++ b/src/Access/RoleCache.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include @@ -56,8 +56,8 @@ namespace } -RoleCache::RoleCache(const AccessControlManager & manager_) - : manager(manager_), cache(600000 /* 10 minutes */) {} +RoleCache::RoleCache(const AccessControl & access_control_) + : access_control(access_control_), cache(600000 /* 10 minutes */) {} RoleCache::~RoleCache() = default; @@ -136,7 +136,7 @@ RolePtr RoleCache::getRole(const UUID & role_id) if (role_from_cache) return role_from_cache->first; - auto subscription = manager.subscribeForChanges(role_id, + auto subscription = access_control.subscribeForChanges(role_id, [this, role_id](const UUID &, const AccessEntityPtr & entity) { auto changed_role = entity ? typeid_cast(entity) : nullptr; @@ -146,7 +146,7 @@ RolePtr RoleCache::getRole(const UUID & role_id) roleRemoved(role_id); }); - auto role = manager.tryRead(role_id); + auto role = access_control.tryRead(role_id); if (role) { auto cache_value = Poco::SharedPtr>( diff --git a/src/Access/RoleCache.h b/src/Access/RoleCache.h index 0a1d03c1a80..42f4eec5b49 100644 --- a/src/Access/RoleCache.h +++ b/src/Access/RoleCache.h @@ -9,14 +9,14 @@ namespace DB { -class AccessControlManager; +class AccessControl; struct Role; using RolePtr = std::shared_ptr; class RoleCache { public: - RoleCache(const AccessControlManager & manager_); + RoleCache(const AccessControl & access_control_); ~RoleCache(); std::shared_ptr getEnabledRoles( @@ -30,7 +30,7 @@ private: void roleChanged(const UUID & role_id, const RolePtr & changed_role); void roleRemoved(const UUID & role_id); - const AccessControlManager & manager; + const AccessControl & access_control; Poco::ExpireCache> cache; std::map> enabled_roles; mutable std::mutex mutex; diff --git a/src/Access/RolesOrUsersSet.cpp b/src/Access/RolesOrUsersSet.cpp index 97acd5c1a76..810198eeb98 100644 --- a/src/Access/RolesOrUsersSet.cpp +++ b/src/Access/RolesOrUsersSet.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include @@ -53,40 +53,40 @@ RolesOrUsersSet::RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const std::opti init(ast, nullptr, current_user_id); } -RolesOrUsersSet::RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const AccessControlManager & manager) +RolesOrUsersSet::RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const AccessControl & access_control) { - init(ast, &manager); + init(ast, &access_control); } -RolesOrUsersSet::RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const AccessControlManager & manager, const std::optional & current_user_id) +RolesOrUsersSet::RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const AccessControl & access_control, const std::optional & current_user_id) { - init(ast, &manager, current_user_id); + init(ast, &access_control, current_user_id); } -void RolesOrUsersSet::init(const ASTRolesOrUsersSet & ast, const AccessControlManager * manager, const std::optional & current_user_id) +void RolesOrUsersSet::init(const ASTRolesOrUsersSet & ast, const AccessControl * access_control, const std::optional & current_user_id) { all = ast.all; - auto name_to_id = [&ast, manager](const String & name) -> UUID + auto name_to_id = [&ast, access_control](const String & name) -> UUID { if (ast.id_mode) return parse(name); - assert(manager); + assert(access_control); if (ast.allow_users && ast.allow_roles) { - auto id = manager->find(name); + auto id = access_control->find(name); if (id) return *id; - return manager->getID(name); + return access_control->getID(name); } else if (ast.allow_users) { - return manager->getID(name); + return access_control->getID(name); } else { assert(ast.allow_roles); - return manager->getID(name); + return access_control->getID(name); } }; @@ -147,7 +147,7 @@ std::shared_ptr RolesOrUsersSet::toAST() const } -std::shared_ptr RolesOrUsersSet::toASTWithNames(const AccessControlManager & manager) const +std::shared_ptr RolesOrUsersSet::toASTWithNames(const AccessControl & access_control) const { auto ast = std::make_shared(); ast->all = all; @@ -157,7 +157,7 @@ std::shared_ptr RolesOrUsersSet::toASTWithNames(const Access ast->names.reserve(ids.size()); for (const UUID & id : ids) { - auto name = manager.tryReadName(id); + auto name = access_control.tryReadName(id); if (name) ast->names.emplace_back(std::move(*name)); } @@ -169,7 +169,7 @@ std::shared_ptr RolesOrUsersSet::toASTWithNames(const Access ast->except_names.reserve(except_ids.size()); for (const UUID & except_id : except_ids) { - auto except_name = manager.tryReadName(except_id); + auto except_name = access_control.tryReadName(except_id); if (except_name) ast->except_names.emplace_back(std::move(*except_name)); } @@ -187,9 +187,9 @@ String RolesOrUsersSet::toString() const } -String RolesOrUsersSet::toStringWithNames(const AccessControlManager & manager) const +String RolesOrUsersSet::toStringWithNames(const AccessControl & access_control) const { - auto ast = toASTWithNames(manager); + auto ast = toASTWithNames(access_control); return serializeAST(*ast); } @@ -253,25 +253,25 @@ bool RolesOrUsersSet::match(const UUID & user_id, const boost::container::flat_s std::vector RolesOrUsersSet::getMatchingIDs() const { if (all) - throw Exception("getAllMatchingIDs() can't get ALL ids without manager", ErrorCodes::LOGICAL_ERROR); + throw Exception("getAllMatchingIDs() can't get ALL ids without access_control", ErrorCodes::LOGICAL_ERROR); std::vector res; boost::range::set_difference(ids, except_ids, std::back_inserter(res)); return res; } -std::vector RolesOrUsersSet::getMatchingIDs(const AccessControlManager & manager) const +std::vector RolesOrUsersSet::getMatchingIDs(const AccessControl & access_control) const { if (!all) return getMatchingIDs(); std::vector res; - for (const UUID & id : manager.findAll()) + for (const UUID & id : access_control.findAll()) { if (match(id)) res.push_back(id); } - for (const UUID & id : manager.findAll()) + for (const UUID & id : access_control.findAll()) { if (match(id)) res.push_back(id); diff --git a/src/Access/RolesOrUsersSet.h b/src/Access/RolesOrUsersSet.h index 871bb0c0758..1d5842e31a6 100644 --- a/src/Access/RolesOrUsersSet.h +++ b/src/Access/RolesOrUsersSet.h @@ -10,7 +10,7 @@ namespace DB { class ASTRolesOrUsersSet; -class AccessControlManager; +class AccessControl; /// Represents a set of users/roles like @@ -31,18 +31,18 @@ struct RolesOrUsersSet RolesOrUsersSet(const UUID & id); RolesOrUsersSet(const std::vector & ids_); - /// The constructor from AST requires the AccessControlManager if `ast.id_mode == false`. + /// The constructor from AST requires the AccessControl if `ast.id_mode == false`. RolesOrUsersSet(const ASTRolesOrUsersSet & ast); RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const std::optional & current_user_id); - RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const AccessControlManager & manager); - RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const AccessControlManager & manager, const std::optional & current_user_id); + RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const AccessControl & access_control); + RolesOrUsersSet(const ASTRolesOrUsersSet & ast, const AccessControl & access_control, const std::optional & current_user_id); std::shared_ptr toAST() const; - std::shared_ptr toASTWithNames(const AccessControlManager & manager) const; + std::shared_ptr toASTWithNames(const AccessControl & access_control) const; String toString() const; - String toStringWithNames(const AccessControlManager & manager) const; - Strings toStringsWithNames(const AccessControlManager & manager) const; + String toStringWithNames(const AccessControl & access_control) const; + Strings toStringsWithNames(const AccessControl & access_control) const; bool empty() const; void clear(); @@ -57,7 +57,7 @@ struct RolesOrUsersSet std::vector getMatchingIDs() const; /// Returns a list of matching users and roles. - std::vector getMatchingIDs(const AccessControlManager & manager) const; + std::vector getMatchingIDs(const AccessControl & access_control) const; friend bool operator ==(const RolesOrUsersSet & lhs, const RolesOrUsersSet & rhs); friend bool operator !=(const RolesOrUsersSet & lhs, const RolesOrUsersSet & rhs) { return !(lhs == rhs); } @@ -67,7 +67,7 @@ struct RolesOrUsersSet boost::container::flat_set except_ids; private: - void init(const ASTRolesOrUsersSet & ast, const AccessControlManager * manager = nullptr, const std::optional & current_user_id = {}); + void init(const ASTRolesOrUsersSet & ast, const AccessControl * access_control = nullptr, const std::optional & current_user_id = {}); }; } diff --git a/src/Access/RowPolicyCache.cpp b/src/Access/RowPolicyCache.cpp index c8402dbdbd4..b5b6dd99438 100644 --- a/src/Access/RowPolicyCache.cpp +++ b/src/Access/RowPolicyCache.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include @@ -92,8 +92,8 @@ void RowPolicyCache::PolicyInfo::setPolicy(const RowPolicyPtr & policy_) } -RowPolicyCache::RowPolicyCache(const AccessControlManager & access_control_manager_) - : access_control_manager(access_control_manager_) +RowPolicyCache::RowPolicyCache(const AccessControl & access_control_) + : access_control(access_control_) { } @@ -131,7 +131,7 @@ void RowPolicyCache::ensureAllRowPoliciesRead() return; all_policies_read = true; - subscription = access_control_manager.subscribeForChanges( + subscription = access_control.subscribeForChanges( [&](const UUID & id, const AccessEntityPtr & entity) { if (entity) @@ -140,9 +140,9 @@ void RowPolicyCache::ensureAllRowPoliciesRead() rowPolicyRemoved(id); }); - for (const UUID & id : access_control_manager.findAll()) + for (const UUID & id : access_control.findAll()) { - auto quota = access_control_manager.tryRead(id); + auto quota = access_control.tryRead(id); if (quota) all_policies.emplace(id, PolicyInfo(quota)); } diff --git a/src/Access/RowPolicyCache.h b/src/Access/RowPolicyCache.h index 959eb989d4c..6834def58b6 100644 --- a/src/Access/RowPolicyCache.h +++ b/src/Access/RowPolicyCache.h @@ -9,13 +9,13 @@ namespace DB { -class AccessControlManager; +class AccessControl; /// Stores read and parsed row policies. class RowPolicyCache { public: - RowPolicyCache(const AccessControlManager & access_control_manager_); + RowPolicyCache(const AccessControl & access_control_); ~RowPolicyCache(); std::shared_ptr getEnabledRowPolicies(const UUID & user_id, const boost::container::flat_set & enabled_roles); @@ -38,7 +38,7 @@ private: void mixConditions(); void mixConditionsFor(EnabledRowPolicies & enabled); - const AccessControlManager & access_control_manager; + const AccessControl & access_control; std::unordered_map all_policies; bool all_policies_read = false; scope_guard subscription; diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 316f869fc79..be59fc13136 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -18,7 +18,7 @@ namespace ErrorCodes } -SettingsConstraints::SettingsConstraints(const AccessControlManager & manager_) : manager(&manager_) +SettingsConstraints::SettingsConstraints(const AccessControl & access_control_) : access_control(&access_control_) { } @@ -200,8 +200,8 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh }; if (reaction == THROW_ON_VIOLATION) - manager->checkSettingNameIsAllowed(setting_name); - else if (!manager->isSettingNameAllowed(setting_name)) + access_control->checkSettingNameIsAllowed(setting_name); + else if (!access_control->isSettingNameAllowed(setting_name)) return false; Field current_value, new_value; diff --git a/src/Access/SettingsConstraints.h b/src/Access/SettingsConstraints.h index 4259fe15e25..ec0421e060d 100644 --- a/src/Access/SettingsConstraints.h +++ b/src/Access/SettingsConstraints.h @@ -14,7 +14,7 @@ namespace DB struct Settings; struct SettingChange; class SettingsChanges; -class AccessControlManager; +class AccessControl; /** Checks if specified changes of settings are allowed or not. @@ -51,7 +51,7 @@ class AccessControlManager; class SettingsConstraints { public: - SettingsConstraints(const AccessControlManager & manager_); + SettingsConstraints(const AccessControl & access_control_); SettingsConstraints(const SettingsConstraints & src); SettingsConstraints & operator =(const SettingsConstraints & src); SettingsConstraints(SettingsConstraints && src); @@ -109,7 +109,7 @@ private: const Constraint * tryGetConstraint(const std::string_view & setting_name) const; std::unordered_map constraints; - const AccessControlManager * manager = nullptr; + const AccessControl * access_control = nullptr; }; } diff --git a/src/Access/SettingsConstraintsAndProfileIDs.h b/src/Access/SettingsConstraintsAndProfileIDs.h index 5538a10555e..c1f3f59af6b 100644 --- a/src/Access/SettingsConstraintsAndProfileIDs.h +++ b/src/Access/SettingsConstraintsAndProfileIDs.h @@ -15,7 +15,7 @@ struct SettingsConstraintsAndProfileIDs std::vector current_profiles; std::vector enabled_profiles; - SettingsConstraintsAndProfileIDs(const AccessControlManager & manager_) : constraints(manager_) {} + SettingsConstraintsAndProfileIDs(const AccessControl & access_control_) : constraints(access_control_) {} }; } diff --git a/src/Access/SettingsProfileElement.cpp b/src/Access/SettingsProfileElement.cpp index a5d6dbf60f7..ea6edef94a6 100644 --- a/src/Access/SettingsProfileElement.cpp +++ b/src/Access/SettingsProfileElement.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include @@ -17,19 +17,19 @@ SettingsProfileElement::SettingsProfileElement(const ASTSettingsProfileElement & init(ast, nullptr); } -SettingsProfileElement::SettingsProfileElement(const ASTSettingsProfileElement & ast, const AccessControlManager & manager) +SettingsProfileElement::SettingsProfileElement(const ASTSettingsProfileElement & ast, const AccessControl & access_control) { - init(ast, &manager); + init(ast, &access_control); } -void SettingsProfileElement::init(const ASTSettingsProfileElement & ast, const AccessControlManager * manager) +void SettingsProfileElement::init(const ASTSettingsProfileElement & ast, const AccessControl * access_control) { - auto name_to_id = [id_mode{ast.id_mode}, manager](const String & name_) -> UUID + auto name_to_id = [id_mode{ast.id_mode}, access_control](const String & name_) -> UUID { if (id_mode) return parse(name_); - assert(manager); - return manager->getID(name_); + assert(access_control); + return access_control->getID(name_); }; if (!ast.parent_profile.empty()) @@ -40,8 +40,8 @@ void SettingsProfileElement::init(const ASTSettingsProfileElement & ast, const A setting_name = ast.setting_name; /// Optionally check if a setting with that name is allowed. - if (manager) - manager->checkSettingNameIsAllowed(setting_name); + if (access_control) + access_control->checkSettingNameIsAllowed(setting_name); value = ast.value; min_value = ast.min_value; @@ -76,13 +76,13 @@ std::shared_ptr SettingsProfileElement::toAST() const } -std::shared_ptr SettingsProfileElement::toASTWithNames(const AccessControlManager & manager) const +std::shared_ptr SettingsProfileElement::toASTWithNames(const AccessControl & access_control) const { auto ast = std::make_shared(); if (parent_profile) { - auto parent_profile_name = manager.tryReadName(*parent_profile); + auto parent_profile_name = access_control.tryReadName(*parent_profile); if (parent_profile_name) ast->parent_profile = *parent_profile_name; } @@ -103,10 +103,10 @@ SettingsProfileElements::SettingsProfileElements(const ASTSettingsProfileElement emplace_back(*ast_element); } -SettingsProfileElements::SettingsProfileElements(const ASTSettingsProfileElements & ast, const AccessControlManager & manager) +SettingsProfileElements::SettingsProfileElements(const ASTSettingsProfileElements & ast, const AccessControl & access_control) { for (const auto & ast_element : ast.elements) - emplace_back(*ast_element, manager); + emplace_back(*ast_element, access_control); } @@ -118,11 +118,11 @@ std::shared_ptr SettingsProfileElements::toAST() con return res; } -std::shared_ptr SettingsProfileElements::toASTWithNames(const AccessControlManager & manager) const +std::shared_ptr SettingsProfileElements::toASTWithNames(const AccessControl & access_control) const { auto res = std::make_shared(); for (const auto & element : *this) - res->elements.push_back(element.toASTWithNames(manager)); + res->elements.push_back(element.toASTWithNames(access_control)); return res; } @@ -155,9 +155,9 @@ SettingsChanges SettingsProfileElements::toSettingsChanges() const return res; } -SettingsConstraints SettingsProfileElements::toSettingsConstraints(const AccessControlManager & manager) const +SettingsConstraints SettingsProfileElements::toSettingsConstraints(const AccessControl & access_control) const { - SettingsConstraints res{manager}; + SettingsConstraints res{access_control}; for (const auto & elem : *this) { if (!elem.setting_name.empty()) diff --git a/src/Access/SettingsProfileElement.h b/src/Access/SettingsProfileElement.h index d0e2343e726..aadc5fc9e3f 100644 --- a/src/Access/SettingsProfileElement.h +++ b/src/Access/SettingsProfileElement.h @@ -13,7 +13,7 @@ class SettingsChanges; class SettingsConstraints; class ASTSettingsProfileElement; class ASTSettingsProfileElements; -class AccessControlManager; +class AccessControl; struct SettingsProfileElement @@ -35,14 +35,14 @@ struct SettingsProfileElement SettingsProfileElement() {} - /// The constructor from AST requires the AccessControlManager if `ast.id_mode == false`. + /// The constructor from AST requires the AccessControl if `ast.id_mode == false`. SettingsProfileElement(const ASTSettingsProfileElement & ast); - SettingsProfileElement(const ASTSettingsProfileElement & ast, const AccessControlManager & manager); + SettingsProfileElement(const ASTSettingsProfileElement & ast, const AccessControl & access_control); std::shared_ptr toAST() const; - std::shared_ptr toASTWithNames(const AccessControlManager & manager) const; + std::shared_ptr toASTWithNames(const AccessControl & access_control) const; private: - void init(const ASTSettingsProfileElement & ast, const AccessControlManager * manager); + void init(const ASTSettingsProfileElement & ast, const AccessControl * access_control); }; @@ -51,17 +51,17 @@ class SettingsProfileElements : public std::vector public: SettingsProfileElements() {} - /// The constructor from AST requires the AccessControlManager if `ast.id_mode == false`. + /// The constructor from AST requires the AccessControl if `ast.id_mode == false`. SettingsProfileElements(const ASTSettingsProfileElements & ast); - SettingsProfileElements(const ASTSettingsProfileElements & ast, const AccessControlManager & manager); + SettingsProfileElements(const ASTSettingsProfileElements & ast, const AccessControl & access_control); std::shared_ptr toAST() const; - std::shared_ptr toASTWithNames(const AccessControlManager & manager) const; + std::shared_ptr toASTWithNames(const AccessControl & access_control) const; void merge(const SettingsProfileElements & other); Settings toSettings() const; SettingsChanges toSettingsChanges() const; - SettingsConstraints toSettingsConstraints(const AccessControlManager & manager) const; + SettingsConstraints toSettingsConstraints(const AccessControl & access_control) const; std::vector toProfileIDs() const; }; diff --git a/src/Access/SettingsProfilesCache.cpp b/src/Access/SettingsProfilesCache.cpp index 3cd73720c3e..2a3dedbbd7a 100644 --- a/src/Access/SettingsProfilesCache.cpp +++ b/src/Access/SettingsProfilesCache.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -12,8 +12,8 @@ namespace ErrorCodes extern const int THERE_IS_NO_PROFILE; } -SettingsProfilesCache::SettingsProfilesCache(const AccessControlManager & manager_) - : manager(manager_) {} +SettingsProfilesCache::SettingsProfilesCache(const AccessControl & access_control_) + : access_control(access_control_) {} SettingsProfilesCache::~SettingsProfilesCache() = default; @@ -25,7 +25,7 @@ void SettingsProfilesCache::ensureAllProfilesRead() return; all_profiles_read = true; - subscription = manager.subscribeForChanges( + subscription = access_control.subscribeForChanges( [&](const UUID & id, const AccessEntityPtr & entity) { if (entity) @@ -34,9 +34,9 @@ void SettingsProfilesCache::ensureAllProfilesRead() profileRemoved(id); }); - for (const UUID & id : manager.findAll()) + for (const UUID & id : access_control.findAll()) { - auto profile = manager.tryRead(id); + auto profile = access_control.tryRead(id); if (profile) { all_profiles.emplace(id, profile); @@ -138,11 +138,11 @@ void SettingsProfilesCache::mergeSettingsAndConstraintsFor(EnabledSettings & ena merged_settings.merge(enabled.params.settings_from_enabled_roles); merged_settings.merge(enabled.params.settings_from_user); - auto info = std::make_shared(manager); + auto info = std::make_shared(access_control); info->profiles = enabled.params.settings_from_user.toProfileIDs(); substituteProfiles(merged_settings, info->profiles_with_implicit, info->names_of_profiles); info->settings = merged_settings.toSettingsChanges(); - info->constraints = merged_settings.toSettingsConstraints(manager); + info->constraints = merged_settings.toSettingsConstraints(access_control); enabled.setInfo(std::move(info)); } @@ -225,13 +225,13 @@ std::shared_ptr SettingsProfilesCache::getSettingsPr SettingsProfileElements elements = all_profiles[profile_id]->elements; - auto info = std::make_shared(manager); + auto info = std::make_shared(access_control); info->profiles.push_back(profile_id); info->profiles_with_implicit.push_back(profile_id); substituteProfiles(elements, info->profiles_with_implicit, info->names_of_profiles); info->settings = elements.toSettingsChanges(); - info->constraints.merge(elements.toSettingsConstraints(manager)); + info->constraints.merge(elements.toSettingsConstraints(access_control)); profile_infos_cache.add(profile_id, info); return info; diff --git a/src/Access/SettingsProfilesCache.h b/src/Access/SettingsProfilesCache.h index e5ba70b9df8..da852275ff5 100644 --- a/src/Access/SettingsProfilesCache.h +++ b/src/Access/SettingsProfilesCache.h @@ -9,7 +9,7 @@ namespace DB { -class AccessControlManager; +class AccessControl; struct SettingsProfile; using SettingsProfilePtr = std::shared_ptr; struct SettingsProfilesInfo; @@ -18,7 +18,7 @@ struct SettingsProfilesInfo; class SettingsProfilesCache { public: - SettingsProfilesCache(const AccessControlManager & manager_); + SettingsProfilesCache(const AccessControl & access_control_); ~SettingsProfilesCache(); void setDefaultProfileName(const String & default_profile_name); @@ -39,7 +39,7 @@ private: void mergeSettingsAndConstraintsFor(EnabledSettings & enabled) const; void substituteProfiles(SettingsProfileElements & elements, std::vector & substituted_profiles, std::unordered_map & names_of_substituted_profiles) const; - const AccessControlManager & manager; + const AccessControl & access_control; std::unordered_map all_profiles; std::unordered_map profiles_by_name; bool all_profiles_read = false; diff --git a/src/Access/SettingsProfilesInfo.cpp b/src/Access/SettingsProfilesInfo.cpp index 7efb16dd865..d8b139020e8 100644 --- a/src/Access/SettingsProfilesInfo.cpp +++ b/src/Access/SettingsProfilesInfo.cpp @@ -29,7 +29,7 @@ bool operator==(const SettingsProfilesInfo & lhs, const SettingsProfilesInfo & r std::shared_ptr SettingsProfilesInfo::getConstraintsAndProfileIDs(const std::shared_ptr & previous) const { - auto res = std::make_shared(manager); + auto res = std::make_shared(access_control); res->current_profiles = profiles; if (previous) diff --git a/src/Access/SettingsProfilesInfo.h b/src/Access/SettingsProfilesInfo.h index f554ba8d89e..b7f9b1da28d 100644 --- a/src/Access/SettingsProfilesInfo.h +++ b/src/Access/SettingsProfilesInfo.h @@ -29,7 +29,7 @@ struct SettingsProfilesInfo /// Names of all the profiles in `profiles`. std::unordered_map names_of_profiles; - SettingsProfilesInfo(const AccessControlManager & manager_) : constraints(manager_), manager(manager_) {} + SettingsProfilesInfo(const AccessControl & access_control_) : constraints(access_control_), access_control(access_control_) {} std::shared_ptr getConstraintsAndProfileIDs( const std::shared_ptr & previous = nullptr) const; @@ -47,7 +47,7 @@ struct SettingsProfilesInfo } private: - const AccessControlManager & manager; + const AccessControl & access_control; }; } diff --git a/src/Functions/currentProfiles.cpp b/src/Functions/currentProfiles.cpp index c578268160e..849cd026de8 100644 --- a/src/Functions/currentProfiles.cpp +++ b/src/Functions/currentProfiles.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include @@ -35,7 +35,7 @@ namespace explicit FunctionCurrentProfiles(const ContextPtr & context) { - const auto & manager = context->getAccessControlManager(); + const auto & manager = context->getAccessControl(); std::vector profile_ids; if constexpr (kind == Kind::CURRENT_PROFILES) diff --git a/src/Functions/currentRoles.cpp b/src/Functions/currentRoles.cpp index c2545edd002..f176f51f6c9 100644 --- a/src/Functions/currentRoles.cpp +++ b/src/Functions/currentRoles.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include @@ -47,7 +47,7 @@ namespace else { static_assert(kind == Kind::DEFAULT_ROLES); - const auto & manager = context->getAccessControlManager(); + const auto & manager = context->getAccessControl(); if (auto user = context->getUser()) role_names = manager.tryReadNames(user->granted_roles.findGranted(user->default_roles)); } diff --git a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp index 92b574c0036..703615972c4 100644 --- a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include @@ -73,7 +73,7 @@ namespace BlockIO InterpreterCreateQuotaQuery::execute() { auto & query = query_ptr->as(); - auto & access_control = getContext()->getAccessControlManager(); + auto & access_control = getContext()->getAccessControl(); getContext()->checkAccess(query.alter ? AccessType::ALTER_QUOTA : AccessType::CREATE_QUOTA); if (!query.cluster.empty()) diff --git a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp index 62b44449726..d623d510ffd 100644 --- a/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRoleQuery.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include @@ -34,7 +34,7 @@ namespace BlockIO InterpreterCreateRoleQuery::execute() { const auto & query = query_ptr->as(); - auto & access_control = getContext()->getAccessControlManager(); + auto & access_control = getContext()->getAccessControl(); if (query.alter) getContext()->checkAccess(AccessType::ALTER_ROLE); else diff --git a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp index b07ac0a6a10..37347b37619 100644 --- a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include @@ -44,7 +44,7 @@ namespace BlockIO InterpreterCreateRowPolicyQuery::execute() { auto & query = query_ptr->as(); - auto & access_control = getContext()->getAccessControlManager(); + auto & access_control = getContext()->getAccessControl(); getContext()->checkAccess(query.alter ? AccessType::ALTER_ROW_POLICY : AccessType::CREATE_ROW_POLICY); if (!query.cluster.empty()) diff --git a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp index fa83c6cff93..95c2a58388a 100644 --- a/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateSettingsProfileQuery.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include @@ -42,7 +42,7 @@ namespace BlockIO InterpreterCreateSettingsProfileQuery::execute() { auto & query = query_ptr->as(); - auto & access_control = getContext()->getAccessControlManager(); + auto & access_control = getContext()->getAccessControl(); if (query.alter) getContext()->checkAccess(AccessType::ALTER_SETTINGS_PROFILE); else diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index 721d31bec48..33d85afb7c3 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include @@ -79,7 +79,7 @@ namespace BlockIO InterpreterCreateUserQuery::execute() { const auto & query = query_ptr->as(); - auto & access_control = getContext()->getAccessControlManager(); + auto & access_control = getContext()->getAccessControl(); auto access = getContext()->getAccess(); access->checkAccess(query.alter ? AccessType::ALTER_USER : AccessType::CREATE_USER); diff --git a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp index 04937aec46a..c00bbe4f379 100644 --- a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include @@ -25,7 +25,7 @@ using EntityType = IAccessEntity::Type; BlockIO InterpreterDropAccessEntityQuery::execute() { auto & query = query_ptr->as(); - auto & access_control = getContext()->getAccessControlManager(); + auto & access_control = getContext()->getAccessControl(); getContext()->checkAccess(getRequiredAccess()); if (!query.cluster.empty()) diff --git a/src/Interpreters/Access/InterpreterGrantQuery.cpp b/src/Interpreters/Access/InterpreterGrantQuery.cpp index f748f9ac02e..3ace4531de6 100644 --- a/src/Interpreters/Access/InterpreterGrantQuery.cpp +++ b/src/Interpreters/Access/InterpreterGrantQuery.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include @@ -51,7 +51,7 @@ namespace /// Extracts roles which are going to be granted or revoked from a query. void collectRolesToGrantOrRevoke( - const AccessControlManager & access_control, + const AccessControl & access_control, const ASTGrantQuery & query, std::vector & roles_to_grant, RolesOrUsersSet & roles_to_revoke) @@ -121,7 +121,7 @@ namespace } /// Checks if grantees are allowed for the current user, throws an exception if not. - void checkGranteesAreAllowed(const AccessControlManager & access_control, const ContextAccess & current_user_access, const std::vector & grantee_ids) + void checkGranteesAreAllowed(const AccessControl & access_control, const ContextAccess & current_user_access, const std::vector & grantee_ids) { auto current_user = current_user_access.getUser(); if (!current_user || (current_user->grantees == RolesOrUsersSet::AllTag{})) @@ -139,7 +139,7 @@ namespace /// Checks if the current user has enough access rights granted with grant option to grant or revoke specified access rights. void checkGrantOption( - const AccessControlManager & access_control, + const AccessControl & access_control, const ContextAccess & current_user_access, const std::vector & grantees_from_query, bool & need_check_grantees_are_allowed, @@ -205,7 +205,7 @@ namespace /// Checks if the current user has enough roles granted with admin option to grant or revoke specified roles. void checkAdminOption( - const AccessControlManager & access_control, + const AccessControl & access_control, const ContextAccess & current_user_access, const std::vector & grantees_from_query, bool & need_check_grantees_are_allowed, @@ -382,7 +382,7 @@ BlockIO InterpreterGrantQuery::execute() if (!query.access_rights_elements.empty() && query.access_rights_elements[0].is_partial_revoke && !query.is_revoke) throw Exception("A partial revoke should be revoked, not granted", ErrorCodes::LOGICAL_ERROR); - auto & access_control = getContext()->getAccessControlManager(); + auto & access_control = getContext()->getAccessControl(); auto current_user_access = getContext()->getAccess(); std::vector grantees = RolesOrUsersSet{*query.grantees, access_control, getContext()->getUserID()}.getMatchingIDs(access_control); diff --git a/src/Interpreters/Access/InterpreterSetRoleQuery.cpp b/src/Interpreters/Access/InterpreterSetRoleQuery.cpp index 82b100e9d48..6acb9ab5e19 100644 --- a/src/Interpreters/Access/InterpreterSetRoleQuery.cpp +++ b/src/Interpreters/Access/InterpreterSetRoleQuery.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include @@ -28,7 +28,7 @@ BlockIO InterpreterSetRoleQuery::execute() void InterpreterSetRoleQuery::setRole(const ASTSetRoleQuery & query) { - auto & access_control = getContext()->getAccessControlManager(); + auto & access_control = getContext()->getAccessControl(); auto session_context = getContext()->getSessionContext(); auto user = session_context->getUser(); @@ -62,7 +62,7 @@ void InterpreterSetRoleQuery::setDefaultRole(const ASTSetRoleQuery & query) { getContext()->checkAccess(AccessType::ALTER_USER); - auto & access_control = getContext()->getAccessControlManager(); + auto & access_control = getContext()->getAccessControl(); std::vector to_users = RolesOrUsersSet{*query.to_users, access_control, getContext()->getUserID()}.getMatchingIDs(access_control); RolesOrUsersSet roles_from_query{*query.roles, access_control}; diff --git a/src/Interpreters/Access/InterpreterShowAccessQuery.cpp b/src/Interpreters/Access/InterpreterShowAccessQuery.cpp index 2ecd51ff2a9..a385f6c8d7a 100644 --- a/src/Interpreters/Access/InterpreterShowAccessQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowAccessQuery.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include @@ -49,7 +49,7 @@ QueryPipeline InterpreterShowAccessQuery::executeImpl() const std::vector InterpreterShowAccessQuery::getEntities() const { - const auto & access_control = getContext()->getAccessControlManager(); + const auto & access_control = getContext()->getAccessControl(); getContext()->checkAccess(AccessType::SHOW_ACCESS); std::vector entities; @@ -71,7 +71,7 @@ std::vector InterpreterShowAccessQuery::getEntities() const ASTs InterpreterShowAccessQuery::getCreateAndGrantQueries() const { auto entities = getEntities(); - const auto & access_control = getContext()->getAccessControlManager(); + const auto & access_control = getContext()->getAccessControl(); ASTs create_queries, grant_queries; for (const auto & entity : entities) diff --git a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp index b7952324819..ca6003e2cc0 100644 --- a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include #include #include @@ -40,7 +40,7 @@ namespace { ASTPtr getCreateQueryImpl( const User & user, - const AccessControlManager * manager /* not used if attach_mode == true */, + const AccessControl * access_control /* not used if attach_mode == true */, bool attach_mode) { auto query = std::make_shared(); @@ -56,7 +56,7 @@ namespace if (attach_mode) query->default_roles = user.default_roles.toAST(); else - query->default_roles = user.default_roles.toASTWithNames(*manager); + query->default_roles = user.default_roles.toASTWithNames(*access_control); } if (user.auth_data.getType() != AuthenticationType::NO_PASSWORD) @@ -70,7 +70,7 @@ namespace if (attach_mode) query->settings = user.settings.toAST(); else - query->settings = user.settings.toASTWithNames(*manager); + query->settings = user.settings.toASTWithNames(*access_control); } if (user.grantees != RolesOrUsersSet::AllTag{}) @@ -78,7 +78,7 @@ namespace if (attach_mode) query->grantees = user.grantees.toAST(); else - query->grantees = user.grantees.toASTWithNames(*manager); + query->grantees = user.grantees.toASTWithNames(*access_control); query->grantees->use_keyword_any = true; } @@ -93,7 +93,7 @@ namespace } - ASTPtr getCreateQueryImpl(const Role & role, const AccessControlManager * manager, bool attach_mode) + ASTPtr getCreateQueryImpl(const Role & role, const AccessControl * access_control, bool attach_mode) { auto query = std::make_shared(); query->names.emplace_back(role.getName()); @@ -104,14 +104,14 @@ namespace if (attach_mode) query->settings = role.settings.toAST(); else - query->settings = role.settings.toASTWithNames(*manager); + query->settings = role.settings.toASTWithNames(*access_control); } return query; } - ASTPtr getCreateQueryImpl(const SettingsProfile & profile, const AccessControlManager * manager, bool attach_mode) + ASTPtr getCreateQueryImpl(const SettingsProfile & profile, const AccessControl * access_control, bool attach_mode) { auto query = std::make_shared(); query->names.emplace_back(profile.getName()); @@ -122,7 +122,7 @@ namespace if (attach_mode) query->settings = profile.elements.toAST(); else - query->settings = profile.elements.toASTWithNames(*manager); + query->settings = profile.elements.toASTWithNames(*access_control); if (query->settings) query->settings->setUseInheritKeyword(true); } @@ -132,7 +132,7 @@ namespace if (attach_mode) query->to_roles = profile.to_roles.toAST(); else - query->to_roles = profile.to_roles.toASTWithNames(*manager); + query->to_roles = profile.to_roles.toASTWithNames(*access_control); } return query; @@ -141,7 +141,7 @@ namespace ASTPtr getCreateQueryImpl( const Quota & quota, - const AccessControlManager * manager /* not used if attach_mode == true */, + const AccessControl * access_control /* not used if attach_mode == true */, bool attach_mode) { auto query = std::make_shared(); @@ -168,7 +168,7 @@ namespace if (attach_mode) query->roles = quota.to_roles.toAST(); else - query->roles = quota.to_roles.toASTWithNames(*manager); + query->roles = quota.to_roles.toASTWithNames(*access_control); } return query; @@ -177,7 +177,7 @@ namespace ASTPtr getCreateQueryImpl( const RowPolicy & policy, - const AccessControlManager * manager /* not used if attach_mode == true */, + const AccessControl * access_control /* not used if attach_mode == true */, bool attach_mode) { auto query = std::make_shared(); @@ -204,7 +204,7 @@ namespace if (attach_mode) query->roles = policy.to_roles.toAST(); else - query->roles = policy.to_roles.toASTWithNames(*manager); + query->roles = policy.to_roles.toASTWithNames(*access_control); } return query; @@ -212,19 +212,19 @@ namespace ASTPtr getCreateQueryImpl( const IAccessEntity & entity, - const AccessControlManager * manager /* not used if attach_mode == true */, + const AccessControl * access_control /* not used if attach_mode == true */, bool attach_mode) { if (const User * user = typeid_cast(&entity)) - return getCreateQueryImpl(*user, manager, attach_mode); + return getCreateQueryImpl(*user, access_control, attach_mode); if (const Role * role = typeid_cast(&entity)) - return getCreateQueryImpl(*role, manager, attach_mode); + return getCreateQueryImpl(*role, access_control, attach_mode); if (const RowPolicy * policy = typeid_cast(&entity)) - return getCreateQueryImpl(*policy, manager, attach_mode); + return getCreateQueryImpl(*policy, access_control, attach_mode); if (const Quota * quota = typeid_cast(&entity)) - return getCreateQueryImpl(*quota, manager, attach_mode); + return getCreateQueryImpl(*quota, access_control, attach_mode); if (const SettingsProfile * profile = typeid_cast(&entity)) - return getCreateQueryImpl(*profile, manager, attach_mode); + return getCreateQueryImpl(*profile, access_control, attach_mode); throw Exception(entity.outputTypeAndName() + ": type is not supported by SHOW CREATE query", ErrorCodes::NOT_IMPLEMENTED); } @@ -277,7 +277,7 @@ QueryPipeline InterpreterShowCreateAccessEntityQuery::executeImpl() std::vector InterpreterShowCreateAccessEntityQuery::getEntities() const { auto & show_query = query_ptr->as(); - const auto & access_control = getContext()->getAccessControlManager(); + const auto & access_control = getContext()->getAccessControl(); getContext()->checkAccess(getRequiredAccess()); show_query.replaceEmptyDatabase(getContext()->getCurrentDatabase()); std::vector entities; @@ -348,7 +348,7 @@ ASTs InterpreterShowCreateAccessEntityQuery::getCreateQueries() const auto entities = getEntities(); ASTs list; - const auto & access_control = getContext()->getAccessControlManager(); + const auto & access_control = getContext()->getAccessControl(); for (const auto & entity : entities) list.push_back(getCreateQuery(*entity, access_control)); @@ -356,7 +356,7 @@ ASTs InterpreterShowCreateAccessEntityQuery::getCreateQueries() const } -ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuery(const IAccessEntity & entity, const AccessControlManager & access_control) +ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuery(const IAccessEntity & entity, const AccessControl & access_control) { return getCreateQueryImpl(entity, &access_control, false); } diff --git a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.h b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.h index 0aedeb18be4..9d84e68568b 100644 --- a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.h +++ b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.h @@ -7,7 +7,7 @@ namespace DB { -class AccessControlManager; +class AccessControl; class Context; class AccessRightsElements; struct IAccessEntity; @@ -26,7 +26,7 @@ public: bool ignoreQuota() const override { return true; } bool ignoreLimits() const override { return true; } - static ASTPtr getCreateQuery(const IAccessEntity & entity, const AccessControlManager & access_control); + static ASTPtr getCreateQuery(const IAccessEntity & entity, const AccessControl & access_control); static ASTPtr getAttachQuery(const IAccessEntity & entity); private: diff --git a/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp b/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp index 930dc0dc5da..788856dbfe0 100644 --- a/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include @@ -27,7 +27,7 @@ namespace template ASTs getGrantQueriesImpl( const T & grantee, - const AccessControlManager * manager /* not used if attach_mode == true */, + const AccessControl * access_control /* not used if attach_mode == true */, bool attach_mode = false) { ASTs res; @@ -75,7 +75,7 @@ namespace if (attach_mode) grant_query->roles = RolesOrUsersSet{element.ids}.toAST(); else - grant_query->roles = RolesOrUsersSet{element.ids}.toASTWithNames(*manager); + grant_query->roles = RolesOrUsersSet{element.ids}.toASTWithNames(*access_control); res.push_back(std::move(grant_query)); } @@ -84,13 +84,13 @@ namespace ASTs getGrantQueriesImpl( const IAccessEntity & entity, - const AccessControlManager * manager /* not used if attach_mode == true */, + const AccessControl * access_control /* not used if attach_mode == true */, bool attach_mode = false) { if (const User * user = typeid_cast(&entity)) - return getGrantQueriesImpl(*user, manager, attach_mode); + return getGrantQueriesImpl(*user, access_control, attach_mode); if (const Role * role = typeid_cast(&entity)) - return getGrantQueriesImpl(*role, manager, attach_mode); + return getGrantQueriesImpl(*role, access_control, attach_mode); throw Exception(entity.outputTypeAndName() + " is expected to be user or role", ErrorCodes::LOGICAL_ERROR); } @@ -136,7 +136,7 @@ QueryPipeline InterpreterShowGrantsQuery::executeImpl() std::vector InterpreterShowGrantsQuery::getEntities() const { const auto & show_query = query_ptr->as(); - const auto & access_control = getContext()->getAccessControlManager(); + const auto & access_control = getContext()->getAccessControl(); auto ids = RolesOrUsersSet{*show_query.for_roles, access_control, getContext()->getUserID()}.getMatchingIDs(access_control); std::vector entities; @@ -155,7 +155,7 @@ std::vector InterpreterShowGrantsQuery::getEntities() const ASTs InterpreterShowGrantsQuery::getGrantQueries() const { auto entities = getEntities(); - const auto & access_control = getContext()->getAccessControlManager(); + const auto & access_control = getContext()->getAccessControl(); ASTs grant_queries; for (const auto & entity : entities) @@ -165,7 +165,7 @@ ASTs InterpreterShowGrantsQuery::getGrantQueries() const } -ASTs InterpreterShowGrantsQuery::getGrantQueries(const IAccessEntity & user_or_role, const AccessControlManager & access_control) +ASTs InterpreterShowGrantsQuery::getGrantQueries(const IAccessEntity & user_or_role, const AccessControl & access_control) { return getGrantQueriesImpl(user_or_role, &access_control, false); } diff --git a/src/Interpreters/Access/InterpreterShowGrantsQuery.h b/src/Interpreters/Access/InterpreterShowGrantsQuery.h index 06bdcf169b1..bab147c279e 100644 --- a/src/Interpreters/Access/InterpreterShowGrantsQuery.h +++ b/src/Interpreters/Access/InterpreterShowGrantsQuery.h @@ -8,7 +8,7 @@ namespace DB { -class AccessControlManager; +class AccessControl; class ASTShowGrantsQuery; struct IAccessEntity; using AccessEntityPtr = std::shared_ptr; @@ -20,7 +20,7 @@ public: BlockIO execute() override; - static ASTs getGrantQueries(const IAccessEntity & user_or_role, const AccessControlManager & access_control); + static ASTs getGrantQueries(const IAccessEntity & user_or_role, const AccessControl & access_control); static ASTs getAttachGrantQueries(const IAccessEntity & user_or_role); bool ignoreQuota() const override { return true; } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7f31df9159c..f2d449c04d7 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -34,7 +34,7 @@ #include #include #include -#include +#include #include #include #include @@ -206,7 +206,7 @@ struct ContextSharedPart String default_profile_name; /// Default profile name used for default values. String system_profile_name; /// Profile used by system processes String buffer_profile_name; /// Profile used by Buffer engine for flushing to the underlying - std::unique_ptr access_control_manager; + std::unique_ptr access_control; mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks. mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files. mutable UncompressedCachePtr index_uncompressed_cache; /// The cache of decompressed blocks for MergeTree indices. @@ -279,7 +279,7 @@ struct ContextSharedPart Context::ConfigReloadCallback config_reload_callback; ContextSharedPart() - : access_control_manager(std::make_unique()), macros(std::make_unique()) + : access_control(std::make_unique()), macros(std::make_unique()) { /// TODO: make it singleton (?) static std::atomic num_calls{0}; @@ -371,7 +371,7 @@ struct ContextSharedPart distributed_schedule_pool.reset(); message_broker_schedule_pool.reset(); ddl_worker.reset(); - access_control_manager.reset(); + access_control.reset(); /// Stop trace collector if any trace_collector.reset(); @@ -635,7 +635,7 @@ void Context::setConfig(const ConfigurationPtr & config) { auto lock = getLock(); shared->config = config; - shared->access_control_manager->setExternalAuthenticatorsConfig(*shared->config); + shared->access_control->setExternalAuthenticatorsConfig(*shared->config); } const Poco::Util::AbstractConfiguration & Context::getConfigRef() const @@ -645,33 +645,33 @@ const Poco::Util::AbstractConfiguration & Context::getConfigRef() const } -AccessControlManager & Context::getAccessControlManager() +AccessControl & Context::getAccessControl() { - return *shared->access_control_manager; + return *shared->access_control; } -const AccessControlManager & Context::getAccessControlManager() const +const AccessControl & Context::getAccessControl() const { - return *shared->access_control_manager; + return *shared->access_control; } void Context::setExternalAuthenticatorsConfig(const Poco::Util::AbstractConfiguration & config) { auto lock = getLock(); - shared->access_control_manager->setExternalAuthenticatorsConfig(config); + shared->access_control->setExternalAuthenticatorsConfig(config); } std::unique_ptr Context::makeGSSAcceptorContext() const { auto lock = getLock(); - return std::make_unique(shared->access_control_manager->getExternalAuthenticators().getKerberosParams()); + return std::make_unique(shared->access_control->getExternalAuthenticators().getKerberosParams()); } void Context::setUsersConfig(const ConfigurationPtr & config) { auto lock = getLock(); shared->users_config = config; - shared->access_control_manager->setUsersConfig(*shared->users_config); + shared->access_control->setUsersConfig(*shared->users_config); } ConfigurationPtr Context::getUsersConfig() @@ -686,7 +686,7 @@ void Context::setUser(const UUID & user_id_) user_id = user_id_; - access = getAccessControlManager().getContextAccess( + access = getAccessControl().getContextAccess( user_id_, /* current_roles = */ {}, /* use_default_roles = */ true, settings, current_database, client_info); auto user = access->getUser(); @@ -759,7 +759,7 @@ void Context::calculateAccessRights() { auto lock = getLock(); if (user_id) - access = getAccessControlManager().getContextAccess( + access = getAccessControl().getContextAccess( *user_id, current_roles ? *current_roles : std::vector{}, /* use_default_roles = */ false, @@ -808,10 +808,10 @@ void Context::setInitialRowPolicy() initial_row_policy = nullptr; if (client_info.initial_user == client_info.current_user) return; - auto initial_user_id = getAccessControlManager().find(client_info.initial_user); + auto initial_user_id = getAccessControl().find(client_info.initial_user); if (!initial_user_id) return; - initial_row_policy = getAccessControlManager().getEnabledRowPolicies(*initial_user_id, {}); + initial_row_policy = getAccessControl().getEnabledRowPolicies(*initial_user_id, {}); } @@ -832,7 +832,7 @@ void Context::setCurrentProfile(const String & profile_name) auto lock = getLock(); try { - UUID profile_id = getAccessControlManager().getID(profile_name); + UUID profile_id = getAccessControl().getID(profile_name); setCurrentProfile(profile_id); } catch (Exception & e) @@ -845,7 +845,7 @@ void Context::setCurrentProfile(const String & profile_name) void Context::setCurrentProfile(const UUID & profile_id) { auto lock = getLock(); - auto profile_info = getAccessControlManager().getSettingsProfileInfo(profile_id); + auto profile_info = getAccessControl().getSettingsProfileInfo(profile_id); checkSettingsConstraints(profile_info->settings); applySettingsChanges(profile_info->settings); settings_constraints_and_current_profiles = profile_info->getConstraintsAndProfileIDs(settings_constraints_and_current_profiles); @@ -1153,7 +1153,7 @@ std::shared_ptr Context::getSettingsCons auto lock = getLock(); if (settings_constraints_and_current_profiles) return settings_constraints_and_current_profiles; - static auto no_constraints_or_profiles = std::make_shared(getAccessControlManager()); + static auto no_constraints_or_profiles = std::make_shared(getAccessControl()); return no_constraints_or_profiles; } @@ -2641,7 +2641,7 @@ void Context::setApplicationType(ApplicationType type) void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & config) { shared->default_profile_name = config.getString("default_profile", "default"); - getAccessControlManager().setDefaultProfileName(shared->default_profile_name); + getAccessControl().setDefaultProfileName(shared->default_profile_name); shared->system_profile_name = config.getString("system_profile", shared->default_profile_name); setCurrentProfile(shared->system_profile_name); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index cc6df875f7d..bc87f570366 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -85,7 +85,7 @@ class ActionLocksManager; using ActionLocksManagerPtr = std::shared_ptr; class ShellCommand; class ICompressionCodec; -class AccessControlManager; +class AccessControl; class Credentials; class GSSAcceptorContext; struct SettingsConstraintsAndProfileIDs; @@ -363,8 +363,8 @@ public: void setConfig(const ConfigurationPtr & config); const Poco::Util::AbstractConfiguration & getConfigRef() const; - AccessControlManager & getAccessControlManager(); - const AccessControlManager & getAccessControlManager() const; + AccessControl & getAccessControl(); + const AccessControl & getAccessControl() const; /// Sets external authenticators config (LDAP, Kerberos). void setExternalAuthenticatorsConfig(const Poco::Util::AbstractConfiguration & config); diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index ac8187980c3..00be194f1eb 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include #include @@ -273,7 +273,7 @@ Session::~Session() AuthenticationType Session::getAuthenticationType(const String & user_name) const { - return global_context->getAccessControlManager().read(user_name)->auth_data.getType(); + return global_context->getAccessControl().read(user_name)->auth_data.getType(); } AuthenticationType Session::getAuthenticationTypeOrLogInFailure(const String & user_name) const @@ -310,7 +310,7 @@ void Session::authenticate(const Credentials & credentials_, const Poco::Net::So try { - user_id = global_context->getAccessControlManager().login(credentials_, address.host()); + user_id = global_context->getAccessControl().login(credentials_, address.host()); LOG_DEBUG(log, "{} Authenticated with global context as user {}", toString(auth_id), user_id ? toString(*user_id) : ""); } diff --git a/src/Storages/System/StorageSystemGrants.cpp b/src/Storages/System/StorageSystemGrants.cpp index 42917033a28..d443830ee0a 100644 --- a/src/Storages/System/StorageSystemGrants.cpp +++ b/src/Storages/System/StorageSystemGrants.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include @@ -38,7 +38,7 @@ NamesAndTypesList StorageSystemGrants::getNamesAndTypes() void StorageSystemGrants::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { context->checkAccess(AccessType::SHOW_USERS | AccessType::SHOW_ROLES); - const auto & access_control = context->getAccessControlManager(); + const auto & access_control = context->getAccessControl(); std::vector ids = access_control.findAll(); boost::range::push_back(ids, access_control.findAll()); diff --git a/src/Storages/System/StorageSystemPrivileges.cpp b/src/Storages/System/StorageSystemPrivileges.cpp index ff3c6eecc6b..6a4d2e1087e 100644 --- a/src/Storages/System/StorageSystemPrivileges.cpp +++ b/src/Storages/System/StorageSystemPrivileges.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/src/Storages/System/StorageSystemQuotaLimits.cpp b/src/Storages/System/StorageSystemQuotaLimits.cpp index 6d837018053..330b9935b48 100644 --- a/src/Storages/System/StorageSystemQuotaLimits.cpp +++ b/src/Storages/System/StorageSystemQuotaLimits.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include @@ -72,7 +72,7 @@ NamesAndTypesList StorageSystemQuotaLimits::getNamesAndTypes() void StorageSystemQuotaLimits::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { context->checkAccess(AccessType::SHOW_QUOTAS); - const auto & access_control = context->getAccessControlManager(); + const auto & access_control = context->getAccessControl(); std::vector ids = access_control.findAll(); size_t column_index = 0; diff --git a/src/Storages/System/StorageSystemQuotaUsage.cpp b/src/Storages/System/StorageSystemQuotaUsage.cpp index 2d2c5ec2864..a08f6686030 100644 --- a/src/Storages/System/StorageSystemQuotaUsage.cpp +++ b/src/Storages/System/StorageSystemQuotaUsage.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/System/StorageSystemQuotas.cpp b/src/Storages/System/StorageSystemQuotas.cpp index eaa2f79364b..2294af87fed 100644 --- a/src/Storages/System/StorageSystemQuotas.cpp +++ b/src/Storages/System/StorageSystemQuotas.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -55,7 +55,7 @@ NamesAndTypesList StorageSystemQuotas::getNamesAndTypes() void StorageSystemQuotas::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { context->checkAccess(AccessType::SHOW_QUOTAS); - const auto & access_control = context->getAccessControlManager(); + const auto & access_control = context->getAccessControl(); std::vector ids = access_control.findAll(); size_t column_index = 0; diff --git a/src/Storages/System/StorageSystemQuotasUsage.cpp b/src/Storages/System/StorageSystemQuotasUsage.cpp index 98a77dde9f4..fae0629a209 100644 --- a/src/Storages/System/StorageSystemQuotasUsage.cpp +++ b/src/Storages/System/StorageSystemQuotasUsage.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include @@ -16,7 +16,7 @@ NamesAndTypesList StorageSystemQuotasUsage::getNamesAndTypes() void StorageSystemQuotasUsage::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { context->checkAccess(AccessType::SHOW_QUOTAS); - auto all_quotas_usage = context->getAccessControlManager().getAllQuotasUsage(); + auto all_quotas_usage = context->getAccessControl().getAllQuotasUsage(); StorageSystemQuotaUsage::fillDataImpl(res_columns, context, /* add_column_is_current = */ true, all_quotas_usage); } } diff --git a/src/Storages/System/StorageSystemRoleGrants.cpp b/src/Storages/System/StorageSystemRoleGrants.cpp index 32984afcfc5..080c73726bc 100644 --- a/src/Storages/System/StorageSystemRoleGrants.cpp +++ b/src/Storages/System/StorageSystemRoleGrants.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include @@ -34,7 +34,7 @@ NamesAndTypesList StorageSystemRoleGrants::getNamesAndTypes() void StorageSystemRoleGrants::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { context->checkAccess(AccessType::SHOW_USERS | AccessType::SHOW_ROLES); - const auto & access_control = context->getAccessControlManager(); + const auto & access_control = context->getAccessControl(); std::vector ids = access_control.findAll(); boost::range::push_back(ids, access_control.findAll()); diff --git a/src/Storages/System/StorageSystemRoles.cpp b/src/Storages/System/StorageSystemRoles.cpp index 24a5b3ede4d..fcc45d1374f 100644 --- a/src/Storages/System/StorageSystemRoles.cpp +++ b/src/Storages/System/StorageSystemRoles.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include @@ -26,7 +26,7 @@ NamesAndTypesList StorageSystemRoles::getNamesAndTypes() void StorageSystemRoles::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { context->checkAccess(AccessType::SHOW_ROLES); - const auto & access_control = context->getAccessControlManager(); + const auto & access_control = context->getAccessControl(); std::vector ids = access_control.findAll(); size_t column_index = 0; diff --git a/src/Storages/System/StorageSystemRowPolicies.cpp b/src/Storages/System/StorageSystemRowPolicies.cpp index 73323b13d79..202ec5078d8 100644 --- a/src/Storages/System/StorageSystemRowPolicies.cpp +++ b/src/Storages/System/StorageSystemRowPolicies.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -55,7 +55,7 @@ NamesAndTypesList StorageSystemRowPolicies::getNamesAndTypes() void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { context->checkAccess(AccessType::SHOW_ROW_POLICIES); - const auto & access_control = context->getAccessControlManager(); + const auto & access_control = context->getAccessControl(); std::vector ids = access_control.findAll(); size_t column_index = 0; diff --git a/src/Storages/System/StorageSystemSettingsProfileElements.cpp b/src/Storages/System/StorageSystemSettingsProfileElements.cpp index fa824091238..b2991baf9cb 100644 --- a/src/Storages/System/StorageSystemSettingsProfileElements.cpp +++ b/src/Storages/System/StorageSystemSettingsProfileElements.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include @@ -40,7 +40,7 @@ NamesAndTypesList StorageSystemSettingsProfileElements::getNamesAndTypes() void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { context->checkAccess(AccessType::SHOW_SETTINGS_PROFILES); - const auto & access_control = context->getAccessControlManager(); + const auto & access_control = context->getAccessControl(); std::vector ids = access_control.findAll(); boost::range::push_back(ids, access_control.findAll()); boost::range::push_back(ids, access_control.findAll()); diff --git a/src/Storages/System/StorageSystemSettingsProfiles.cpp b/src/Storages/System/StorageSystemSettingsProfiles.cpp index c81e9665d16..132f10ea194 100644 --- a/src/Storages/System/StorageSystemSettingsProfiles.cpp +++ b/src/Storages/System/StorageSystemSettingsProfiles.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -33,7 +33,7 @@ NamesAndTypesList StorageSystemSettingsProfiles::getNamesAndTypes() void StorageSystemSettingsProfiles::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { context->checkAccess(AccessType::SHOW_SETTINGS_PROFILES); - const auto & access_control = context->getAccessControlManager(); + const auto & access_control = context->getAccessControl(); std::vector ids = access_control.findAll(); size_t column_index = 0; diff --git a/src/Storages/System/StorageSystemUserDirectories.cpp b/src/Storages/System/StorageSystemUserDirectories.cpp index d4a2d5169fa..3c550cd1cfe 100644 --- a/src/Storages/System/StorageSystemUserDirectories.cpp +++ b/src/Storages/System/StorageSystemUserDirectories.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include namespace DB @@ -23,7 +23,7 @@ NamesAndTypesList StorageSystemUserDirectories::getNamesAndTypes() void StorageSystemUserDirectories::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - const auto & access_control = context->getAccessControlManager(); + const auto & access_control = context->getAccessControl(); auto storages = access_control.getStorages(); size_t column_index = 0; diff --git a/src/Storages/System/StorageSystemUsers.cpp b/src/Storages/System/StorageSystemUsers.cpp index 135bde2a3ff..ca88fa688a0 100644 --- a/src/Storages/System/StorageSystemUsers.cpp +++ b/src/Storages/System/StorageSystemUsers.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -59,7 +59,7 @@ NamesAndTypesList StorageSystemUsers::getNamesAndTypes() void StorageSystemUsers::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { context->checkAccess(AccessType::SHOW_USERS); - const auto & access_control = context->getAccessControlManager(); + const auto & access_control = context->getAccessControl(); std::vector ids = access_control.findAll(); size_t column_index = 0; From 79ea35ec492b5c7bac23c869052c4266b66dcc2a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 2 Nov 2021 14:38:02 +0300 Subject: [PATCH 376/396] Update README.md --- utils/changelog/README.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/utils/changelog/README.md b/utils/changelog/README.md index 69a190fdedc..b52b1e1707b 100644 --- a/utils/changelog/README.md +++ b/utils/changelog/README.md @@ -8,6 +8,10 @@ Dependencies: apt-get install git curl jq python3 python3-fuzzywuzzy ``` +Update information about tags: +``` +git fetch --tags +``` Usage example: From ad06ed3526fd66bf5d77a31744a6f2b8aa40ce8d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 2 Nov 2021 14:38:17 +0300 Subject: [PATCH 377/396] Update README.md --- utils/changelog/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/changelog/README.md b/utils/changelog/README.md index b52b1e1707b..28135d088ac 100644 --- a/utils/changelog/README.md +++ b/utils/changelog/README.md @@ -1,4 +1,4 @@ -## Generate changelog +## How To Generate Changelog Generate github token: * https://github.com/settings/tokens - keep all checkboxes unchecked, no scopes need to be enabled. From 14e7ea6685b9b723dd66ba640ce00feda87af6a6 Mon Sep 17 00:00:00 2001 From: Yuriy Chernyshov Date: Tue, 2 Nov 2021 14:41:30 +0300 Subject: [PATCH 378/396] Remove remaining usages of Y_IGNORE --- programs/server/Server.cpp | 2 +- utils/keeper-data-dumper/main.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ce7d6973f68..ec5b01048a6 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -86,7 +86,7 @@ # include "config_core.h" # include "Common/config_version.h" # if USE_OPENCL -# include "Common/BitonicSort.h" // Y_IGNORE +# include "Common/BitonicSort.h" # endif #endif diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index ed6a7aea972..485f8c09faf 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -4,7 +4,7 @@ #include #include #include -#include // Y_IGNORE +#include #include #include #include From 3b12c54bf96cb2596183f6b4c45e45abafaf2236 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 2 Nov 2021 15:05:59 +0300 Subject: [PATCH 379/396] Fix PVS check --- .../Serializations/SerializationNullable.cpp | 50 +++++++++---------- 1 file changed, 25 insertions(+), 25 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index acec3790196..9ae78403d11 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -285,7 +285,7 @@ ReturnType SerializationNullable::deserializeTextEscapedImpl(IColumn & column, R template ReturnType SerializationNullable::deserializeTextEscapedAndRawImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, - const SerializationPtr & nested) + const SerializationPtr & nested_serialization) { const String & null_representation = settings.tsv.null_representation; @@ -293,14 +293,14 @@ ReturnType SerializationNullable::deserializeTextEscapedAndRawImpl(IColumn & col if (istr.eof() || (!null_representation.empty() && *istr.position() != null_representation[0])) { /// This is not null, surely. - return safeDeserialize(column, *nested, + return safeDeserialize(column, *nested_serialization, [] { return false; }, - [&nested, &istr, &settings] (IColumn & nested_column) + [&nested_serialization, &istr, &settings] (IColumn & nested_column) { if constexpr (escaped) - nested->deserializeTextEscaped(nested_column, istr, settings); + nested_serialization->deserializeTextEscaped(nested_column, istr, settings); else - nested->deserializeTextRaw(nested_column, istr, settings); + nested_serialization->deserializeTextRaw(nested_column, istr, settings); }); } @@ -315,14 +315,14 @@ ReturnType SerializationNullable::deserializeTextEscapedAndRawImpl(IColumn & col istr.position() = pos; return false; }; - auto deserialize_nested = [&nested, &settings, &istr] (IColumn & nested_column) + auto deserialize_nested = [&nested_serialization, &settings, &istr] (IColumn & nested_column) { if constexpr (escaped) - nested->deserializeTextEscaped(nested_column, istr, settings); + nested_serialization->deserializeTextEscaped(nested_column, istr, settings); else - nested->deserializeTextRaw(nested_column, istr, settings); + nested_serialization->deserializeTextRaw(nested_column, istr, settings); }; - return safeDeserialize(column, *nested, check_for_null, deserialize_nested); + return safeDeserialize(column, *nested_serialization, check_for_null, deserialize_nested); } /// We don't have enough data in buffer to check if it's a null. @@ -340,13 +340,13 @@ ReturnType SerializationNullable::deserializeTextEscapedAndRawImpl(IColumn & col return false; }; - auto deserialize_nested = [&nested, &settings, &buf, &null_representation, &istr] (IColumn & nested_column) + auto deserialize_nested = [&nested_serialization, &settings, &buf, &null_representation, &istr] (IColumn & nested_column) { auto * pos = buf.position(); if constexpr (escaped) - nested->deserializeTextEscaped(nested_column, buf, settings); + nested_serialization->deserializeTextEscaped(nested_column, buf, settings); else - nested->deserializeTextRaw(nested_column, buf, settings); + nested_serialization->deserializeTextRaw(nested_column, buf, settings); /// Check that we don't have any unread data in PeekableReadBuffer own memory. if (likely(!buf.hasUnreadData())) return; @@ -361,9 +361,9 @@ ReturnType SerializationNullable::deserializeTextEscapedAndRawImpl(IColumn & col WriteBufferFromOwnString parsed_value; if constexpr (escaped) - nested->serializeTextEscaped(nested_column, nested_column.size() - 1, parsed_value, settings); + nested_serialization->serializeTextEscaped(nested_column, nested_column.size() - 1, parsed_value, settings); else - nested->serializeTextRaw(nested_column, nested_column.size() - 1, parsed_value, settings); + nested_serialization->serializeTextRaw(nested_column, nested_column.size() - 1, parsed_value, settings); throw DB::ParsingException("Error while parsing \"" + std::string(pos, buf.buffer().end()) + std::string(istr.position(), std::min(size_t(10), istr.available())) + "\" as Nullable" + " at position " + std::to_string(istr.count()) + ": got \"" + std::string(pos, buf.position() - pos) + "\", which was deserialized as \"" @@ -371,7 +371,7 @@ ReturnType SerializationNullable::deserializeTextEscapedAndRawImpl(IColumn & col ErrorCodes::CANNOT_READ_ALL_DATA); }; - return safeDeserialize(column, *nested, check_for_null, deserialize_nested); + return safeDeserialize(column, *nested_serialization, check_for_null, deserialize_nested); } void SerializationNullable::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -456,15 +456,15 @@ void SerializationNullable::deserializeTextCSV(IColumn & column, ReadBuffer & is template ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, - const SerializationPtr & nested) + const SerializationPtr & nested_serialization) { const String & null_representation = settings.csv.null_representation; if (istr.eof() || (!null_representation.empty() && *istr.position() != null_representation[0])) { /// This is not null, surely. - return safeDeserialize(column, *nested, + return safeDeserialize(column, *nested_serialization, [] { return false; }, - [&nested, &istr, &settings] (IColumn & nested_column) { nested->deserializeTextCSV(nested_column, istr, settings); }); + [&nested_serialization, &istr, &settings] (IColumn & nested_column) { nested->deserializeTextCSV(nested_column, istr, settings); }); } /// Check if we have enough data in buffer to check if it's a null. @@ -478,11 +478,11 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB istr.position() = pos; return false; }; - auto deserialize_nested = [&nested, &settings, &istr] (IColumn & nested_column) + auto deserialize_nested = [&nested_serialization, &settings, &istr] (IColumn & nested_column) { - nested->deserializeTextCSV(nested_column, istr, settings); + nested_serialization->deserializeTextCSV(nested_column, istr, settings); }; - return safeDeserialize(column, *nested, check_for_null, deserialize_nested); + return safeDeserialize(column, *nested_serialization, check_for_null, deserialize_nested); } /// We don't have enough data in buffer to check if it's a null. @@ -500,10 +500,10 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB return false; }; - auto deserialize_nested = [&nested, &settings, &buf, &null_representation, &istr] (IColumn & nested_column) + auto deserialize_nested = [&nested_serialization, &settings, &buf, &null_representation, &istr] (IColumn & nested_column) { auto * pos = buf.position(); - nested->deserializeTextCSV(nested_column, buf, settings); + nested_serialization->deserializeTextCSV(nested_column, buf, settings); /// Check that we don't have any unread data in PeekableReadBuffer own memory. if (likely(!buf.hasUnreadData())) return; @@ -518,7 +518,7 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB "for large input.", ErrorCodes::CANNOT_READ_ALL_DATA); WriteBufferFromOwnString parsed_value; - nested->serializeTextCSV(nested_column, nested_column.size() - 1, parsed_value, settings); + nested_serialization->serializeTextCSV(nested_column, nested_column.size() - 1, parsed_value, settings); throw DB::ParsingException("Error while parsing \"" + std::string(pos, buf.buffer().end()) + std::string(istr.position(), std::min(size_t(10), istr.available())) + "\" as Nullable" + " at position " + std::to_string(istr.count()) + ": got \"" + std::string(pos, buf.position() - pos) + "\", which was deserialized as \"" @@ -526,7 +526,7 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB ErrorCodes::CANNOT_READ_ALL_DATA); }; - return safeDeserialize(column, *nested, check_for_null, deserialize_nested); + return safeDeserialize(column, *nested_serialization, check_for_null, deserialize_nested); } void SerializationNullable::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const From 1d2f197e70ea1cb1249926b15ac0fcb0ce478cc3 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 2 Nov 2021 16:09:20 +0300 Subject: [PATCH 380/396] Fix build --- src/DataTypes/Serializations/SerializationNullable.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index 9ae78403d11..5e2b31ebb9d 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -464,7 +464,7 @@ ReturnType SerializationNullable::deserializeTextCSVImpl(IColumn & column, ReadB /// This is not null, surely. return safeDeserialize(column, *nested_serialization, [] { return false; }, - [&nested_serialization, &istr, &settings] (IColumn & nested_column) { nested->deserializeTextCSV(nested_column, istr, settings); }); + [&nested_serialization, &istr, &settings] (IColumn & nested_column) { nested_serialization->deserializeTextCSV(nested_column, istr, settings); }); } /// Check if we have enough data in buffer to check if it's a null. From 62c5951dd7888e029e442d756200867d04745de3 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 2 Nov 2021 16:26:14 +0300 Subject: [PATCH 381/396] Fix segfault in formatRow function --- src/Functions/formatRow.cpp | 7 +++++++ tests/queries/0_stateless/02113_format_row_bug.reference | 0 tests/queries/0_stateless/02113_format_row_bug.sql | 5 +++++ 3 files changed, 12 insertions(+) create mode 100644 tests/queries/0_stateless/02113_format_row_bug.reference create mode 100644 tests/queries/0_stateless/02113_format_row_bug.sql diff --git a/src/Functions/formatRow.cpp b/src/Functions/formatRow.cpp index ee9696cf34f..3f9d3e782d7 100644 --- a/src/Functions/formatRow.cpp +++ b/src/Functions/formatRow.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include @@ -18,6 +19,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int UNKNOWN_FORMAT; + extern const int BAD_ARGUMENTS; } namespace @@ -70,6 +72,11 @@ public: writeChar('\0', buffer); offsets[row] = buffer.count(); }); + + /// This function make sense only for row output formats. + if (!dynamic_cast(out.get())) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot turn rows into a {} format strings. {} function supports only row output formats", format_name, getName()); + out->write(arg_columns); return col_str; } diff --git a/tests/queries/0_stateless/02113_format_row_bug.reference b/tests/queries/0_stateless/02113_format_row_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02113_format_row_bug.sql b/tests/queries/0_stateless/02113_format_row_bug.sql new file mode 100644 index 00000000000..971a2bee2b0 --- /dev/null +++ b/tests/queries/0_stateless/02113_format_row_bug.sql @@ -0,0 +1,5 @@ +select formatRow('ORC', number, toDate(number)) from numbers(5); -- { serverError 36 } +select formatRow('Parquet', number, toDate(number)) from numbers(5); -- { serverError 36 } +select formatRow('Arrow', number, toDate(number)) from numbers(5); -- { serverError 36 } +select formatRow('Native', number, toDate(number)) from numbers(5); -- { serverError 36 } + From bc2c5c6033b3125560e504eff0a06982fe7ac493 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 2 Nov 2021 16:38:55 +0300 Subject: [PATCH 382/396] Add fuzzer check to actions --- .github/workflows/main.yml | 29 +++++ tests/ci/ast_fuzzer_check.py | 216 +++++++++++++++++++++++++++++++++++ 2 files changed, 245 insertions(+) create mode 100644 tests/ci/ast_fuzzer_check.py diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 0a4beb50708..d085a307d81 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -236,6 +236,35 @@ jobs: docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH + ASTFuzzerTestDebug: + 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: Fuzzer + env: + TEMP_PATH: ${{runner.temp}}/ast_fuzzer_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'AST fuzzer (debug, actions)' + REPO_COPY: ${{runner.temp}}/ast_fuzzer_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 ast_fuzzer_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') }} diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py new file mode 100644 index 00000000000..322e2b519dd --- /dev/null +++ b/tests/ci/ast_fuzzer_check.py @@ -0,0 +1,216 @@ +#!/usr/bin/env python3 + +import logging +import subprocess +import os +import json +import time +import sys + +from github import Github +import requests + +from s3_helper import S3Helper +from get_robot_token import get_best_robot_token +from pr_info import PRInfo + + +DOWNLOAD_RETRIES_COUNT = 5 +IMAGE_NAME = 'clickhouse/fuzzer' + +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 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 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 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(pr_number, sha, download_url, workspace_path, image): + return f'docker run --network=host --volume={workspace_path}:/workspace ' \ + '--cap-add syslog --cap-add sys_admin ' \ + f'-e PR_TO_TEST={pr_number} -e SHA_TO_TEST={sha} -e BINARY_URL_TO_DOWNLOAD={download_url} '\ + f'{image}' + +def get_commit(gh, commit_sha): + repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) + commit = repo.get_commit(commit_sha) + return commit + +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()) + + images_path = os.path.join(temp_path, 'changed_images.json') + + docker_image = IMAGE_NAME + if 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] + + 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") + + for url in urls: + if url.endswith('/clickhouse'): + build_url = url + break + else: + raise Exception("Cannot binary clickhouse among build results") + + logging.info("Got build url %s", build_url) + + workspace_path = os.path.join(temp_path, 'workspace') + if not os.path.exists(workspace_path): + os.makedirs(workspace_path) + + run_command = get_run_command(pr_info.number, pr_info.sha, build_url, workspace_path, docker_image) + logging.info("Going to run %s", run_command) + + run_log_path = os.path.join(temp_path, "runlog.log") + 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) + + check_name_lower = check_name.lower().replace('(', '').replace(')', '').replace(' ', '') + s3_prefix = f'{pr_info.number}/{pr_info.sha}/fuzzer_{check_name_lower}/' + paths = { + 'main.log': os.path.join(workspace_path, 'workspace/main.log'), + 'server.log': os.path.join(workspace_path, 'workspace/server.log'), + 'fuzzer.log': os.path.join(workspace_path, 'workspace/fuzzer.log'), + 'report.html': os.path.join(workspace_path, 'workspace/report.html'), + } + + s3_helper = S3Helper('https://s3.amazonaws.com') + for f in paths: + try: + paths[f] = s3_helper.upload_test_report_to_s3(paths[f], s3_prefix + '/' + os.path.basename(f)) + except: + paths[f] = '' + + report_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + if paths['main.log']: + report_url = paths['main.log'] + if paths['server.log']: + report_url = paths['server.log'] + if paths['fuzzer.log']: + report_url = paths['fuzzer.log'] + if paths['report.html']: + report_url = paths['report.html'] + + # Try to get status message saved by the fuzzer + try: + with open(os.path.join(workspace_path, 'workspace/status.txt'), 'r', encoding='utf-8') as status_f: + status = status_f.readline().rstrip('\n') + + with open(os.path.join(workspace_path, 'workspace/description.txt'), 'r', encoding='utf-8') as desc_f: + description = desc_f.readline().rstrip('\n')[:140] + except: + status = 'failure' + description = 'Task failed: $?=' + str(retcode) + + logging.info("Result: '%s', '%s', '%s'", status, description, report_url) + print(f"::notice ::Report url: {report_url}") + commit = get_commit(gh, pr_info.sha) + commit.create_status(context=check_name, description=description, state=status, target_url=report_url) From 01684ad05a65671382409dec7838ac1383240c1f Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 2 Nov 2021 16:53:21 +0300 Subject: [PATCH 383/396] Fix --- .../JSONPath/Parsers/ParserJSONPathMemberAccess.cpp | 9 ++------- .../0_stateless/01889_sql_json_functions.reference | 1 + tests/queries/0_stateless/01889_sql_json_functions.sql | 1 + 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp b/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp index c7f047eb8fb..0ea3bfbd013 100644 --- a/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp +++ b/src/Functions/JSONPath/Parsers/ParserJSONPathMemberAccess.cpp @@ -17,22 +17,17 @@ namespace DB bool ParserJSONPathMemberAccess::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (pos->type != TokenType::Dot) - { return false; - } + ++pos; - if (pos->type != TokenType::BareWord) - { + if (pos->type != TokenType::BareWord && pos->type !=TokenType::QuotedIdentifier) return false; - } ParserIdentifier name_p; ASTPtr member_name; if (!name_p.parse(pos, member_name, expected)) - { return false; - } auto member_access = std::make_shared(); node = member_access; diff --git a/tests/queries/0_stateless/01889_sql_json_functions.reference b/tests/queries/0_stateless/01889_sql_json_functions.reference index 593f2fb2d20..fd8989611a8 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.reference +++ b/tests/queries/0_stateless/01889_sql_json_functions.reference @@ -9,6 +9,7 @@ null +"bar" --JSON_QUERY-- [{"hello":1}] [1] diff --git a/tests/queries/0_stateless/01889_sql_json_functions.sql b/tests/queries/0_stateless/01889_sql_json_functions.sql index 087f029e635..f68fe63ecab 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.sql +++ b/tests/queries/0_stateless/01889_sql_json_functions.sql @@ -11,6 +11,7 @@ SELECT JSON_VALUE('{"hello":["world","world2"]}', '$.hello'); SELECT JSON_VALUE('{"hello":{"world":"!"}}', '$.hello'); SELECT JSON_VALUE('{hello:world}', '$.hello'); -- invalid json => default value (empty string) SELECT JSON_VALUE('', '$.hello'); +SELECT JSON_VALUE('{"foo foo":"bar"}', '$."foo foo"'); SELECT '--JSON_QUERY--'; SELECT JSON_QUERY('{"hello":1}', '$'); From 1cb708fd1b4287657cf4babfd3f5f4de5902b65a Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 2 Nov 2021 17:13:58 +0300 Subject: [PATCH 384/396] Update 02113_format_row_bug.sql --- tests/queries/0_stateless/02113_format_row_bug.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02113_format_row_bug.sql b/tests/queries/0_stateless/02113_format_row_bug.sql index 971a2bee2b0..c2144ca1537 100644 --- a/tests/queries/0_stateless/02113_format_row_bug.sql +++ b/tests/queries/0_stateless/02113_format_row_bug.sql @@ -1,5 +1,6 @@ +-- Tags: no-fasttest + select formatRow('ORC', number, toDate(number)) from numbers(5); -- { serverError 36 } select formatRow('Parquet', number, toDate(number)) from numbers(5); -- { serverError 36 } select formatRow('Arrow', number, toDate(number)) from numbers(5); -- { serverError 36 } select formatRow('Native', number, toDate(number)) from numbers(5); -- { serverError 36 } - From cd77f268260cd640f49e9d36acdc971e4a61feac Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 2 Nov 2021 18:37:55 +0300 Subject: [PATCH 385/396] Fix fuzzer check --- .github/workflows/main.yml | 2 +- tests/ci/ast_fuzzer_check.py | 23 +++++++++++++---------- 2 files changed, 14 insertions(+), 11 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index d085a307d81..520ba89209c 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -289,7 +289,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH FinishCheck: - needs: [StyleCheck, DockerHubPush, CheckLabels, BuilderReport, FastTest, FunctionalStatelessTestDebug, FunctionalStatefulTestDebug, DocsCheck, StressTestDebug] + needs: [StyleCheck, DockerHubPush, CheckLabels, BuilderReport, FastTest, FunctionalStatelessTestDebug, FunctionalStatefulTestDebug, DocsCheck, StressTestDebug, ASTFuzzerTestDebug] runs-on: [self-hosted, style-checker] steps: - name: Check out repository code diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index 322e2b519dd..84df764f950 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -91,7 +91,7 @@ def build_config_to_string(build_config): def get_run_command(pr_number, sha, download_url, workspace_path, image): return f'docker run --network=host --volume={workspace_path}:/workspace ' \ '--cap-add syslog --cap-add sys_admin ' \ - f'-e PR_TO_TEST={pr_number} -e SHA_TO_TEST={sha} -e BINARY_URL_TO_DOWNLOAD={download_url} '\ + f'-e PR_TO_TEST={pr_number} -e SHA_TO_TEST={sha} -e BINARY_URL_TO_DOWNLOAD="{download_url}" '\ f'{image}' def get_commit(gh, commit_sha): @@ -176,6 +176,7 @@ if __name__ == "__main__": check_name_lower = check_name.lower().replace('(', '').replace(')', '').replace(' ', '') s3_prefix = f'{pr_info.number}/{pr_info.sha}/fuzzer_{check_name_lower}/' paths = { + 'runlog.log': run_log_path, 'main.log': os.path.join(workspace_path, 'workspace/main.log'), 'server.log': os.path.join(workspace_path, 'workspace/server.log'), 'fuzzer.log': os.path.join(workspace_path, 'workspace/fuzzer.log'), @@ -189,15 +190,17 @@ if __name__ == "__main__": except: paths[f] = '' - report_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" - if paths['main.log']: - report_url = paths['main.log'] - if paths['server.log']: - report_url = paths['server.log'] - if paths['fuzzer.log']: - report_url = paths['fuzzer.log'] - if paths['report.html']: - report_url = paths['report.html'] + report_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + if paths['runlog.log']: + report_url = paths['runlog.log'] + if paths['main.log']: + report_url = paths['main.log'] + if paths['server.log']: + report_url = paths['server.log'] + if paths['fuzzer.log']: + report_url = paths['fuzzer.log'] + if paths['report.html']: + report_url = paths['report.html'] # Try to get status message saved by the fuzzer try: From 94f0214a8fa5b2bc9993ea842f1d4ffcfe7d8c06 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 2 Nov 2021 22:29:58 +0300 Subject: [PATCH 386/396] Remove paths --- tests/ci/ast_fuzzer_check.py | 17 +++++++++-------- tests/ci/metrics_lambda/app.py | 25 ++++++++++++++++++++++--- 2 files changed, 31 insertions(+), 11 deletions(-) diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index 84df764f950..0ed93d792fd 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -177,17 +177,18 @@ if __name__ == "__main__": s3_prefix = f'{pr_info.number}/{pr_info.sha}/fuzzer_{check_name_lower}/' paths = { 'runlog.log': run_log_path, - 'main.log': os.path.join(workspace_path, 'workspace/main.log'), - 'server.log': os.path.join(workspace_path, 'workspace/server.log'), - 'fuzzer.log': os.path.join(workspace_path, 'workspace/fuzzer.log'), - 'report.html': os.path.join(workspace_path, 'workspace/report.html'), + 'main.log': os.path.join(workspace_path, 'main.log'), + 'server.log': os.path.join(workspace_path, 'server.log'), + 'fuzzer.log': os.path.join(workspace_path, 'fuzzer.log'), + 'report.html': os.path.join(workspace_path, 'report.html'), } s3_helper = S3Helper('https://s3.amazonaws.com') for f in paths: try: - paths[f] = s3_helper.upload_test_report_to_s3(paths[f], s3_prefix + '/' + os.path.basename(f)) - except: + paths[f] = s3_helper.upload_test_report_to_s3(paths[f], s3_prefix + '/' + f) + except Exception as ex: + logging.info("Exception uploading file %s text %s", f, ex) paths[f] = '' report_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" @@ -204,10 +205,10 @@ if __name__ == "__main__": # Try to get status message saved by the fuzzer try: - with open(os.path.join(workspace_path, 'workspace/status.txt'), 'r', encoding='utf-8') as status_f: + with open(os.path.join(workspace_path, 'status.txt'), 'r', encoding='utf-8') as status_f: status = status_f.readline().rstrip('\n') - with open(os.path.join(workspace_path, 'workspace/description.txt'), 'r', encoding='utf-8') as desc_f: + with open(os.path.join(workspace_path, 'description.txt'), 'r', encoding='utf-8') as desc_f: description = desc_f.readline().rstrip('\n')[:140] except: status = 'failure' diff --git a/tests/ci/metrics_lambda/app.py b/tests/ci/metrics_lambda/app.py index d88e3e45b07..231a7203fbb 100644 --- a/tests/ci/metrics_lambda/app.py +++ b/tests/ci/metrics_lambda/app.py @@ -129,7 +129,18 @@ def push_metrics_to_cloudwatch(listed_runners, namespace): client.put_metric_data(Namespace=namespace, MetricData=metrics_data) -def main(github_secret_key, github_app_id, push_to_cloudwatch): +def delete_runner(access_token, runner): + headers = { + "Authorization": f"token {access_token}", + "Accept": "application/vnd.github.v3+json", + } + + response = requests.delete(f"https://api.github.com/orgs/ClickHouse/actions/runners/{runner.id}", headers=headers) + response.raise_for_status() + print(f"Response code deleting {runner.name} is {response.status_code}") + return response.status_code == 204 + +def main(github_secret_key, github_app_id, push_to_cloudwatch, delete_offline_runners): payload = { "iat": int(time.time()) - 60, "exp": int(time.time()) + (10 * 60), @@ -145,9 +156,16 @@ def main(github_secret_key, github_app_id, push_to_cloudwatch): if push_to_cloudwatch: push_metrics_to_cloudwatch(group_runners, 'RunnersMetrics/' + group) else: - print(group) + print(group, f"({len(group_runners)})") for runner in group_runners: print('\t', runner) + if delete_offline_runners: + print("Going to delete offline runners") + for runner in runners: + if runner.offline: + print("Deleting runner", runner) + delete_runner(access_token, runner) + if __name__ == "__main__": @@ -156,6 +174,7 @@ if __name__ == "__main__": parser.add_argument('-k', '--private-key', help='Private key') parser.add_argument('-a', '--app-id', type=int, help='GitHub application ID', required=True) parser.add_argument('--push-to-cloudwatch', action='store_true', help='Store received token in parameter store') + parser.add_argument('--delete-offline', action='store_true', help='Remove offline runners') args = parser.parse_args() @@ -171,4 +190,4 @@ if __name__ == "__main__": with open(args.private_key_path, 'r') as key_file: private_key = key_file.read() - main(private_key, args.app_id, args.push_to_cloudwatch) + main(private_key, args.app_id, args.push_to_cloudwatch, args.delete_offline) From cb05c0504e3db4500c723022ee09c7aa834e0a27 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 2 Nov 2021 16:05:33 +0300 Subject: [PATCH 387/396] Move more implementations from headers to cpp. --- src/Access/AccessRights.h | 1 + src/Access/Common/AccessFlags.cpp | 378 +++++++++++++++++ src/Access/Common/AccessFlags.h | 382 +----------------- src/Access/Common/AccessRightsElement.cpp | 88 +++- src/Access/Common/AccessRightsElement.h | 81 +--- src/Access/Common/AccessType.cpp | 56 +++ src/Access/Common/AccessType.h | 50 +-- src/Access/Common/AllowedClientHosts.cpp | 264 +++++++++++- src/Access/Common/AllowedClientHosts.h | 263 ------------ src/Access/ContextAccess.h | 1 + .../Access/InterpreterGrantQuery.cpp | 1 + 11 files changed, 808 insertions(+), 757 deletions(-) create mode 100644 src/Access/Common/AccessFlags.cpp create mode 100644 src/Access/Common/AccessType.cpp diff --git a/src/Access/AccessRights.h b/src/Access/AccessRights.h index 32a4462d212..c3f75b8c303 100644 --- a/src/Access/AccessRights.h +++ b/src/Access/AccessRights.h @@ -2,6 +2,7 @@ #include #include +#include #include #include diff --git a/src/Access/Common/AccessFlags.cpp b/src/Access/Common/AccessFlags.cpp new file mode 100644 index 00000000000..e7dddbdcba2 --- /dev/null +++ b/src/Access/Common/AccessFlags.cpp @@ -0,0 +1,378 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int UNKNOWN_ACCESS_TYPE; + extern const int LOGICAL_ERROR; +} + +namespace +{ + using Flags = std::bitset; + + class Helper + { + public: + static const Helper & instance() + { + static const Helper res; + return res; + } + + Flags accessTypeToFlags(AccessType type) const + { + return access_type_to_flags_mapping[static_cast(type)]; + } + + Flags keywordToFlags(const std::string_view & keyword) const + { + auto it = keyword_to_flags_map.find(keyword); + if (it == keyword_to_flags_map.end()) + { + String uppercased_keyword{keyword}; + boost::to_upper(uppercased_keyword); + it = keyword_to_flags_map.find(uppercased_keyword); + if (it == keyword_to_flags_map.end()) + throw Exception("Unknown access type: " + String(keyword), ErrorCodes::UNKNOWN_ACCESS_TYPE); + } + return it->second; + } + + Flags keywordsToFlags(const std::vector & keywords) const + { + Flags res; + for (const auto & keyword : keywords) + res |= keywordToFlags(keyword); + return res; + } + + Flags keywordsToFlags(const Strings & keywords) const + { + Flags res; + for (const auto & keyword : keywords) + res |= keywordToFlags(keyword); + return res; + } + + std::vector flagsToAccessTypes(const Flags & flags_) const + { + std::vector access_types; + flagsToAccessTypesRec(flags_, access_types, *all_node); + return access_types; + } + + std::vector flagsToKeywords(const Flags & flags_) const + { + std::vector keywords; + flagsToKeywordsRec(flags_, keywords, *all_node); + return keywords; + } + + String flagsToString(const Flags & flags_) const + { + auto keywords = flagsToKeywords(flags_); + if (keywords.empty()) + return "USAGE"; + String str; + for (const auto & keyword : keywords) + { + if (!str.empty()) + str += ", "; + str += keyword; + } + return str; + } + + const Flags & getAllFlags() const { return all_flags; } + const Flags & getGlobalFlags() const { return all_flags_for_target[GLOBAL]; } + const Flags & getDatabaseFlags() const { return all_flags_for_target[DATABASE]; } + const Flags & getTableFlags() const { return all_flags_for_target[TABLE]; } + const Flags & getColumnFlags() const { return all_flags_for_target[COLUMN]; } + const Flags & getDictionaryFlags() const { return all_flags_for_target[DICTIONARY]; } + const Flags & getAllFlagsGrantableOnGlobalLevel() const { return getAllFlags(); } + const Flags & getAllFlagsGrantableOnDatabaseLevel() const { return all_flags_grantable_on_database_level; } + const Flags & getAllFlagsGrantableOnTableLevel() const { return all_flags_grantable_on_table_level; } + const Flags & getAllFlagsGrantableOnColumnLevel() const { return getColumnFlags(); } + + private: + enum NodeType + { + UNKNOWN = -2, + GROUP = -1, + GLOBAL, + DATABASE, + TABLE, + VIEW = TABLE, + COLUMN, + DICTIONARY, + }; + + struct Node; + using NodePtr = std::unique_ptr; + + struct Node + { + const String keyword; + NodeType node_type = UNKNOWN; + AccessType access_type = AccessType::NONE; + Strings aliases; + Flags flags; + std::vector children; + + explicit Node(String keyword_) : keyword(std::move(keyword_)) {} + Node(String keyword_, NodeType node_type_) : keyword(std::move(keyword_)), node_type(node_type_) {} + + void setFlag(size_t flag) { flags.set(flag); } + + void addChild(NodePtr child) + { + flags |= child->flags; + children.push_back(std::move(child)); + } + }; + + static String replaceUnderscoreWithSpace(const std::string_view & str) + { + String res{str}; + boost::replace_all(res, "_", " "); + return res; + } + + static Strings splitAliases(const std::string_view & str) + { + Strings aliases; + boost::split(aliases, str, boost::is_any_of(",")); + for (auto & alias : aliases) + boost::trim(alias); + return aliases; + } + + static void makeNode( + AccessType access_type, + const std::string_view & name, + const std::string_view & aliases, + NodeType node_type, + const std::string_view & parent_group_name, + std::unordered_map & nodes, + std::unordered_map & owned_nodes, + size_t & next_flag) + { + NodePtr node; + auto keyword = replaceUnderscoreWithSpace(name); + auto it = owned_nodes.find(keyword); + if (it != owned_nodes.end()) + { + node = std::move(it->second); + owned_nodes.erase(it); + } + else + { + if (nodes.count(keyword)) + throw Exception(keyword + " declared twice", ErrorCodes::LOGICAL_ERROR); + node = std::make_unique(keyword, node_type); + nodes[node->keyword] = node.get(); + } + + node->access_type = access_type; + node->node_type = node_type; + node->aliases = splitAliases(aliases); + if (node_type != GROUP) + node->setFlag(next_flag++); + + bool has_parent_group = (parent_group_name != std::string_view{"NONE"}); + if (!has_parent_group) + { + std::string_view keyword_as_string_view = node->keyword; + owned_nodes[keyword_as_string_view] = std::move(node); + return; + } + + auto parent_keyword = replaceUnderscoreWithSpace(parent_group_name); + auto it_parent = nodes.find(parent_keyword); + if (it_parent == nodes.end()) + { + auto parent_node = std::make_unique(parent_keyword); + it_parent = nodes.emplace(parent_node->keyword, parent_node.get()).first; + assert(!owned_nodes.count(parent_node->keyword)); + std::string_view parent_keyword_as_string_view = parent_node->keyword; + owned_nodes[parent_keyword_as_string_view] = std::move(parent_node); + } + it_parent->second->addChild(std::move(node)); + } + + void makeNodes() + { + std::unordered_map owned_nodes; + std::unordered_map nodes; + size_t next_flag = 0; + +# define MAKE_ACCESS_FLAGS_NODE(name, aliases, node_type, parent_group_name) \ + makeNode(AccessType::name, #name, aliases, node_type, #parent_group_name, nodes, owned_nodes, next_flag); + + APPLY_FOR_ACCESS_TYPES(MAKE_ACCESS_FLAGS_NODE) + +# undef MAKE_ACCESS_FLAGS_NODE + + if (!owned_nodes.count("NONE")) + throw Exception("'NONE' not declared", ErrorCodes::LOGICAL_ERROR); + if (!owned_nodes.count("ALL")) + throw Exception("'ALL' not declared", ErrorCodes::LOGICAL_ERROR); + + all_node = std::move(owned_nodes["ALL"]); + none_node = std::move(owned_nodes["NONE"]); + owned_nodes.erase("ALL"); + owned_nodes.erase("NONE"); + + if (!owned_nodes.empty()) + { + const auto & unused_node = *(owned_nodes.begin()->second); + if (unused_node.node_type == UNKNOWN) + throw Exception("Parent group '" + unused_node.keyword + "' not found", ErrorCodes::LOGICAL_ERROR); + else + throw Exception("Access type '" + unused_node.keyword + "' should have parent group", ErrorCodes::LOGICAL_ERROR); + } + } + + void makeKeywordToFlagsMap(Node * start_node = nullptr) + { + if (!start_node) + { + makeKeywordToFlagsMap(none_node.get()); + start_node = all_node.get(); + } + + start_node->aliases.emplace_back(start_node->keyword); + for (auto & alias : start_node->aliases) + { + boost::to_upper(alias); + keyword_to_flags_map[alias] = start_node->flags; + } + + for (auto & child : start_node->children) + makeKeywordToFlagsMap(child.get()); + } + + void makeAccessTypeToFlagsMapping(Node * start_node = nullptr) + { + if (!start_node) + { + makeAccessTypeToFlagsMapping(none_node.get()); + start_node = all_node.get(); + } + + size_t index = static_cast(start_node->access_type); + access_type_to_flags_mapping.resize(std::max(index + 1, access_type_to_flags_mapping.size())); + access_type_to_flags_mapping[index] = start_node->flags; + + for (auto & child : start_node->children) + makeAccessTypeToFlagsMapping(child.get()); + } + + void collectAllFlags(const Node * start_node = nullptr) + { + if (!start_node) + { + start_node = all_node.get(); + all_flags = start_node->flags; + } + if (start_node->node_type != GROUP) + { + assert(static_cast(start_node->node_type) < std::size(all_flags_for_target)); + all_flags_for_target[start_node->node_type] |= start_node->flags; + } + for (const auto & child : start_node->children) + collectAllFlags(child.get()); + + all_flags_grantable_on_table_level = all_flags_for_target[TABLE] | all_flags_for_target[DICTIONARY] | all_flags_for_target[COLUMN]; + all_flags_grantable_on_database_level = all_flags_for_target[DATABASE] | all_flags_grantable_on_table_level; + } + + Helper() + { + makeNodes(); + makeKeywordToFlagsMap(); + makeAccessTypeToFlagsMapping(); + collectAllFlags(); + } + + static void flagsToAccessTypesRec(const Flags & flags_, std::vector & access_types, const Node & start_node) + { + Flags matching_flags = (flags_ & start_node.flags); + if (matching_flags.any()) + { + if (matching_flags == start_node.flags) + { + access_types.push_back(start_node.access_type); + } + else + { + for (const auto & child : start_node.children) + flagsToAccessTypesRec(flags_, access_types, *child); + } + } + } + + static void flagsToKeywordsRec(const Flags & flags_, std::vector & keywords, const Node & start_node) + { + Flags matching_flags = (flags_ & start_node.flags); + if (matching_flags.any()) + { + if (matching_flags == start_node.flags) + { + keywords.push_back(start_node.keyword); + } + else + { + for (const auto & child : start_node.children) + flagsToKeywordsRec(flags_, keywords, *child); + } + } + } + + NodePtr all_node; + NodePtr none_node; + std::unordered_map keyword_to_flags_map; + std::vector access_type_to_flags_mapping; + Flags all_flags; + Flags all_flags_for_target[static_cast(DICTIONARY) + 1]; + Flags all_flags_grantable_on_database_level; + Flags all_flags_grantable_on_table_level; + }; +} + + +AccessFlags::AccessFlags(AccessType type) : flags(Helper::instance().accessTypeToFlags(type)) {} +AccessFlags::AccessFlags(const std::string_view & keyword) : flags(Helper::instance().keywordToFlags(keyword)) {} +AccessFlags::AccessFlags(const std::vector & keywords) : flags(Helper::instance().keywordsToFlags(keywords)) {} +AccessFlags::AccessFlags(const Strings & keywords) : flags(Helper::instance().keywordsToFlags(keywords)) {} +String AccessFlags::toString() const { return Helper::instance().flagsToString(flags); } +std::vector AccessFlags::toAccessTypes() const { return Helper::instance().flagsToAccessTypes(flags); } +std::vector AccessFlags::toKeywords() const { return Helper::instance().flagsToKeywords(flags); } +AccessFlags AccessFlags::allFlags() { return Helper::instance().getAllFlags(); } +AccessFlags AccessFlags::allGlobalFlags() { return Helper::instance().getGlobalFlags(); } +AccessFlags AccessFlags::allDatabaseFlags() { return Helper::instance().getDatabaseFlags(); } +AccessFlags AccessFlags::allTableFlags() { return Helper::instance().getTableFlags(); } +AccessFlags AccessFlags::allColumnFlags() { return Helper::instance().getColumnFlags(); } +AccessFlags AccessFlags::allDictionaryFlags() { return Helper::instance().getDictionaryFlags(); } +AccessFlags AccessFlags::allFlagsGrantableOnGlobalLevel() { return Helper::instance().getAllFlagsGrantableOnGlobalLevel(); } +AccessFlags AccessFlags::allFlagsGrantableOnDatabaseLevel() { return Helper::instance().getAllFlagsGrantableOnDatabaseLevel(); } +AccessFlags AccessFlags::allFlagsGrantableOnTableLevel() { return Helper::instance().getAllFlagsGrantableOnTableLevel(); } +AccessFlags AccessFlags::allFlagsGrantableOnColumnLevel() { return Helper::instance().getAllFlagsGrantableOnColumnLevel(); } + +AccessFlags operator |(AccessType left, AccessType right) { return AccessFlags(left) | right; } +AccessFlags operator &(AccessType left, AccessType right) { return AccessFlags(left) & right; } +AccessFlags operator -(AccessType left, AccessType right) { return AccessFlags(left) - right; } +AccessFlags operator ~(AccessType x) { return ~AccessFlags(x); } + +} diff --git a/src/Access/Common/AccessFlags.h b/src/Access/Common/AccessFlags.h index 87dc17522ab..5a5452c50ea 100644 --- a/src/Access/Common/AccessFlags.h +++ b/src/Access/Common/AccessFlags.h @@ -1,21 +1,14 @@ #pragma once #include -#include -#include -#include -#include -#include #include -#include +#include +#include namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} +using Strings = std::vector; /// Represents a combination of access types which can be granted globally, on databases, tables, columns, etc. /// For example "SELECT, CREATE USER" is an access type. @@ -111,374 +104,17 @@ public: /// The same as allColumnFlags(). static AccessFlags allFlagsGrantableOnColumnLevel(); + static constexpr size_t SIZE = 128; private: - static constexpr size_t NUM_FLAGS = 128; - using Flags = std::bitset; + using Flags = std::bitset; Flags flags; AccessFlags(const Flags & flags_) : flags(flags_) {} - - template - class Impl; }; - -namespace ErrorCodes -{ - extern const int UNKNOWN_ACCESS_TYPE; -} - -template -class AccessFlags::Impl -{ -public: - static const Impl & instance() - { - static const Impl res; - return res; - } - - Flags accessTypeToFlags(AccessType type) const - { - return access_type_to_flags_mapping[static_cast(type)]; - } - - Flags keywordToFlags(const std::string_view & keyword) const - { - auto it = keyword_to_flags_map.find(keyword); - if (it == keyword_to_flags_map.end()) - { - String uppercased_keyword{keyword}; - boost::to_upper(uppercased_keyword); - it = keyword_to_flags_map.find(uppercased_keyword); - if (it == keyword_to_flags_map.end()) - throw Exception("Unknown access type: " + String(keyword), ErrorCodes::UNKNOWN_ACCESS_TYPE); - } - return it->second; - } - - Flags keywordsToFlags(const std::vector & keywords) const - { - Flags res; - for (const auto & keyword : keywords) - res |= keywordToFlags(keyword); - return res; - } - - Flags keywordsToFlags(const Strings & keywords) const - { - Flags res; - for (const auto & keyword : keywords) - res |= keywordToFlags(keyword); - return res; - } - - std::vector flagsToAccessTypes(const Flags & flags_) const - { - std::vector access_types; - flagsToAccessTypesRec(flags_, access_types, *all_node); - return access_types; - } - - std::vector flagsToKeywords(const Flags & flags_) const - { - std::vector keywords; - flagsToKeywordsRec(flags_, keywords, *all_node); - return keywords; - } - - String flagsToString(const Flags & flags_) const - { - auto keywords = flagsToKeywords(flags_); - if (keywords.empty()) - return "USAGE"; - String str; - for (const auto & keyword : keywords) - { - if (!str.empty()) - str += ", "; - str += keyword; - } - return str; - } - - const Flags & getAllFlags() const { return all_flags; } - const Flags & getGlobalFlags() const { return all_flags_for_target[GLOBAL]; } - const Flags & getDatabaseFlags() const { return all_flags_for_target[DATABASE]; } - const Flags & getTableFlags() const { return all_flags_for_target[TABLE]; } - const Flags & getColumnFlags() const { return all_flags_for_target[COLUMN]; } - const Flags & getDictionaryFlags() const { return all_flags_for_target[DICTIONARY]; } - const Flags & getAllFlagsGrantableOnGlobalLevel() const { return getAllFlags(); } - const Flags & getAllFlagsGrantableOnDatabaseLevel() const { return all_flags_grantable_on_database_level; } - const Flags & getAllFlagsGrantableOnTableLevel() const { return all_flags_grantable_on_table_level; } - const Flags & getAllFlagsGrantableOnColumnLevel() const { return getColumnFlags(); } - -private: - enum NodeType - { - UNKNOWN = -2, - GROUP = -1, - GLOBAL, - DATABASE, - TABLE, - VIEW = TABLE, - COLUMN, - DICTIONARY, - }; - - struct Node; - using NodePtr = std::unique_ptr; - - struct Node - { - const String keyword; - NodeType node_type; - AccessType access_type = AccessType::NONE; - Strings aliases; - Flags flags; - std::vector children; - - Node(String keyword_, NodeType node_type_ = UNKNOWN) : keyword(std::move(keyword_)), node_type(node_type_) {} - - void setFlag(size_t flag) { flags.set(flag); } - - void addChild(NodePtr child) - { - flags |= child->flags; - children.push_back(std::move(child)); - } - }; - - static String replaceUnderscoreWithSpace(const std::string_view & str) - { - String res{str}; - boost::replace_all(res, "_", " "); - return res; - } - - static Strings splitAliases(const std::string_view & str) - { - Strings aliases; - boost::split(aliases, str, boost::is_any_of(",")); - for (auto & alias : aliases) - boost::trim(alias); - return aliases; - } - - static void makeNode( - AccessType access_type, - const std::string_view & name, - const std::string_view & aliases, - NodeType node_type, - const std::string_view & parent_group_name, - std::unordered_map & nodes, - std::unordered_map & owned_nodes, - size_t & next_flag) - { - NodePtr node; - auto keyword = replaceUnderscoreWithSpace(name); - auto it = owned_nodes.find(keyword); - if (it != owned_nodes.end()) - { - node = std::move(it->second); - owned_nodes.erase(it); - } - else - { - if (nodes.count(keyword)) - throw Exception(keyword + " declared twice", ErrorCodes::LOGICAL_ERROR); - node = std::make_unique(keyword, node_type); - nodes[node->keyword] = node.get(); - } - - node->access_type = access_type; - node->node_type = node_type; - node->aliases = splitAliases(aliases); - if (node_type != GROUP) - node->setFlag(next_flag++); - - bool has_parent_group = (parent_group_name != std::string_view{"NONE"}); - if (!has_parent_group) - { - std::string_view keyword_as_string_view = node->keyword; - owned_nodes[keyword_as_string_view] = std::move(node); - return; - } - - auto parent_keyword = replaceUnderscoreWithSpace(parent_group_name); - auto it_parent = nodes.find(parent_keyword); - if (it_parent == nodes.end()) - { - auto parent_node = std::make_unique(parent_keyword); - it_parent = nodes.emplace(parent_node->keyword, parent_node.get()).first; - assert(!owned_nodes.count(parent_node->keyword)); - std::string_view parent_keyword_as_string_view = parent_node->keyword; - owned_nodes[parent_keyword_as_string_view] = std::move(parent_node); - } - it_parent->second->addChild(std::move(node)); - } - - void makeNodes() - { - std::unordered_map owned_nodes; - std::unordered_map nodes; - size_t next_flag = 0; - -#define MAKE_ACCESS_FLAGS_NODE(name, aliases, node_type, parent_group_name) \ - makeNode(AccessType::name, #name, aliases, node_type, #parent_group_name, nodes, owned_nodes, next_flag); - - APPLY_FOR_ACCESS_TYPES(MAKE_ACCESS_FLAGS_NODE) - -#undef MAKE_ACCESS_FLAGS_NODE - - if (!owned_nodes.count("NONE")) - throw Exception("'NONE' not declared", ErrorCodes::LOGICAL_ERROR); - if (!owned_nodes.count("ALL")) - throw Exception("'ALL' not declared", ErrorCodes::LOGICAL_ERROR); - - all_node = std::move(owned_nodes["ALL"]); - none_node = std::move(owned_nodes["NONE"]); - owned_nodes.erase("ALL"); - owned_nodes.erase("NONE"); - - if (!owned_nodes.empty()) - { - const auto & unused_node = *(owned_nodes.begin()->second); - if (unused_node.node_type == UNKNOWN) - throw Exception("Parent group '" + unused_node.keyword + "' not found", ErrorCodes::LOGICAL_ERROR); - else - throw Exception("Access type '" + unused_node.keyword + "' should have parent group", ErrorCodes::LOGICAL_ERROR); - } - } - - void makeKeywordToFlagsMap(Node * start_node = nullptr) - { - if (!start_node) - { - makeKeywordToFlagsMap(none_node.get()); - start_node = all_node.get(); - } - - start_node->aliases.emplace_back(start_node->keyword); - for (auto & alias : start_node->aliases) - { - boost::to_upper(alias); - keyword_to_flags_map[alias] = start_node->flags; - } - - for (auto & child : start_node->children) - makeKeywordToFlagsMap(child.get()); - } - - void makeAccessTypeToFlagsMapping(Node * start_node = nullptr) - { - if (!start_node) - { - makeAccessTypeToFlagsMapping(none_node.get()); - start_node = all_node.get(); - } - - size_t index = static_cast(start_node->access_type); - access_type_to_flags_mapping.resize(std::max(index + 1, access_type_to_flags_mapping.size())); - access_type_to_flags_mapping[index] = start_node->flags; - - for (auto & child : start_node->children) - makeAccessTypeToFlagsMapping(child.get()); - } - - void collectAllFlags(const Node * start_node = nullptr) - { - if (!start_node) - { - start_node = all_node.get(); - all_flags = start_node->flags; - } - if (start_node->node_type != GROUP) - { - assert(static_cast(start_node->node_type) < std::size(all_flags_for_target)); - all_flags_for_target[start_node->node_type] |= start_node->flags; - } - for (const auto & child : start_node->children) - collectAllFlags(child.get()); - - all_flags_grantable_on_table_level = all_flags_for_target[TABLE] | all_flags_for_target[DICTIONARY] | all_flags_for_target[COLUMN]; - all_flags_grantable_on_database_level = all_flags_for_target[DATABASE] | all_flags_grantable_on_table_level; - } - - Impl() - { - makeNodes(); - makeKeywordToFlagsMap(); - makeAccessTypeToFlagsMapping(); - collectAllFlags(); - } - - static void flagsToAccessTypesRec(const Flags & flags_, std::vector & access_types, const Node & start_node) - { - Flags matching_flags = (flags_ & start_node.flags); - if (matching_flags.any()) - { - if (matching_flags == start_node.flags) - { - access_types.push_back(start_node.access_type); - } - else - { - for (const auto & child : start_node.children) - flagsToAccessTypesRec(flags_, access_types, *child); - } - } - } - - static void flagsToKeywordsRec(const Flags & flags_, std::vector & keywords, const Node & start_node) - { - Flags matching_flags = (flags_ & start_node.flags); - if (matching_flags.any()) - { - if (matching_flags == start_node.flags) - { - keywords.push_back(start_node.keyword); - } - else - { - for (const auto & child : start_node.children) - flagsToKeywordsRec(flags_, keywords, *child); - } - } - } - - NodePtr all_node; - NodePtr none_node; - std::unordered_map keyword_to_flags_map; - std::vector access_type_to_flags_mapping; - Flags all_flags; - Flags all_flags_for_target[static_cast(DICTIONARY) + 1]; - Flags all_flags_grantable_on_database_level; - Flags all_flags_grantable_on_table_level; -}; - - -inline AccessFlags::AccessFlags(AccessType type) : flags(Impl<>::instance().accessTypeToFlags(type)) {} -inline AccessFlags::AccessFlags(const std::string_view & keyword) : flags(Impl<>::instance().keywordToFlags(keyword)) {} -inline AccessFlags::AccessFlags(const std::vector & keywords) : flags(Impl<>::instance().keywordsToFlags(keywords)) {} -inline AccessFlags::AccessFlags(const Strings & keywords) : flags(Impl<>::instance().keywordsToFlags(keywords)) {} -inline String AccessFlags::toString() const { return Impl<>::instance().flagsToString(flags); } -inline std::vector AccessFlags::toAccessTypes() const { return Impl<>::instance().flagsToAccessTypes(flags); } -inline std::vector AccessFlags::toKeywords() const { return Impl<>::instance().flagsToKeywords(flags); } -inline AccessFlags AccessFlags::allFlags() { return Impl<>::instance().getAllFlags(); } -inline AccessFlags AccessFlags::allGlobalFlags() { return Impl<>::instance().getGlobalFlags(); } -inline AccessFlags AccessFlags::allDatabaseFlags() { return Impl<>::instance().getDatabaseFlags(); } -inline AccessFlags AccessFlags::allTableFlags() { return Impl<>::instance().getTableFlags(); } -inline AccessFlags AccessFlags::allColumnFlags() { return Impl<>::instance().getColumnFlags(); } -inline AccessFlags AccessFlags::allDictionaryFlags() { return Impl<>::instance().getDictionaryFlags(); } -inline AccessFlags AccessFlags::allFlagsGrantableOnGlobalLevel() { return Impl<>::instance().getAllFlagsGrantableOnGlobalLevel(); } -inline AccessFlags AccessFlags::allFlagsGrantableOnDatabaseLevel() { return Impl<>::instance().getAllFlagsGrantableOnDatabaseLevel(); } -inline AccessFlags AccessFlags::allFlagsGrantableOnTableLevel() { return Impl<>::instance().getAllFlagsGrantableOnTableLevel(); } -inline AccessFlags AccessFlags::allFlagsGrantableOnColumnLevel() { return Impl<>::instance().getAllFlagsGrantableOnColumnLevel(); } - -inline AccessFlags operator |(AccessType left, AccessType right) { return AccessFlags(left) | right; } -inline AccessFlags operator &(AccessType left, AccessType right) { return AccessFlags(left) & right; } -inline AccessFlags operator -(AccessType left, AccessType right) { return AccessFlags(left) - right; } -inline AccessFlags operator ~(AccessType x) { return ~AccessFlags(x); } +AccessFlags operator |(AccessType left, AccessType right); +AccessFlags operator &(AccessType left, AccessType right); +AccessFlags operator -(AccessType left, AccessType right); +AccessFlags operator ~(AccessType x); } diff --git a/src/Access/Common/AccessRightsElement.cpp b/src/Access/Common/AccessRightsElement.cpp index 5d62de69e40..9913fc02f4a 100644 --- a/src/Access/Common/AccessRightsElement.cpp +++ b/src/Access/Common/AccessRightsElement.cpp @@ -145,10 +145,85 @@ namespace } +AccessRightsElement::AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_) + : access_flags(access_flags_), database(database_), any_database(false) +{ +} + +AccessRightsElement::AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_) + : access_flags(access_flags_), database(database_), table(table_), any_database(false), any_table(false) +{ +} + +AccessRightsElement::AccessRightsElement( + AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const std::string_view & column_) + : access_flags(access_flags_) + , database(database_) + , table(table_) + , columns({String{column_}}) + , any_database(false) + , any_table(false) + , any_column(false) +{ +} + +AccessRightsElement::AccessRightsElement( + AccessFlags access_flags_, + const std::string_view & database_, + const std::string_view & table_, + const std::vector & columns_) + : access_flags(access_flags_), database(database_), table(table_), any_database(false), any_table(false), any_column(false) +{ + columns.resize(columns_.size()); + for (size_t i = 0; i != columns_.size(); ++i) + columns[i] = String{columns_[i]}; +} + +AccessRightsElement::AccessRightsElement( + AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const Strings & columns_) + : access_flags(access_flags_) + , database(database_) + , table(table_) + , columns(columns_) + , any_database(false) + , any_table(false) + , any_column(false) +{ +} + +void AccessRightsElement::eraseNonGrantable() +{ + if (!any_column) + access_flags &= AccessFlags::allFlagsGrantableOnColumnLevel(); + else if (!any_table) + access_flags &= AccessFlags::allFlagsGrantableOnTableLevel(); + else if (!any_database) + access_flags &= AccessFlags::allFlagsGrantableOnDatabaseLevel(); + else + access_flags &= AccessFlags::allFlagsGrantableOnGlobalLevel(); +} + +void AccessRightsElement::replaceEmptyDatabase(const String & current_database) +{ + if (isEmptyDatabase()) + database = current_database; +} + String AccessRightsElement::toString() const { return toStringImpl(*this, true); } String AccessRightsElement::toStringWithoutOptions() const { return toStringImpl(*this, false); } -String AccessRightsElements::toString() const { return toStringImpl(*this, true); } -String AccessRightsElements::toStringWithoutOptions() const { return toStringImpl(*this, false); } + + +bool AccessRightsElements::empty() const { return std::all_of(begin(), end(), [](const AccessRightsElement & e) { return e.empty(); }); } + +bool AccessRightsElements::sameDatabaseAndTable() const +{ + return (size() < 2) || std::all_of(std::next(begin()), end(), [this](const AccessRightsElement & e) { return e.sameDatabaseAndTable(front()); }); +} + +bool AccessRightsElements::sameOptions() const +{ + return (size() < 2) || std::all_of(std::next(begin()), end(), [this](const AccessRightsElement & e) { return e.sameOptions(front()); }); +} void AccessRightsElements::eraseNonGrantable() { @@ -159,4 +234,13 @@ void AccessRightsElements::eraseNonGrantable() }); } +void AccessRightsElements::replaceEmptyDatabase(const String & current_database) +{ + for (auto & element : *this) + element.replaceEmptyDatabase(current_database); +} + +String AccessRightsElements::toString() const { return toStringImpl(*this, true); } +String AccessRightsElements::toStringWithoutOptions() const { return toStringImpl(*this, false); } + } diff --git a/src/Access/Common/AccessRightsElement.h b/src/Access/Common/AccessRightsElement.h index 012c8aa0d12..4fb58b39185 100644 --- a/src/Access/Common/AccessRightsElement.h +++ b/src/Access/Common/AccessRightsElement.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB @@ -27,51 +28,19 @@ struct AccessRightsElement AccessRightsElement(AccessFlags access_flags_) : access_flags(access_flags_) {} - AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_) - : access_flags(access_flags_), database(database_), any_database(false) - { - } - - AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_) - : access_flags(access_flags_), database(database_), table(table_), any_database(false), any_table(false) - { - } - + AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_); + AccessRightsElement(AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_); AccessRightsElement( - AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const std::string_view & column_) - : access_flags(access_flags_) - , database(database_) - , table(table_) - , columns({String{column_}}) - , any_database(false) - , any_table(false) - , any_column(false) - { - } + AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const std::string_view & column_); AccessRightsElement( AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, - const std::vector & columns_) - : access_flags(access_flags_), database(database_), table(table_), any_database(false), any_table(false), any_column(false) - { - columns.resize(columns_.size()); - for (size_t i = 0; i != columns_.size(); ++i) - columns[i] = String{columns_[i]}; - } + const std::vector & columns_); AccessRightsElement( - AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const Strings & columns_) - : access_flags(access_flags_) - , database(database_) - , table(table_) - , columns(columns_) - , any_database(false) - , any_table(false) - , any_column(false) - { - } + AccessFlags access_flags_, const std::string_view & database_, const std::string_view & table_, const Strings & columns_); bool empty() const { return !access_flags || (!any_column && columns.empty()); } @@ -91,26 +60,12 @@ struct AccessRightsElement } /// Resets flags which cannot be granted. - void eraseNonGrantable() - { - if (!any_column) - access_flags &= AccessFlags::allFlagsGrantableOnColumnLevel(); - else if (!any_table) - access_flags &= AccessFlags::allFlagsGrantableOnTableLevel(); - else if (!any_database) - access_flags &= AccessFlags::allFlagsGrantableOnDatabaseLevel(); - else - access_flags &= AccessFlags::allFlagsGrantableOnGlobalLevel(); - } + void eraseNonGrantable(); bool isEmptyDatabase() const { return !any_database && database.empty(); } /// If the database is empty, replaces it with `current_database`. Otherwise does nothing. - void replaceEmptyDatabase(const String & current_database) - { - if (isEmptyDatabase()) - database = current_database; - } + void replaceEmptyDatabase(const String & current_database); /// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table". String toString() const; @@ -125,27 +80,15 @@ public: using Base = std::vector; using Base::Base; - bool empty() const { return std::all_of(begin(), end(), [](const AccessRightsElement & e) { return e.empty(); }); } - - bool sameDatabaseAndTable() const - { - return (size() < 2) || std::all_of(std::next(begin()), end(), [this](const AccessRightsElement & e) { return e.sameDatabaseAndTable(front()); }); - } - - bool sameOptions() const - { - return (size() < 2) || std::all_of(std::next(begin()), end(), [this](const AccessRightsElement & e) { return e.sameOptions(front()); }); - } + bool empty() const; + bool sameDatabaseAndTable() const; + bool sameOptions() const; /// Resets flags which cannot be granted. void eraseNonGrantable(); /// If the database is empty, replaces it with `current_database`. Otherwise does nothing. - void replaceEmptyDatabase(const String & current_database) - { - for (auto & element : *this) - element.replaceEmptyDatabase(current_database); - } + void replaceEmptyDatabase(const String & current_database); /// Returns a human-readable representation like "GRANT SELECT, UPDATE(x, y) ON db.table". String toString() const; diff --git a/src/Access/Common/AccessType.cpp b/src/Access/Common/AccessType.cpp new file mode 100644 index 00000000000..d44d70d78b2 --- /dev/null +++ b/src/Access/Common/AccessType.cpp @@ -0,0 +1,56 @@ +#include +#include +#include + + +namespace DB +{ + +namespace +{ + using Strings = std::vector; + + class AccessTypeToStringConverter + { + public: + static const AccessTypeToStringConverter & instance() + { + static const AccessTypeToStringConverter res; + return res; + } + + std::string_view convert(AccessType type) const + { + return access_type_to_string_mapping[static_cast(type)]; + } + + private: + AccessTypeToStringConverter() + { +#define ACCESS_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING(name, aliases, node_type, parent_group_name) \ + addToMapping(AccessType::name, #name); + + APPLY_FOR_ACCESS_TYPES(ACCESS_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING) + +#undef ACCESS_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING + } + + void addToMapping(AccessType type, const std::string_view & str) + { + String str2{str}; + boost::replace_all(str2, "_", " "); + size_t index = static_cast(type); + access_type_to_string_mapping.resize(std::max(index + 1, access_type_to_string_mapping.size())); + access_type_to_string_mapping[index] = str2; + } + + Strings access_type_to_string_mapping; + }; +} + +std::string_view toString(AccessType type) +{ + return AccessTypeToStringConverter::instance().convert(type); +} + +} diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index aa7dcbb006b..cb6c326cb84 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -1,17 +1,11 @@ #pragma once #include -#include -#include -#include -#include namespace DB { -using Strings = std::vector; - /// Represents an access type which can be granted on databases, tables, columns, etc. enum class AccessType { @@ -198,48 +192,6 @@ enum class AccessType #undef DECLARE_ACCESS_TYPE_ENUM_CONST }; - -namespace impl -{ - template - class AccessTypeToStringConverter - { - public: - static const AccessTypeToStringConverter & instance() - { - static const AccessTypeToStringConverter res; - return res; - } - - std::string_view convert(AccessType type) const - { - return access_type_to_string_mapping[static_cast(type)]; - } - - private: - AccessTypeToStringConverter() - { -#define ACCESS_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING(name, aliases, node_type, parent_group_name) \ - addToMapping(AccessType::name, #name); - - APPLY_FOR_ACCESS_TYPES(ACCESS_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING) - -#undef ACCESS_TYPE_TO_STRING_CONVERTER_ADD_TO_MAPPING - } - - void addToMapping(AccessType type, const std::string_view & str) - { - String str2{str}; - boost::replace_all(str2, "_", " "); - size_t index = static_cast(type); - access_type_to_string_mapping.resize(std::max(index + 1, access_type_to_string_mapping.size())); - access_type_to_string_mapping[index] = str2; - } - - Strings access_type_to_string_mapping; - }; -} - -inline std::string_view toString(AccessType type) { return impl::AccessTypeToStringConverter<>::instance().convert(type); } +std::string_view toString(AccessType type); } diff --git a/src/Access/Common/AllowedClientHosts.cpp b/src/Access/Common/AllowedClientHosts.cpp index c5e87844376..62cef97e90e 100644 --- a/src/Access/Common/AllowedClientHosts.cpp +++ b/src/Access/Common/AllowedClientHosts.cpp @@ -5,9 +5,15 @@ #include #include #include +#include #include -#include +#include +#include #include +#include +#include + +namespace fs = std::filesystem; namespace DB @@ -186,6 +192,262 @@ namespace } +void AllowedClientHosts::IPSubnet::set(const IPAddress & prefix_, const IPAddress & mask_) +{ + prefix = prefix_; + mask = mask_; + + if (prefix.family() != mask.family()) + { + if (prefix.family() == IPAddress::IPv4) + prefix = IPAddress("::ffff:" + prefix.toString()); + + if (mask.family() == IPAddress::IPv4) + mask = IPAddress(96, IPAddress::IPv6) | IPAddress("::ffff:" + mask.toString()); + } + + prefix = prefix & mask; + + if (prefix.family() == IPAddress::IPv4) + { + if ((prefix & IPAddress{8, IPAddress::IPv4}) == IPAddress{"127.0.0.0"}) + { + // 127.XX.XX.XX -> 127.0.0.1 + prefix = IPAddress{"127.0.0.1"}; + mask = IPAddress{32, IPAddress::IPv4}; + } + } + else + { + if ((prefix & IPAddress{104, IPAddress::IPv6}) == IPAddress{"::ffff:127.0.0.0"}) + { + // ::ffff:127.XX.XX.XX -> ::1 + prefix = IPAddress{"::1"}; + mask = IPAddress{128, IPAddress::IPv6}; + } + } +} + +void AllowedClientHosts::IPSubnet::set(const IPAddress & prefix_, size_t num_prefix_bits) +{ + set(prefix_, IPAddress(num_prefix_bits, prefix_.family())); +} + +void AllowedClientHosts::IPSubnet::set(const IPAddress & address) +{ + set(address, address.length() * 8); +} + +AllowedClientHosts::IPSubnet::IPSubnet(const String & str) +{ + size_t slash = str.find('/'); + if (slash == String::npos) + { + set(IPAddress(str)); + return; + } + + IPAddress new_prefix{String{str, 0, slash}}; + String mask_str(str, slash + 1, str.length() - slash - 1); + bool only_digits = (mask_str.find_first_not_of("0123456789") == std::string::npos); + if (only_digits) + set(new_prefix, std::stoul(mask_str)); + else + set(new_prefix, IPAddress{mask_str}); +} + +String AllowedClientHosts::IPSubnet::toString() const +{ + unsigned int prefix_length = mask.prefixLength(); + if (isMaskAllBitsOne()) + return prefix.toString(); + else if (IPAddress{prefix_length, mask.family()} == mask) + return fs::path(prefix.toString()) / std::to_string(prefix_length); + else + return fs::path(prefix.toString()) / mask.toString(); +} + +bool AllowedClientHosts::IPSubnet::isMaskAllBitsOne() const +{ + return mask == IPAddress(mask.length() * 8, mask.family()); +} + + +void AllowedClientHosts::clear() +{ + addresses = {}; + subnets = {}; + names = {}; + name_regexps = {}; + like_patterns = {}; + any_host = false; + local_host = false; +} + +bool AllowedClientHosts::empty() const +{ + return !any_host && !local_host && addresses.empty() && subnets.empty() && names.empty() && name_regexps.empty() && like_patterns.empty(); +} + +void AllowedClientHosts::addAddress(const IPAddress & address) +{ + if (address.isLoopback()) + local_host = true; + else if (boost::range::find(addresses, address) == addresses.end()) + addresses.push_back(address); +} + +void AllowedClientHosts::removeAddress(const IPAddress & address) +{ + if (address.isLoopback()) + local_host = false; + else + boost::range::remove_erase(addresses, address); +} + +void AllowedClientHosts::addSubnet(const IPSubnet & subnet) +{ + if (subnet.getMask().isWildcard()) + any_host = true; + else if (subnet.isMaskAllBitsOne()) + addAddress(subnet.getPrefix()); + else if (boost::range::find(subnets, subnet) == subnets.end()) + subnets.push_back(subnet); +} + +void AllowedClientHosts::removeSubnet(const IPSubnet & subnet) +{ + if (subnet.getMask().isWildcard()) + any_host = false; + else if (subnet.isMaskAllBitsOne()) + removeAddress(subnet.getPrefix()); + else + boost::range::remove_erase(subnets, subnet); +} + +void AllowedClientHosts::addName(const String & name) +{ + if (boost::iequals(name, "localhost")) + local_host = true; + else if (boost::range::find(names, name) == names.end()) + names.push_back(name); +} + +void AllowedClientHosts::removeName(const String & name) +{ + if (boost::iequals(name, "localhost")) + local_host = false; + else + boost::range::remove_erase(names, name); +} + +void AllowedClientHosts::addNameRegexp(const String & name_regexp) +{ + if (boost::iequals(name_regexp, "localhost")) + local_host = true; + else if (name_regexp == ".*") + any_host = true; + else if (boost::range::find(name_regexps, name_regexp) == name_regexps.end()) + name_regexps.push_back(name_regexp); +} + +void AllowedClientHosts::removeNameRegexp(const String & name_regexp) +{ + if (boost::iequals(name_regexp, "localhost")) + local_host = false; + else if (name_regexp == ".*") + any_host = false; + else + boost::range::remove_erase(name_regexps, name_regexp); +} + +void AllowedClientHosts::addLikePattern(const String & pattern) +{ + if (boost::iequals(pattern, "localhost") || (pattern == "127.0.0.1") || (pattern == "::1")) + local_host = true; + else if ((pattern == "%") || (pattern == "0.0.0.0/0") || (pattern == "::/0")) + any_host = true; + else if (boost::range::find(like_patterns, pattern) == name_regexps.end()) + like_patterns.push_back(pattern); +} + +void AllowedClientHosts::removeLikePattern(const String & pattern) +{ + if (boost::iequals(pattern, "localhost") || (pattern == "127.0.0.1") || (pattern == "::1")) + local_host = false; + else if ((pattern == "%") || (pattern == "0.0.0.0/0") || (pattern == "::/0")) + any_host = false; + else + boost::range::remove_erase(like_patterns, pattern); +} + +void AllowedClientHosts::addLocalHost() +{ + local_host = true; +} + +void AllowedClientHosts::removeLocalHost() +{ + local_host = false; +} + +void AllowedClientHosts::addAnyHost() +{ + clear(); + any_host = true; +} + +void AllowedClientHosts::add(const AllowedClientHosts & other) +{ + if (other.containsAnyHost()) + { + addAnyHost(); + return; + } + if (other.containsLocalHost()) + addLocalHost(); + for (const IPAddress & address : other.getAddresses()) + addAddress(address); + for (const IPSubnet & subnet : other.getSubnets()) + addSubnet(subnet); + for (const String & name : other.getNames()) + addName(name); + for (const String & name_regexp : other.getNameRegexps()) + addNameRegexp(name_regexp); + for (const String & like_pattern : other.getLikePatterns()) + addLikePattern(like_pattern); +} + +void AllowedClientHosts::remove(const AllowedClientHosts & other) +{ + if (other.containsAnyHost()) + { + clear(); + return; + } + if (other.containsLocalHost()) + removeLocalHost(); + for (const IPAddress & address : other.getAddresses()) + removeAddress(address); + for (const IPSubnet & subnet : other.getSubnets()) + removeSubnet(subnet); + for (const String & name : other.getNames()) + removeName(name); + for (const String & name_regexp : other.getNameRegexps()) + removeNameRegexp(name_regexp); + for (const String & like_pattern : other.getLikePatterns()) + removeLikePattern(like_pattern); +} + + +bool operator ==(const AllowedClientHosts & lhs, const AllowedClientHosts & rhs) +{ + return (lhs.any_host == rhs.any_host) && (lhs.local_host == rhs.local_host) && (lhs.addresses == rhs.addresses) + && (lhs.subnets == rhs.subnets) && (lhs.names == rhs.names) && (lhs.name_regexps == rhs.name_regexps) + && (lhs.like_patterns == rhs.like_patterns); +} + + bool AllowedClientHosts::contains(const IPAddress & client_address) const { if (any_host) diff --git a/src/Access/Common/AllowedClientHosts.h b/src/Access/Common/AllowedClientHosts.h index 30c0dac076e..6a6e3d73eef 100644 --- a/src/Access/Common/AllowedClientHosts.h +++ b/src/Access/Common/AllowedClientHosts.h @@ -2,18 +2,11 @@ #include #include -#include #include -#include -#include -#include -#include -namespace fs = std::filesystem; namespace DB { - using Strings = std::vector; /// Represents lists of hosts a user is allowed to connect to server from. @@ -129,260 +122,4 @@ private: bool local_host = false; }; - -inline void AllowedClientHosts::IPSubnet::set(const IPAddress & prefix_, const IPAddress & mask_) -{ - prefix = prefix_; - mask = mask_; - - if (prefix.family() != mask.family()) - { - if (prefix.family() == IPAddress::IPv4) - prefix = IPAddress("::ffff:" + prefix.toString()); - - if (mask.family() == IPAddress::IPv4) - mask = IPAddress(96, IPAddress::IPv6) | IPAddress("::ffff:" + mask.toString()); - } - - prefix = prefix & mask; - - if (prefix.family() == IPAddress::IPv4) - { - if ((prefix & IPAddress{8, IPAddress::IPv4}) == IPAddress{"127.0.0.0"}) - { - // 127.XX.XX.XX -> 127.0.0.1 - prefix = IPAddress{"127.0.0.1"}; - mask = IPAddress{32, IPAddress::IPv4}; - } - } - else - { - if ((prefix & IPAddress{104, IPAddress::IPv6}) == IPAddress{"::ffff:127.0.0.0"}) - { - // ::ffff:127.XX.XX.XX -> ::1 - prefix = IPAddress{"::1"}; - mask = IPAddress{128, IPAddress::IPv6}; - } - } -} - -inline void AllowedClientHosts::IPSubnet::set(const IPAddress & prefix_, size_t num_prefix_bits) -{ - set(prefix_, IPAddress(num_prefix_bits, prefix_.family())); -} - -inline void AllowedClientHosts::IPSubnet::set(const IPAddress & address) -{ - set(address, address.length() * 8); -} - -inline AllowedClientHosts::IPSubnet::IPSubnet(const String & str) -{ - size_t slash = str.find('/'); - if (slash == String::npos) - { - set(IPAddress(str)); - return; - } - - IPAddress new_prefix{String{str, 0, slash}}; - String mask_str(str, slash + 1, str.length() - slash - 1); - bool only_digits = (mask_str.find_first_not_of("0123456789") == std::string::npos); - if (only_digits) - set(new_prefix, std::stoul(mask_str)); - else - set(new_prefix, IPAddress{mask_str}); -} - -inline String AllowedClientHosts::IPSubnet::toString() const -{ - unsigned int prefix_length = mask.prefixLength(); - if (isMaskAllBitsOne()) - return prefix.toString(); - else if (IPAddress{prefix_length, mask.family()} == mask) - return fs::path(prefix.toString()) / std::to_string(prefix_length); - else - return fs::path(prefix.toString()) / mask.toString(); -} - -inline bool AllowedClientHosts::IPSubnet::isMaskAllBitsOne() const -{ - return mask == IPAddress(mask.length() * 8, mask.family()); -} - - -inline void AllowedClientHosts::clear() -{ - addresses = {}; - subnets = {}; - names = {}; - name_regexps = {}; - like_patterns = {}; - any_host = false; - local_host = false; -} - -inline bool AllowedClientHosts::empty() const -{ - return !any_host && !local_host && addresses.empty() && subnets.empty() && names.empty() && name_regexps.empty() && like_patterns.empty(); -} - -inline void AllowedClientHosts::addAddress(const IPAddress & address) -{ - if (address.isLoopback()) - local_host = true; - else if (boost::range::find(addresses, address) == addresses.end()) - addresses.push_back(address); -} - -inline void AllowedClientHosts::removeAddress(const IPAddress & address) -{ - if (address.isLoopback()) - local_host = false; - else - boost::range::remove_erase(addresses, address); -} - -inline void AllowedClientHosts::addSubnet(const IPSubnet & subnet) -{ - if (subnet.getMask().isWildcard()) - any_host = true; - else if (subnet.isMaskAllBitsOne()) - addAddress(subnet.getPrefix()); - else if (boost::range::find(subnets, subnet) == subnets.end()) - subnets.push_back(subnet); -} - -inline void AllowedClientHosts::removeSubnet(const IPSubnet & subnet) -{ - if (subnet.getMask().isWildcard()) - any_host = false; - else if (subnet.isMaskAllBitsOne()) - removeAddress(subnet.getPrefix()); - else - boost::range::remove_erase(subnets, subnet); -} - -inline void AllowedClientHosts::addName(const String & name) -{ - if (boost::iequals(name, "localhost")) - local_host = true; - else if (boost::range::find(names, name) == names.end()) - names.push_back(name); -} - -inline void AllowedClientHosts::removeName(const String & name) -{ - if (boost::iequals(name, "localhost")) - local_host = false; - else - boost::range::remove_erase(names, name); -} - -inline void AllowedClientHosts::addNameRegexp(const String & name_regexp) -{ - if (boost::iequals(name_regexp, "localhost")) - local_host = true; - else if (name_regexp == ".*") - any_host = true; - else if (boost::range::find(name_regexps, name_regexp) == name_regexps.end()) - name_regexps.push_back(name_regexp); -} - -inline void AllowedClientHosts::removeNameRegexp(const String & name_regexp) -{ - if (boost::iequals(name_regexp, "localhost")) - local_host = false; - else if (name_regexp == ".*") - any_host = false; - else - boost::range::remove_erase(name_regexps, name_regexp); -} - -inline void AllowedClientHosts::addLikePattern(const String & pattern) -{ - if (boost::iequals(pattern, "localhost") || (pattern == "127.0.0.1") || (pattern == "::1")) - local_host = true; - else if ((pattern == "%") || (pattern == "0.0.0.0/0") || (pattern == "::/0")) - any_host = true; - else if (boost::range::find(like_patterns, pattern) == name_regexps.end()) - like_patterns.push_back(pattern); -} - -inline void AllowedClientHosts::removeLikePattern(const String & pattern) -{ - if (boost::iequals(pattern, "localhost") || (pattern == "127.0.0.1") || (pattern == "::1")) - local_host = false; - else if ((pattern == "%") || (pattern == "0.0.0.0/0") || (pattern == "::/0")) - any_host = false; - else - boost::range::remove_erase(like_patterns, pattern); -} - -inline void AllowedClientHosts::addLocalHost() -{ - local_host = true; -} - -inline void AllowedClientHosts::removeLocalHost() -{ - local_host = false; -} - -inline void AllowedClientHosts::addAnyHost() -{ - clear(); - any_host = true; -} - -inline void AllowedClientHosts::add(const AllowedClientHosts & other) -{ - if (other.containsAnyHost()) - { - addAnyHost(); - return; - } - if (other.containsLocalHost()) - addLocalHost(); - for (const IPAddress & address : other.getAddresses()) - addAddress(address); - for (const IPSubnet & subnet : other.getSubnets()) - addSubnet(subnet); - for (const String & name : other.getNames()) - addName(name); - for (const String & name_regexp : other.getNameRegexps()) - addNameRegexp(name_regexp); - for (const String & like_pattern : other.getLikePatterns()) - addLikePattern(like_pattern); -} - -inline void AllowedClientHosts::remove(const AllowedClientHosts & other) -{ - if (other.containsAnyHost()) - { - clear(); - return; - } - if (other.containsLocalHost()) - removeLocalHost(); - for (const IPAddress & address : other.getAddresses()) - removeAddress(address); - for (const IPSubnet & subnet : other.getSubnets()) - removeSubnet(subnet); - for (const String & name : other.getNames()) - removeName(name); - for (const String & name_regexp : other.getNameRegexps()) - removeNameRegexp(name_regexp); - for (const String & like_pattern : other.getLikePatterns()) - removeLikePattern(like_pattern); -} - - -inline bool operator ==(const AllowedClientHosts & lhs, const AllowedClientHosts & rhs) -{ - return (lhs.any_host == rhs.any_host) && (lhs.local_host == rhs.local_host) && (lhs.addresses == rhs.addresses) - && (lhs.subnets == rhs.subnets) && (lhs.names == rhs.names) && (lhs.name_regexps == rhs.name_regexps) - && (lhs.like_patterns == rhs.like_patterns); -} - } diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index 14cac3a828f..a7c91faf43b 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -8,6 +8,7 @@ #include #include #include +#include namespace Poco { class Logger; } diff --git a/src/Interpreters/Access/InterpreterGrantQuery.cpp b/src/Interpreters/Access/InterpreterGrantQuery.cpp index 3ace4531de6..e17af6877be 100644 --- a/src/Interpreters/Access/InterpreterGrantQuery.cpp +++ b/src/Interpreters/Access/InterpreterGrantQuery.cpp @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB { From 44875ae79d30e2e3ee4055f1bb8b5ca8db6b9871 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 3 Nov 2021 00:06:08 +0300 Subject: [PATCH 388/396] fix pvs check --- src/Processors/Transforms/FillingTransform.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index ae1717b9e21..52b84c98a08 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -45,6 +45,7 @@ static FillColumnDescription::StepFunction getStepFunction( FOR_EACH_INTERVAL_KIND(DECLARE_CASE) #undef DECLARE_CASE } + __builtin_unreachable(); } static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & type) @@ -102,7 +103,6 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & descr.step_func = getStepFunction(*descr.step_kind, get(descr.fill_step), date_time->getTimeZone()); else if (const auto * date_time64 = checkAndGetDataType(type.get())) { - const auto & time_zone = date_time64->getTimeZone(); const auto & step_dec = get &>(descr.fill_step); Int64 step = DecimalUtils::convertTo(step_dec.getValue(), step_dec.getScale()); @@ -110,7 +110,7 @@ static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & { #define DECLARE_CASE(NAME) \ case IntervalKind::NAME: \ - descr.step_func = [step, &time_zone](Field & field) \ + descr.step_func = [step, &time_zone = date_time64->getTimeZone()](Field & field) \ { \ auto field_decimal = get>(field); \ auto components = DecimalUtils::splitWithScaleMultiplier(field_decimal.getValue(), field_decimal.getScaleMultiplier()); \ From afd960db4e96fc4e5d3283224b8263bad4de665c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 3 Nov 2021 08:09:19 +0300 Subject: [PATCH 389/396] perf: remove redundant query profiler set queries --- docker/test/performance-comparison/compare.sh | 5 ----- 1 file changed, 5 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 2e8517d71c9..b6a06be2ac7 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -308,12 +308,7 @@ function get_profiles_watchdog function get_profiles { # Collect the profiles - clickhouse-client --port $LEFT_SERVER_PORT --query "set query_profiler_cpu_time_period_ns = 0" - clickhouse-client --port $LEFT_SERVER_PORT --query "set query_profiler_real_time_period_ns = 0" clickhouse-client --port $LEFT_SERVER_PORT --query "system flush logs" & - - clickhouse-client --port $RIGHT_SERVER_PORT --query "set query_profiler_cpu_time_period_ns = 0" - clickhouse-client --port $RIGHT_SERVER_PORT --query "set query_profiler_real_time_period_ns = 0" clickhouse-client --port $RIGHT_SERVER_PORT --query "system flush logs" & wait From 82d6fead89cff86978b9960fb482d11ef5cfcb49 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 3 Nov 2021 08:13:08 +0300 Subject: [PATCH 390/396] perf: add missing DROP TABLE queries --- tests/performance/sum_map.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/performance/sum_map.xml b/tests/performance/sum_map.xml index bc9f9be2a18..f55af077023 100644 --- a/tests/performance/sum_map.xml +++ b/tests/performance/sum_map.xml @@ -31,4 +31,6 @@ SELECT {func}(key, val) FROM sum_map_{scale} FORMAT Null SELECT {func}((key, val)) FROM sum_map_{scale} FORMAT Null + + DROP TABLE sum_map_{scale} From acef85b17d69568381c55a3c3ce4f0b9d5ea37aa Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 3 Nov 2021 08:19:18 +0300 Subject: [PATCH 391/396] perf: cleanup server overrides --- .../config.d/zzz-perf-comparison-tweaks-config.xml | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml b/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml index cc5dc3795bb..0cff3f16f1a 100644 --- a/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml +++ b/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml @@ -11,16 +11,7 @@ true - - - - - - system -
metric_log
- 7500 - 1000 - + 1000000000 From 837fe8554f9adca4cedf9e7eb0115b0b7067c13c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 3 Nov 2021 08:20:53 +0300 Subject: [PATCH 392/396] perf: remove zookeeper to avoid configuring DDLWorker --- .../config/config.d/zzz-perf-comparison-tweaks-config.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml b/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml index 0cff3f16f1a..795ca1da0aa 100644 --- a/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml +++ b/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml @@ -5,6 +5,7 @@ + :: From ff19d223053467a441137faa8f50c4e758436c43 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 3 Nov 2021 08:25:01 +0300 Subject: [PATCH 393/396] perf: remove over *_log tables that does not exported into artifacts --- .../config/config.d/zzz-perf-comparison-tweaks-config.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml b/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml index 795ca1da0aa..292665c4f68 100644 --- a/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml +++ b/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml @@ -13,6 +13,11 @@ + + + + + 1000000000 From 0047d2eb4f5b2fc5825b8a858d60d5b1e3f2b2e9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 3 Nov 2021 08:34:09 +0300 Subject: [PATCH 394/396] Remove metric_log/part_log overrides in tests (enabled by default) --- programs/server/config.d/metric_log.xml | 1 - programs/server/config.d/part_log.xml | 1 - tests/config/config.d/metric_log.xml | 8 -------- tests/config/config.d/part_log.xml | 8 -------- tests/config/install.sh | 2 -- 5 files changed, 20 deletions(-) delete mode 120000 programs/server/config.d/metric_log.xml delete mode 120000 programs/server/config.d/part_log.xml delete mode 100644 tests/config/config.d/metric_log.xml delete mode 100644 tests/config/config.d/part_log.xml diff --git a/programs/server/config.d/metric_log.xml b/programs/server/config.d/metric_log.xml deleted file mode 120000 index 7f033c60a64..00000000000 --- a/programs/server/config.d/metric_log.xml +++ /dev/null @@ -1 +0,0 @@ -../../../tests/config/config.d/metric_log.xml \ No newline at end of file diff --git a/programs/server/config.d/part_log.xml b/programs/server/config.d/part_log.xml deleted file mode 120000 index d97ea7f226d..00000000000 --- a/programs/server/config.d/part_log.xml +++ /dev/null @@ -1 +0,0 @@ -../../../tests/config/config.d/part_log.xml \ No newline at end of file diff --git a/tests/config/config.d/metric_log.xml b/tests/config/config.d/metric_log.xml deleted file mode 100644 index ea829d15975..00000000000 --- a/tests/config/config.d/metric_log.xml +++ /dev/null @@ -1,8 +0,0 @@ - - - system - metric_log
- 7500 - 1000 -
-
diff --git a/tests/config/config.d/part_log.xml b/tests/config/config.d/part_log.xml deleted file mode 100644 index ce9847a49fb..00000000000 --- a/tests/config/config.d/part_log.xml +++ /dev/null @@ -1,8 +0,0 @@ - - - system - part_log
- - 7500 -
-
diff --git a/tests/config/install.sh b/tests/config/install.sh index a451c9f3ed1..edf897ca430 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -17,9 +17,7 @@ mkdir -p $DEST_CLIENT_PATH ln -sf $SRC_PATH/config.d/zookeeper.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/listen.xml $DEST_SERVER_PATH/config.d/ -ln -sf $SRC_PATH/config.d/part_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/text_log.xml $DEST_SERVER_PATH/config.d/ -ln -sf $SRC_PATH/config.d/metric_log.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/custom_settings_prefixes.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/macros.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/disks.xml $DEST_SERVER_PATH/config.d/ From 59c8ed9b0cbbd6a871ca27e7fe8a729aeea718c9 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 3 Nov 2021 13:25:45 +0300 Subject: [PATCH 395/396] Fixed is_cancelled predicate inside MergeTask (#30996) --- src/Storages/MergeTree/MergeTask.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 5cb819c44a4..4a438795c88 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -255,9 +255,12 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() ctx->is_cancelled = [merges_blocker = global_ctx->merges_blocker, ttl_merges_blocker = global_ctx->ttl_merges_blocker, - need_remove = ctx->need_remove_expired_values]() -> bool + need_remove = ctx->need_remove_expired_values, + merge_list_element = global_ctx->merge_list_element_ptr]() -> bool { - return merges_blocker->isCancelled() || (need_remove && ttl_merges_blocker->isCancelled()); + return merges_blocker->isCancelled() + || (need_remove && ttl_merges_blocker->isCancelled()) + || merge_list_element->is_cancelled.load(std::memory_order_relaxed); }; /// This is the end of preparation. Execution will be per block. From 08545795b5fe5d5e497b3ac51776307b41717123 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Nov 2021 12:23:33 +0300 Subject: [PATCH 396/396] Separate option for enabling fuse syntax for sum, avg, count --- src/Core/Settings.h | 4 ++-- src/Interpreters/TreeRewriter.cpp | 2 +- tests/queries/0_stateless/01744_fuse_sum_count_aggregate.sql | 2 ++ 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ab6a4f9a4bf..f8b574a21d8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -459,7 +459,8 @@ class IColumn; M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \ M(Bool, enable_global_with_statement, true, "Propagate WITH statements to UNION queries and all subqueries", 0) \ M(Bool, aggregate_functions_null_for_empty, false, "Rewrite all aggregate functions in a query, adding -OrNull suffix to them", 0) \ - M(Bool, optimize_syntax_fuse_functions, false, "Fuse aggregate functions (`sum, avg, count` with identical arguments into one `sumCount`, quantile-family functions with the same argument into `quantiles*(...)[...]`)", 0) \ + M(Bool, optimize_syntax_fuse_functions, true, "Allow apply syntax optimisation: fuse aggregate functions", 0) \ + M(Bool, optimize_fuse_sum_count_avg, false, "Fuse functions `sum, avg, count` with identical arguments into one `sumCount` (`optimize_syntax_fuse_functions should be enabled)", 0) \ M(Bool, flatten_nested, true, "If true, columns of type Nested will be flatten to separate array columns instead of one array of tuples", 0) \ M(Bool, asterisk_include_materialized_columns, false, "Include MATERIALIZED columns for wildcard query", 0) \ M(Bool, asterisk_include_alias_columns, false, "Include ALIAS columns for wildcard query", 0) \ @@ -542,7 +543,6 @@ class IColumn; M(HandleKafkaErrorMode, handle_kafka_error_mode, HandleKafkaErrorMode::DEFAULT, "Obsolete setting, does nothing.", 0) \ M(Bool, database_replicated_ddl_output, true, "Obsolete setting, does nothing.", 0) \ M(UInt64, replication_alter_columns_timeout, 60, "Obsolete setting, does nothing.", 0) \ - M(Bool, optimize_fuse_sum_count_avg, false, "Obsolete, use optimize_syntax_fuse_functions", 0) \ M(UInt64, odbc_max_field_size, 0, "Obsolete setting, does nothing.", 0) \ /** The section above is for obsolete settings. Do not add anything there. */ diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index e0968b7fce4..571958a1f79 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1175,7 +1175,7 @@ void TreeRewriter::normalize( // if we have at least two different functions. E.g. we will replace sum(x) // and count(x) with sumCount(x).1 and sumCount(x).2, and sumCount() will // be calculated only once because of CSE. - if (settings.optimize_fuse_sum_count_avg || settings.optimize_syntax_fuse_functions) + if (settings.optimize_fuse_sum_count_avg && settings.optimize_syntax_fuse_functions) { FuseSumCountAggregatesVisitor::Data data; FuseSumCountAggregatesVisitor(data).visit(query); diff --git a/tests/queries/0_stateless/01744_fuse_sum_count_aggregate.sql b/tests/queries/0_stateless/01744_fuse_sum_count_aggregate.sql index 4648889ca27..5b6ed440ba4 100644 --- a/tests/queries/0_stateless/01744_fuse_sum_count_aggregate.sql +++ b/tests/queries/0_stateless/01744_fuse_sum_count_aggregate.sql @@ -3,6 +3,8 @@ CREATE TABLE fuse_tbl(a Int8, b Int8) Engine = Log; INSERT INTO fuse_tbl SELECT number, number + 1 FROM numbers(1, 20); SET optimize_syntax_fuse_functions = 1; +SET optimize_fuse_sum_count_avg = 1; + SELECT sum(a), sum(b), count(b) from fuse_tbl; EXPLAIN SYNTAX SELECT sum(a), sum(b), count(b) from fuse_tbl; SELECT '---------NOT trigger fuse--------';