From 4798ab19245f22622b16120523537e87c31fd273 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 2 Dec 2018 01:25:33 +0800 Subject: [PATCH 01/71] Use cluster connections pool in DirectoryMonitor --- .../Storages/Distributed/DirectoryMonitor.cpp | 36 +++++++++++++------ 1 file changed, 26 insertions(+), 10 deletions(-) diff --git a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp index 74821f432e7..7cff6705fa5 100644 --- a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp @@ -175,17 +175,33 @@ void StorageDistributedDirectoryMonitor::run() ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::string & name, const StorageDistributed & storage) { auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(storage.context.getSettingsRef()); - const auto pool_factory = [&storage, &timeouts] (const std::string & host, const UInt16 port, - const Protocol::Secure secure, - const std::string & user, const std::string & password, - const std::string & default_database) + const auto pool_factory = [&storage, &timeouts](const std::string & host, const UInt16 port, + const Protocol::Secure secure, + const std::string & user, const std::string & password, + const std::string & default_database) -> ConnectionPoolPtr { - return std::make_shared( - 1, host, port, default_database, - user, password, timeouts, - storage.getName() + '_' + user, - Protocol::Compression::Enable, - secure); + ClusterPtr cluster = storage.getCluster(); + const auto shards_info = cluster->getShardsInfo(); + const auto shards_addresses = cluster->getShardsAddresses(); + + /// existing connections pool have a higher priority + for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) + { + Cluster::Addresses replicas_addresses = shards_addresses[shard_index]; + + for (size_t replica_index = 0; replica_index < replicas_addresses.size(); ++replica_index) + { + Cluster::Address replica_address = replicas_addresses[replica_index]; + + if (replica_address.host_name == host && replica_address.port == port + && replica_address.secure == secure && replica_address.user == user + && replica_address.password == password && replica_address.default_database == default_database) + return shards_info[shard_index].per_replica_pools[replica_index]; + } + } + + return std::make_shared(1, host, port, default_database, user, password, timeouts, + storage.getName() + '_' + user, Protocol::Compression::Enable, secure); }; auto pools = createPoolsForAddresses(name, pool_factory); From 994d1c0fe04800dd8ab9bca0d8856c67b3b75fb6 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sun, 2 Dec 2018 10:17:08 +0800 Subject: [PATCH 02/71] fix excessive copy --- dbms/src/Interpreters/Cluster.cpp | 43 ++++++++++++- dbms/src/Interpreters/Cluster.h | 5 +- .../Storages/Distributed/DirectoryMonitor.cpp | 61 ++++--------------- .../Storages/System/StorageSystemClusters.cpp | 51 ++++++---------- 4 files changed, 78 insertions(+), 82 deletions(-) diff --git a/dbms/src/Interpreters/Cluster.cpp b/dbms/src/Interpreters/Cluster.cpp index 4bf446b107b..b562f986bee 100644 --- a/dbms/src/Interpreters/Cluster.cpp +++ b/dbms/src/Interpreters/Cluster.cpp @@ -122,6 +122,47 @@ String Cluster::Address::toStringFull() const + ((secure == Protocol::Secure::Enable) ? "+secure" : ""); } +void Cluster::Address::fromFullString(const String & full_string, Cluster::Address & address) +{ + const char * address_begin = full_string.data(); + const char * address_end = address_begin + full_string.size(); + + Protocol::Secure secure = Protocol::Secure::Disable; + const char * secure_tag = "+secure"; + if (endsWith(full_string, secure_tag)) + { + address_end -= strlen(secure_tag); + secure = Protocol::Secure::Enable; + } + + const char * user_pw_end = strchr(full_string.data(), '@'); + const char * colon = strchr(full_string.data(), ':'); + if (!user_pw_end || !colon) + throw Exception("Incorrect user[:password]@host:port#default_database format " + full_string, ErrorCodes::SYNTAX_ERROR); + + const bool has_pw = colon < user_pw_end; + const char * host_end = has_pw ? strchr(user_pw_end + 1, ':') : colon; + if (!host_end) + throw Exception("Incorrect address '" + full_string + "', it does not contain port", ErrorCodes::SYNTAX_ERROR); + + const char * has_db = strchr(full_string.data(), '#'); + const char * port_end = has_db ? has_db : address_end; + + address.secure = secure; + address.port = parse(host_end + 1, port_end - (host_end + 1)); + address.host_name = unescapeForFileName(std::string(user_pw_end + 1, host_end)); + address.user = unescapeForFileName(std::string(address_begin, has_pw ? colon : user_pw_end)); + address.password = has_pw ? unescapeForFileName(std::string(colon + 1, user_pw_end)) : std::string(); + address.default_database = has_db ? unescapeForFileName(std::string(has_db + 1, address_end)) : std::string(); +} + +bool Cluster::Address::operator==(const Cluster::Address & other) const +{ + return other.host_name == host_name && other.port == port + && other.secure == secure && other.user == user + && other.password == password && other.default_database == default_database; +} + /// Implementation of Clusters class @@ -198,7 +239,6 @@ Cluster::Cluster(Poco::Util::AbstractConfiguration & config, const Settings & se const auto weight = config.getInt(prefix + ".weight", default_weight); addresses.emplace_back(config, prefix); - addresses.back().replica_num = 1; const auto & address = addresses.back(); ShardInfo info; @@ -253,7 +293,6 @@ Cluster::Cluster(Poco::Util::AbstractConfiguration & config, const Settings & se if (startsWith(replica_key, "replica")) { replica_addresses.emplace_back(config, partial_prefix + replica_key); - replica_addresses.back().replica_num = current_replica_num; ++current_replica_num; if (!replica_addresses.back().is_local) diff --git a/dbms/src/Interpreters/Cluster.h b/dbms/src/Interpreters/Cluster.h index 8bfbc073c61..4338447dbd7 100644 --- a/dbms/src/Interpreters/Cluster.h +++ b/dbms/src/Interpreters/Cluster.h @@ -59,7 +59,6 @@ public: String password; /// This database is selected when no database is specified for Distributed table String default_database; - UInt32 replica_num; /// The locality is determined at the initialization, and is not changed even if DNS is changed bool is_local; Protocol::Compression compression = Protocol::Compression::Enable; @@ -82,12 +81,16 @@ public: /// Retrurns escaped user:password@resolved_host_address:resolved_host_port#default_database String toStringFull() const; + static void fromFullString(const String & address_full_string, Address & address); + /// Returns initially resolved address Poco::Net::SocketAddress getResolvedAddress() const { return initially_resolved_address; } + bool operator==(const Address & other) const; + private: Poco::Net::SocketAddress initially_resolved_address; }; diff --git a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp index 7cff6705fa5..9ef4e012d8f 100644 --- a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp @@ -48,40 +48,9 @@ namespace for (auto it = boost::make_split_iterator(name, boost::first_finder(",")); it != decltype(it){}; ++it) { - const auto address = boost::copy_range(*it); - const char * address_begin = static_cast(address.data()); - const char * address_end = address_begin + address.size(); - - Protocol::Secure secure = Protocol::Secure::Disable; - const char * secure_tag = "+secure"; - if (endsWith(address, secure_tag)) - { - address_end -= strlen(secure_tag); - secure = Protocol::Secure::Enable; - } - - const char * user_pw_end = strchr(address.data(), '@'); - const char * colon = strchr(address.data(), ':'); - if (!user_pw_end || !colon) - throw Exception{"Shard address '" + address + "' does not match to 'user[:password]@host:port#default_database' pattern", - ErrorCodes::INCORRECT_FILE_NAME}; - - const bool has_pw = colon < user_pw_end; - const char * host_end = has_pw ? strchr(user_pw_end + 1, ':') : colon; - if (!host_end) - throw Exception{"Shard address '" + address + "' does not contain port", ErrorCodes::INCORRECT_FILE_NAME}; - - const char * has_db = strchr(address.data(), '#'); - const char * port_end = has_db ? has_db : address_end; - - const auto user = unescapeForFileName(std::string(address_begin, has_pw ? colon : user_pw_end)); - const auto password = has_pw ? unescapeForFileName(std::string(colon + 1, user_pw_end)) : std::string(); - const auto host = unescapeForFileName(std::string(user_pw_end + 1, host_end)); - const auto port = parse(host_end + 1, port_end - (host_end + 1)); - const auto database = has_db ? unescapeForFileName(std::string(has_db + 1, address_end)) - : std::string(); - - pools.emplace_back(factory(host, port, secure, user, password, database)); + Cluster::Address address; + Cluster::Address::fromFullString(boost::copy_range(*it), address); + pools.emplace_back(factory(address)); } return pools; @@ -175,33 +144,29 @@ void StorageDistributedDirectoryMonitor::run() ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::string & name, const StorageDistributed & storage) { auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(storage.context.getSettingsRef()); - const auto pool_factory = [&storage, &timeouts](const std::string & host, const UInt16 port, - const Protocol::Secure secure, - const std::string & user, const std::string & password, - const std::string & default_database) -> ConnectionPoolPtr + const auto pool_factory = [&storage, &timeouts](const Cluster::Address & address) -> ConnectionPoolPtr { - ClusterPtr cluster = storage.getCluster(); - const auto shards_info = cluster->getShardsInfo(); - const auto shards_addresses = cluster->getShardsAddresses(); + const auto & cluster = storage.getCluster(); + const auto & shards_info = cluster->getShardsInfo(); + const auto & shards_addresses = cluster->getShardsAddresses(); /// existing connections pool have a higher priority for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) { - Cluster::Addresses replicas_addresses = shards_addresses[shard_index]; + const Cluster::Addresses & replicas_addresses = shards_addresses[shard_index]; for (size_t replica_index = 0; replica_index < replicas_addresses.size(); ++replica_index) { - Cluster::Address replica_address = replicas_addresses[replica_index]; + const Cluster::Address & replica_address = replicas_addresses[replica_index]; - if (replica_address.host_name == host && replica_address.port == port - && replica_address.secure == secure && replica_address.user == user - && replica_address.password == password && replica_address.default_database == default_database) + if (address == replica_address) return shards_info[shard_index].per_replica_pools[replica_index]; } } - return std::make_shared(1, host, port, default_database, user, password, timeouts, - storage.getName() + '_' + user, Protocol::Compression::Enable, secure); + return std::make_shared( + 1, address.host_name, address.port, address.default_database, address.user, address.password, timeouts, + storage.getName() + '_' + address.user, Protocol::Compression::Enable, address.secure); }; auto pools = createPoolsForAddresses(name, pool_factory); diff --git a/dbms/src/Storages/System/StorageSystemClusters.cpp b/dbms/src/Storages/System/StorageSystemClusters.cpp index b0ad56e8eb5..b33b2d86d0e 100644 --- a/dbms/src/Storages/System/StorageSystemClusters.cpp +++ b/dbms/src/Storages/System/StorageSystemClusters.cpp @@ -26,44 +26,33 @@ NamesAndTypesList StorageSystemClusters::getNamesAndTypes() void StorageSystemClusters::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { - auto updateColumns = [&](const std::string & cluster_name, const Cluster::ShardInfo & shard_info, const Cluster::Address & address) + for (const auto & name_and_cluster : context.getClusters().getContainer()) { - size_t i = 0; - res_columns[i++]->insert(cluster_name); - res_columns[i++]->insert(shard_info.shard_num); - res_columns[i++]->insert(shard_info.weight); - res_columns[i++]->insert(address.replica_num); - res_columns[i++]->insert(address.host_name); - res_columns[i++]->insert(DNSResolver::instance().resolveHost(address.host_name).toString()); - res_columns[i++]->insert(address.port); - res_columns[i++]->insert(shard_info.isLocal()); - res_columns[i++]->insert(address.user); - res_columns[i++]->insert(address.default_database); - }; - - auto clusters = context.getClusters().getContainer(); - for (const auto & entry : clusters) - { - const std::string cluster_name = entry.first; - const ClusterPtr cluster = entry.second; - const auto & addresses_with_failover = cluster->getShardsAddresses(); + const String & cluster_name = name_and_cluster.first; + const ClusterPtr & cluster = name_and_cluster.second; const auto & shards_info = cluster->getShardsInfo(); + const auto & addresses_with_failover = cluster->getShardsAddresses(); - if (!addresses_with_failover.empty()) + for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index) { - auto it1 = addresses_with_failover.cbegin(); - auto it2 = shards_info.cbegin(); + const auto & shard_info = shards_info[shard_index]; + const auto & shard_addresses = addresses_with_failover[shard_index]; - while (it1 != addresses_with_failover.cend()) + for (size_t replica_index = 0; replica_index < shard_addresses.size(); ++replica_index) { - const auto & addresses = *it1; - const auto & shard_info = *it2; + size_t i = 0; + const auto & address = shard_addresses[replica_index]; - for (const auto & address : addresses) - updateColumns(cluster_name, shard_info, address); - - ++it1; - ++it2; + res_columns[i++]->insert(cluster_name); + res_columns[i++]->insert(shard_info.shard_num); + res_columns[i++]->insert(shard_info.weight); + res_columns[i++]->insert(replica_index + 1); + res_columns[i++]->insert(address.host_name); + res_columns[i++]->insert(DNSResolver::instance().resolveHost(address.host_name).toString()); + res_columns[i++]->insert(address.port); + res_columns[i++]->insert(shard_info.isLocal()); + res_columns[i++]->insert(address.user); + res_columns[i++]->insert(address.default_database); } } } From d734215bc237265ec28971bb5228536e0981d6bf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 30 Dec 2018 01:11:00 +0300 Subject: [PATCH 03/71] Adapted tests for open test data [#CLICKHOUSE-3] --- .../1_stateful/00004_top_counters.reference | 20 +- .../1_stateful/00006_agregates.reference | 2 +- .../queries/1_stateful/00007_uniq.reference | 20 +- dbms/tests/queries/1_stateful/00007_uniq.sql | 2 +- .../queries/1_stateful/00008_uniq.reference | 2 +- dbms/tests/queries/1_stateful/00008_uniq.sql | 2 +- .../00009_uniq_distributed.reference | 2 +- .../1_stateful/00009_uniq_distributed.sql | 2 +- .../00010_quantiles_segfault.reference | 4 +- .../1_stateful/00010_quantiles_segfault.sql | 2 +- .../00013_sorting_of_nested.reference | 20 +- .../00014_filtering_arrays.reference | 20 +- ...otals_and_no_aggregate_functions.reference | 30 +- ...aggregation_uninitialized_memory.reference | 61 +- ...00017_aggregation_uninitialized_memory.sql | 2 +- .../1_stateful/00021_1_select_with_in.sql | 2 +- .../1_stateful/00021_2_select_with_in.sql | 2 +- .../00021_3_select_with_in.reference | 2 +- .../1_stateful/00021_3_select_with_in.sql | 2 +- .../1_stateful/00022_merge_prewhere.reference | 4 +- .../1_stateful/00023_totals_limit.reference | 6 +- .../queries/1_stateful/00023_totals_limit.sql | 2 +- .../00024_random_counters.reference | 1944 ++++++++-------- .../1_stateful/00024_random_counters.sql | 1992 +++++++++-------- .../00030_array_enumerate_uniq.reference | 2 +- .../00031_array_enumerate_uniq.reference | 40 +- .../1_stateful/00031_array_enumerate_uniq.sql | 4 +- .../00032_aggregate_key64.reference | 16 +- .../00033_aggregate_key_string.reference | 18 +- ...00034_aggregate_key_fixed_string.reference | 18 +- .../00035_aggregate_keys128.reference | 18 +- .../00036_aggregate_hashed.reference | 18 +- .../00037_uniq_state_merge1.reference | 200 +- .../00038_uniq_state_merge2.reference | 200 +- ...40_aggregating_materialized_view.reference | 30 +- .../00040_aggregating_materialized_view.sql | 4 +- ...41_aggregating_materialized_view.reference | 30 +- .../00041_aggregating_materialized_view.sql | 4 +- .../1_stateful/00042_any_left_join.reference | 17 +- .../1_stateful/00042_any_left_join.sql | 12 +- .../1_stateful/00043_any_left_join.reference | 17 +- .../1_stateful/00043_any_left_join.sql | 10 +- .../00044_any_left_join_string.reference | 20 +- .../1_stateful/00044_any_left_join_string.sql | 2 - .../queries/1_stateful/00047_bar.reference | 200 +- .../1_stateful/00049_max_string_if.reference | 40 +- .../1_stateful/00050_min_max.reference | 40 +- .../1_stateful/00051_min_max_array.reference | 40 +- .../1_stateful/00052_group_by_in.reference | 28 +- .../queries/1_stateful/00052_group_by_in.sql | 2 +- .../00054_merge_tree_partitions.reference | 22 +- .../00054_merge_tree_partitions.sql | 4 +- .../queries/1_stateful/00056_view.reference | 40 +- ...00059_merge_sorting_empty_array_joined.sql | 2 +- .../00060_move_to_prewhere_and_sets.reference | 2 +- .../1_stateful/00061_storage_buffer.reference | 6 +- .../1_stateful/00061_storage_buffer.sql | 2 +- .../1_stateful/00062_loyalty.reference | 24 +- .../1_stateful/00063_loyalty_joins.reference | 96 +- .../00065_loyalty_with_storage_join.reference | 48 +- .../1_stateful/00067_union_all.reference | 40 +- .../00068_subquery_in_prewhere.reference | 2 +- .../1_stateful/00068_subquery_in_prewhere.sql | 2 +- ...00069_duplicate_aggregation_keys.reference | 274 ++- .../00069_duplicate_aggregation_keys.sql | 2 +- .../00071_merge_tree_optimize_aio.reference | 200 +- ...72_compare_date_and_string_index.reference | 26 +- .../1_stateful/00073_uniq_array.reference | 14 +- .../queries/1_stateful/00073_uniq_array.sql | 2 +- .../1_stateful/00074_full_join.reference | 120 +- .../queries/1_stateful/00074_full_join.sql | 16 +- .../00075_left_array_join.reference | 400 ++-- .../1_stateful/00075_left_array_join.sql | 4 +- .../00077_log_tinylog_stripelog.reference | 8 +- .../00078_group_by_arrays.reference | 68 +- ...rray_join_not_used_joined_column.reference | 172 +- ...0079_array_join_not_used_joined_column.sql | 4 +- .../1_stateful/00080_array_join_and_union.sql | 2 +- ..._group_by_without_key_and_totals.reference | 92 +- .../00081_group_by_without_key_and_totals.sql | 24 +- .../1_stateful/00082_quantiles.reference | 160 +- .../00084_external_aggregation.reference | 40 +- .../queries/1_stateful/00087_where_0.sql | 2 +- ..._functions_with_non_constant_arg.reference | 10 +- ...sition_functions_with_non_constant_arg.sql | 6 +- .../queries/1_stateful/00139_like.reference | 6 +- .../queries/1_stateful/00140_rename.reference | 24 +- .../tests/queries/1_stateful/00140_rename.sql | 24 +- .../1_stateful/00141_transform.reference | 4 +- ...0143_transform_non_const_default.reference | 13 +- ..._functions_of_aggregation_states.reference | 10 +- ...5_aggregate_functions_statistics.reference | 14 +- .../00146_aggregate_function_uniq.reference | 38 +- ...48_monotonic_functions_and_index.reference | 60 +- .../00148_monotonic_functions_and_index.sql | 64 +- ...149_quantiles_timing_distributed.reference | 2 +- ...00150_quantiles_timing_precision.reference | 2 +- 97 files changed, 3737 insertions(+), 3637 deletions(-) diff --git a/dbms/tests/queries/1_stateful/00004_top_counters.reference b/dbms/tests/queries/1_stateful/00004_top_counters.reference index 5667f0332bd..cf2824e45b0 100644 --- a/dbms/tests/queries/1_stateful/00004_top_counters.reference +++ b/dbms/tests/queries/1_stateful/00004_top_counters.reference @@ -1,10 +1,10 @@ -1143050 523264 -731962 475698 -722545 337212 -722889 252197 -2237260 196036 -23057320 147211 -722818 90109 -48221 85379 -19762435 77807 -722884 77492 +1704509 523264 +732797 475698 +598875 337212 +792887 252197 +3807842 196036 +25703952 147211 +716829 90109 +59183 85379 +33010362 77807 +800784 77492 diff --git a/dbms/tests/queries/1_stateful/00006_agregates.reference b/dbms/tests/queries/1_stateful/00006_agregates.reference index 9e49d438c66..6652e97b0cb 100644 --- a/dbms/tests/queries/1_stateful/00006_agregates.reference +++ b/dbms/tests/queries/1_stateful/00006_agregates.reference @@ -1 +1 @@ -281146 8873898 1289.4475214837944 +329039 8873898 1400.8565027454677 diff --git a/dbms/tests/queries/1_stateful/00007_uniq.reference b/dbms/tests/queries/1_stateful/00007_uniq.reference index 10e29f1e3fd..58db3262f53 100644 --- a/dbms/tests/queries/1_stateful/00007_uniq.reference +++ b/dbms/tests/queries/1_stateful/00007_uniq.reference @@ -1,10 +1,10 @@ -213 28 -2 10 -225 7 -17 2 -56 2 -187 2 -4 1 -9 1 -15 1 -16 1 +196 559 +3 161 +241 147 +207 106 +225 62 +1 60 +46 59 +36 58 +104 58 +47 57 diff --git a/dbms/tests/queries/1_stateful/00007_uniq.sql b/dbms/tests/queries/1_stateful/00007_uniq.sql index b602ea63b6e..bc36a2b69e6 100644 --- a/dbms/tests/queries/1_stateful/00007_uniq.sql +++ b/dbms/tests/queries/1_stateful/00007_uniq.sql @@ -1 +1 @@ -SELECT RegionID, uniq(UserID) AS u FROM test.hits WHERE CounterID = 34 GROUP BY RegionID ORDER BY u DESC, RegionID LIMIT 10 +SELECT RegionID, uniq(UserID) AS u FROM test.hits WHERE CounterID = 800784 GROUP BY RegionID ORDER BY u DESC, RegionID LIMIT 10 diff --git a/dbms/tests/queries/1_stateful/00008_uniq.reference b/dbms/tests/queries/1_stateful/00008_uniq.reference index e101845cecc..ab0a024d825 100644 --- a/dbms/tests/queries/1_stateful/00008_uniq.reference +++ b/dbms/tests/queries/1_stateful/00008_uniq.reference @@ -1 +1 @@ -119497 2867 5595 +120665 4057 106 diff --git a/dbms/tests/queries/1_stateful/00008_uniq.sql b/dbms/tests/queries/1_stateful/00008_uniq.sql index a80c3b85812..72d91736ff7 100644 --- a/dbms/tests/queries/1_stateful/00008_uniq.sql +++ b/dbms/tests/queries/1_stateful/00008_uniq.sql @@ -1 +1 @@ -SELECT uniq(UserID), uniqIf(UserID, CounterID = 1143050), uniqIf(FUniqID, RegionID = 213) FROM test.hits +SELECT uniq(UserID), uniqIf(UserID, CounterID = 800784), uniqIf(FUniqID, RegionID = 213) FROM test.hits diff --git a/dbms/tests/queries/1_stateful/00009_uniq_distributed.reference b/dbms/tests/queries/1_stateful/00009_uniq_distributed.reference index e101845cecc..ab0a024d825 100644 --- a/dbms/tests/queries/1_stateful/00009_uniq_distributed.reference +++ b/dbms/tests/queries/1_stateful/00009_uniq_distributed.reference @@ -1 +1 @@ -119497 2867 5595 +120665 4057 106 diff --git a/dbms/tests/queries/1_stateful/00009_uniq_distributed.sql b/dbms/tests/queries/1_stateful/00009_uniq_distributed.sql index b2b99d36d2f..71d90efbccc 100644 --- a/dbms/tests/queries/1_stateful/00009_uniq_distributed.sql +++ b/dbms/tests/queries/1_stateful/00009_uniq_distributed.sql @@ -1 +1 @@ -SELECT uniq(UserID), uniqIf(UserID, CounterID = 1143050), uniqIf(FUniqID, RegionID = 213) FROM remote('127.0.0.{1,2}', test, hits) +SELECT uniq(UserID), uniqIf(UserID, CounterID = 800784), uniqIf(FUniqID, RegionID = 213) FROM remote('127.0.0.{1,2}', test, hits) diff --git a/dbms/tests/queries/1_stateful/00010_quantiles_segfault.reference b/dbms/tests/queries/1_stateful/00010_quantiles_segfault.reference index ae9aba98e8d..5192426eea6 100644 --- a/dbms/tests/queries/1_stateful/00010_quantiles_segfault.reference +++ b/dbms/tests/queries/1_stateful/00010_quantiles_segfault.reference @@ -1,3 +1,3 @@ -http://metrika.yandex.ru/list/ 744 [1995,3123,9638] +http://publicdarokura 1632 [1464,4466,13512] - 3762 [968,1789,9638] + 58032 [1457,3828,14562] diff --git a/dbms/tests/queries/1_stateful/00010_quantiles_segfault.sql b/dbms/tests/queries/1_stateful/00010_quantiles_segfault.sql index 952159c8996..5954ba5e3ee 100644 --- a/dbms/tests/queries/1_stateful/00010_quantiles_segfault.sql +++ b/dbms/tests/queries/1_stateful/00010_quantiles_segfault.sql @@ -1 +1 @@ -SELECT URL AS `ym:ah:URL`, sum((NOT DontCountHits AND NOT Refresh)), quantilesTimingIf(0.1, 0.5, 0.9)((DOMCompleteTiming + LoadEventEndTiming), DOMCompleteTiming != -1 AND LoadEventEndTiming != -1) as t FROM remote('127.0.0.{1,2}', test, hits) WHERE (CounterID = 101500) AND (((DontCountHits = 0) OR (IsNotBounce = 1)) AND (URL != '')) GROUP BY `ym:ah:URL` WITH TOTALS HAVING (sum((NOT DontCountHits AND NOT Refresh)) > 0) AND (count() > 0) ORDER BY sum((NOT DontCountHits AND NOT Refresh)) DESC, URL LIMIT 0, 1 +SELECT URL AS `ym:ah:URL`, sum((NOT DontCountHits AND NOT Refresh)), quantilesTimingIf(0.1, 0.5, 0.9)((DOMCompleteTiming + LoadEventEndTiming), DOMCompleteTiming != -1 AND LoadEventEndTiming != -1) as t FROM remote('127.0.0.{1,2}', test, hits) WHERE (CounterID = 800784) AND (((DontCountHits = 0) OR (IsNotBounce = 1)) AND (URL != '')) GROUP BY `ym:ah:URL` WITH TOTALS HAVING (sum((NOT DontCountHits AND NOT Refresh)) > 0) AND (count() > 0) ORDER BY sum((NOT DontCountHits AND NOT Refresh)) DESC, URL LIMIT 0, 1 diff --git a/dbms/tests/queries/1_stateful/00013_sorting_of_nested.reference b/dbms/tests/queries/1_stateful/00013_sorting_of_nested.reference index cb1cd2eed44..7ac3eca5a46 100644 --- a/dbms/tests/queries/1_stateful/00013_sorting_of_nested.reference +++ b/dbms/tests/queries/1_stateful/00013_sorting_of_nested.reference @@ -1,10 +1,10 @@ -['city','place'] -['city','place'] -['city','place'] -['city','place'] -['region','cat','region','cat','region','cat'] -['region','cat'] -['Логин'] -['region','cat'] -['region','cat'] -['region','cat'] +['Кнопка'] +['pageParams','pageParams'] +['pageParams'] +['gen_time'] +['pageParams'] +['pageParams'] +['pageParams'] +['pageParams'] +['Прав','gen_timestamp','Прав','Действи','affili','gen_timestamp','Эксперимент про отрыв счетчика у папок','Просмотр писем'] +['Марка','gen_time','Марка'] diff --git a/dbms/tests/queries/1_stateful/00014_filtering_arrays.reference b/dbms/tests/queries/1_stateful/00014_filtering_arrays.reference index d431d8935c1..5737513c802 100644 --- a/dbms/tests/queries/1_stateful/00014_filtering_arrays.reference +++ b/dbms/tests/queries/1_stateful/00014_filtering_arrays.reference @@ -1,10 +1,10 @@ -[3691,2864,3579,99,3201,118,11,461,459,88] -[3579,3281,37,3,118,2,11,8,600,38,9,2333] -[3579,3281,37,3,118,2,11,8,600,38,9,2333] -[3579,3281,37,3,118,2,11,8,600,38,9,2333] -[3579,3281,37,3,118,2,11,8,600,38,9,2333] -[3579,3281,37,3,118,2,11,8,600,38,9,2333] -[3579,3281,37,3,118,2,11,8,600,38,9,2333] -[3579,3281,37,3,118,2,11,8,600,38,9,2333] -[3579,2865,9,332,88,102,437,95,448,98,89] -[3579,2865,9,332,88,102,437,95,448,98,89] +[4039,76,119,2179,272,6,289,834,7,195] +[4039,76,119,2179,272,6,289,834,7,195] +[4039,76,119,2179,272,6,289,834,7,195] +[3993,3292,2,27,28,19,3981,10,70,112,1,22,14] +[3993,3292,2,27,28,19,3981,10,70,112,1,22,14] +[3993,1019,2817,2,1,22,9,51,3,1465,72,42] +[3993,1019,2817,2,1,22,9,51,3,1465,72,42] +[3993,1019,2817,2,1,22,9,51,3,1465,72,42] +[3993,1019,2817,2,1,22,9,51,3,1465,72,42] +[3993,925,2,72,76,14] diff --git a/dbms/tests/queries/1_stateful/00015_totals_and_no_aggregate_functions.reference b/dbms/tests/queries/1_stateful/00015_totals_and_no_aggregate_functions.reference index 78c0c2298d3..fd7e1808831 100644 --- a/dbms/tests/queries/1_stateful/00015_totals_and_no_aggregate_functions.reference +++ b/dbms/tests/queries/1_stateful/00015_totals_and_no_aggregate_functions.reference @@ -1,27 +1,27 @@ 0 2 3 -7 -14 -15 -17 +4 +10 +12 +16 18 -19 -26 -27 -33 -34 +22 +24 +30 +32 35 -36 -37 -38 -39 40 41 -46 -47 +42 48 49 50 +51 +52 +55 +58 +61 +62 0 diff --git a/dbms/tests/queries/1_stateful/00017_aggregation_uninitialized_memory.reference b/dbms/tests/queries/1_stateful/00017_aggregation_uninitialized_memory.reference index 838d0e2d7ae..20f3be1c45f 100644 --- a/dbms/tests/queries/1_stateful/00017_aggregation_uninitialized_memory.reference +++ b/dbms/tests/queries/1_stateful/00017_aggregation_uninitialized_memory.reference @@ -1,47 +1,14 @@ -goal://metrika.yandex.ru/add_counter_submit x -http://aist5.ru/ x -http://avg.nation.com/ x -http://avito.ru/ x -http://baraholka.leprosorium.ru/ x -http://bonum.spb.ru/ x -http://clck.yandex.ru/ x -http://clubs.ya.ru/ x -http://direct.yandex.ru/ x -http://facebook.com/ x -http://go.mail.ru/ x -http://google.com.kw/ x -http://google.com/ x -http://google.ru/ x -http://help.yandex.ru/ x -http://hghltd.yandex.net/ x -http://i.yandex.ru/ x -http://krdshop.ru/ x -http://lavkafreida.ru/ x -http://link.2gis.ru/ x -http://m.avito.ru/ x -http://m.vk.com/ x -http://market-preview.yandex.ru/ x -http://market.yandex.ru/ x -http://metrika.yandex.ru/ x -http://metrika.yandex.ua/ x -http://music.yandex.ru/ x -http://news.mail.ru/ x -http://news.yandex.ru/ x -http://partners.yandex.ru/ x -http://passport.yandex.ru/ x -http://postoffice.yandex.ru/ x -http://profkosmetika.net/ x -http://semalt.com/ x -http://shops.pp.ru/ x -http://site.yandex.ru/ x -http://stadium.mvc/ x -http://topas-eco.ru/ x -http://translate.googleusercontent.com/ x -http://virage24.ru/ x -http://vk.com/ x -http://webmaster.yandex.ru/ x -http://yaca.yandex.ru/ x -http://yandex.kz/ x -http://yandex.ru/ x -http://yandex.ua/ x -http://у-полины.рф/ x +https://mamba.ru%2Fitem%2Ftitle,pos,p0,source x +https://mamba.ru%2Fitem%2Ftitle,pos,p1,source x +https://mamba.ru%2Fitem%2Ftitle,pos,p1,source=next=269.html x +https://mamba.ru%2Fitem%2Ftitle,pos,p18,source x +https://mamba.ru%2Fitem%2Ftitle,pos,p2,source x +https://mamba.ru%2Fitem%2Ftitle,pos,p3,source x +https://mamba.ru%2Fitem%2Ftitle,pos,p4,source x +https://mamba.ru%2Fitem%2Ftitle,pos,p8,source x +https://mamba.ru%2Fitem%2Ftitle_servie-ved x +https://mamba.ru%2Fitem%2Ftitled-loratov/ x +https://mamba.ru%2Fitem%2Ftitleri/ x +https://mamba.ru%2Fitem%2Ftitles? x +https://mamba.ru%2Fitem%2Ftitles_with-bbcbb23fb7e00f1 x +https://mamba.ru/ x diff --git a/dbms/tests/queries/1_stateful/00017_aggregation_uninitialized_memory.sql b/dbms/tests/queries/1_stateful/00017_aggregation_uninitialized_memory.sql index c7cecc1e7df..e7fb507e6c5 100644 --- a/dbms/tests/queries/1_stateful/00017_aggregation_uninitialized_memory.sql +++ b/dbms/tests/queries/1_stateful/00017_aggregation_uninitialized_memory.sql @@ -1,2 +1,2 @@ -SELECT DISTINCT (URLHierarchy(URL)[1]) AS q, 'x' AS w FROM test.hits WHERE CounterID = 101500 ORDER BY URL +SELECT DISTINCT (URLHierarchy(URL)[1]) AS q, 'x' AS w FROM test.hits WHERE CounterID = 14917930 ORDER BY URL diff --git a/dbms/tests/queries/1_stateful/00021_1_select_with_in.sql b/dbms/tests/queries/1_stateful/00021_1_select_with_in.sql index 2293985b1de..d0143734bf6 100644 --- a/dbms/tests/queries/1_stateful/00021_1_select_with_in.sql +++ b/dbms/tests/queries/1_stateful/00021_1_select_with_in.sql @@ -1 +1 @@ -select sum(Sign) from test.visits where CounterID in (722545); +select sum(Sign) from test.visits where CounterID in (942285); diff --git a/dbms/tests/queries/1_stateful/00021_2_select_with_in.sql b/dbms/tests/queries/1_stateful/00021_2_select_with_in.sql index b4005b8f24a..414184f3038 100644 --- a/dbms/tests/queries/1_stateful/00021_2_select_with_in.sql +++ b/dbms/tests/queries/1_stateful/00021_2_select_with_in.sql @@ -1 +1 @@ -select sum(Sign) from test.visits where CounterID in (722545, 731962); +select sum(Sign) from test.visits where CounterID in (942285, 577322); diff --git a/dbms/tests/queries/1_stateful/00021_3_select_with_in.reference b/dbms/tests/queries/1_stateful/00021_3_select_with_in.reference index 8dbdb2f57fd..6e33c08a07a 100644 --- a/dbms/tests/queries/1_stateful/00021_3_select_with_in.reference +++ b/dbms/tests/queries/1_stateful/00021_3_select_with_in.reference @@ -1,2 +1,2 @@ 1 -2417 +337212 diff --git a/dbms/tests/queries/1_stateful/00021_3_select_with_in.sql b/dbms/tests/queries/1_stateful/00021_3_select_with_in.sql index 9c473ca71a1..edbb7f481d7 100644 --- a/dbms/tests/queries/1_stateful/00021_3_select_with_in.sql +++ b/dbms/tests/queries/1_stateful/00021_3_select_with_in.sql @@ -1,3 +1,3 @@ select 1 IN (1, 2, 3); -SELECT count() FROM remote('localhost', test, hits) WHERE CounterID IN (101500); +SELECT count() FROM remote('localhost', test, hits) WHERE CounterID IN (598875); diff --git a/dbms/tests/queries/1_stateful/00022_merge_prewhere.reference b/dbms/tests/queries/1_stateful/00022_merge_prewhere.reference index 065392a8670..0516be1fbcf 100644 --- a/dbms/tests/queries/1_stateful/00022_merge_prewhere.reference +++ b/dbms/tests/queries/1_stateful/00022_merge_prewhere.reference @@ -1,2 +1,2 @@ -22948 -22948 +187 +187 diff --git a/dbms/tests/queries/1_stateful/00023_totals_limit.reference b/dbms/tests/queries/1_stateful/00023_totals_limit.reference index 0be276a358c..fc4a02662d7 100644 --- a/dbms/tests/queries/1_stateful/00023_totals_limit.reference +++ b/dbms/tests/queries/1_stateful/00023_totals_limit.reference @@ -13,12 +13,12 @@ "data": [ - [3176497, "3"] + [1604017, "189"] ], - "totals": [0,"3"], + "totals": [0,"4652"], "rows": 1, - "rows_before_limit_at_least": 2 + "rows_before_limit_at_least": 72 } diff --git a/dbms/tests/queries/1_stateful/00023_totals_limit.sql b/dbms/tests/queries/1_stateful/00023_totals_limit.sql index 849b39b2782..2f4ad37ca39 100644 --- a/dbms/tests/queries/1_stateful/00023_totals_limit.sql +++ b/dbms/tests/queries/1_stateful/00023_totals_limit.sql @@ -1,2 +1,2 @@ SET output_format_write_statistics = 0; -SELECT goals_alias.ID AS `ym:s:goalDimension`, uniqIf(UserID, (UserID != 0) AND (`_uniq_Goals` = 1)) FROM test.visits ARRAY JOIN Goals AS goals_alias, arrayEnumerateUniq(Goals.ID) AS `_uniq_Goals` WHERE (CounterID = 101024) GROUP BY `ym:s:goalDimension` WITH TOTALS ORDER BY `ym:s:goalDimension` LIMIT 0, 1 FORMAT JSONCompact; +SELECT goals_alias.ID AS `ym:s:goalDimension`, uniqIf(UserID, (UserID != 0) AND (`_uniq_Goals` = 1)) FROM test.visits ARRAY JOIN Goals AS goals_alias, arrayEnumerateUniq(Goals.ID) AS `_uniq_Goals` WHERE (CounterID = 842440) GROUP BY `ym:s:goalDimension` WITH TOTALS ORDER BY `ym:s:goalDimension` LIMIT 0, 1 FORMAT JSONCompact; diff --git a/dbms/tests/queries/1_stateful/00024_random_counters.reference b/dbms/tests/queries/1_stateful/00024_random_counters.reference index 799513a7112..3b181f11e2a 100644 --- a/dbms/tests/queries/1_stateful/00024_random_counters.reference +++ b/dbms/tests/queries/1_stateful/00024_random_counters.reference @@ -1,992 +1,1000 @@ -1 3 5 51 -5 5 6 37 -2 2 2 30 -1 1 7 207 -5 5 21 1049 -3 4 13 598 -2 3 5 1059 -4 5 11 460 -3 11 11 30 -2 3 34 1930 +1 1 2 35 1 1 1 0 -52 192 268 34081 1 1 1 0 -4 5 12 2256 -2 2 3 176 -3 4 29 2078 -1 1 1 0 -3 5 12 566 -1 1 1 15 -1 2 2 48 -1 1 30 2054 -1 1 1 15 -255 406 596 42463 -2 2 2 32 -1 1 1 15 -1 1 2 7 -1 1 7 146 -1 1 1 27 -1 1 4 373 -1 1 1 20 -1 2 2 0 -1 1 1 0 -1 1 1 0 -1 1 4 31 -1 2 2 0 -1 1 2 204 -1 1 5 212 -5 5 6 233 -1 1 1 0 -1 1 1 16 -1 1 2 1 -1 2 6 40 -2 2 2 18 -1 1 1 17 -1 1 1 15 -2 2 2 16 -1 1 1 15 -1 1 1 15 -1 1 1 0 -15 34 83 2850 -1 1 1 15 -4 4 19 1806 -1 1 4 177 -41 62 119 12064 -1 1 15 1522 -1 1 1 0 -1 1 1 14 -2 3 10 665 -3 4 5 62 -5 6 35 1790 -1 1 5 33 -1 1 1 16 -1 3 5 36 -1 1 1 15 -2 2 4 407 -25 56 138 7687 -4 5 8 2880 -1 1 1 15 -1 1 1 0 -1 1 1 136 -2 2 2 0 -3 3 6 94 -1 1 1 15 -4 4 5 86 -1 1 1 0 -2 10 21 884 -11 11 11 0 -1 1 1 15 -10 12 17 242 -1 1 2 9 -1 1 1 15 -1 1 10 198 -1 1 5 514 -2 3 4 170 -1 1 1 0 -2 2 3 64 -6 6 19 256 -1 1 1 0 -3 3 3 0 -1 2 12 742 -1 1 1 15 -1 1 2 7 -1 1 1 15 -2 2 19 213 -1 2 8 610 -1 1 1 15 -1 1 5 159 -1 1 1 0 -1 1 1 0 -2 2 2 23 -6 7 16 3197 -1 1 7 421 -1 1 1 0 -1 1 6 101 -1 1 1 15 -2 6 6 0 -3 3 4 92 -1 1 1 0 -2 2 2 0 -1 1 1 0 -1 1 1 0 -1 1 1 16 -3 3 5 364 -21 26 29 1670 -29 57 137 7881 -1 1 2 87 -1 1 1 0 -4 4 4 0 -5 8 10 1376 -2 2 3 9 -1 1 1 15 -1 1 2 39 -2 3 3 0 -1 1 1 16 -2 2 2 0 -1 1 1 82 -3 3 3 17 -1 1 1 0 -1 1 4 39 -2 2 2 22 -1 1 16 134 -3 3 5 19 -6 6 6 78 -2 2 5 504 -1 1 1 0 -2 2 2 15 -1 1 1 15 -8 8 11 16 -2 2 2 15 -1 1 1 0 -1 1 1 0 -1 1 1 0 -2 2 2 20 -4 4 5 630 -1 1 1 0 -1 1 15 756 -5 7 12 695 -2 2 2 0 -1 1 5 109 -1 2 3 37 -1 1 1 15 -2 3 7 161 -2 2 3 147 -2 2 2 0 -1 1 1 96 -6 8 15 668 -1 2 3 186 -1 1 6 272 -13 14 130 4927 -2 2 2 0 -1 3 7 1139 -2 2 2 0 -2 2 23 2344 -2 2 2 15 -1 1 1 0 -2 2 2 0 -3 3 3 30 -13 14 54 4248 -2 2 2 12 -2 2 4 80 -1 1 1 0 -1 1 4 1582 -1 1 1 15 -62 149 663 34873 -2 3 5 37 -20 30 44 2679 -1 1 1 21 -1 1 3 55 -12 14 81 8433 -1 1 1 0 -579 935 2082 129790 -3 3 3 0 -1 1 1 15 -2 2 2 16 -1 1 2 13 -10 11 43 975 -1 1 2 37 -11 15 50 4375 -1 1 1 0 -1 1 1 15 -1 1 1 15 -1 1 1 0 -3 3 3 0 -17 24 74 7612 -3 5 7 888 -3 3 3 0 -1 1 1 13 -1 1 1 0 -27 28 85 6098 -1 2 9 379 -1 1 6 411 -1 1 1 0 -1 1 1 14 -1 1 11 385 -1 1 3 93 -1 2 5 163 -1 1 5 201 -1 2 8 222 -1 1 1 0 -1 1 1 0 -1 1 7 288 -1 1 4 129 -2 2 5 448 -1 1 1 0 -32 32 89 11357 -1 1 4 150 -1 1 2 13 -10 10 32 2553 -1 1 1 0 -1 1 1 15 -2 2 2 0 -2 2 4 25 -5 7 11 225 -1 1 1 0 -2 2 7 349 -4 4 4 0 -10 12 22 1981 -1 4 4 56 -1 1 1 15 -6 6 6 78 -1 1 3 41 -1 1 3 41 -5 5 7 224 -1 1 2 390 -1 1 1 0 -8 9 9 302 -1 1 1 15 -1 1 1 0 -1 1 2 878 -1 1 2 208 -9 10 15 1911 -1 1 1 12 -1 1 5 116 -6 7 11 283 -5 5 10 215 -1 1 1 0 -3 3 13 278 -2 3 3 44 -4 4 22 480 -7 7 10 332 -1 1 1 0 -3 3 6 404 -1 1 1 0 -1 1 1 0 -1 1 1 15 -2 2 2 0 -1 1 1 0 -2 2 2 0 -1 1 1 0 -2 7 12 339 -32 118 610 104334 -1 7 10 478 -1 1 1 0 -1 1 1 14 -1 1 1 0 -1 1 1 39 -2 3 4 141 -1 4 13 2158 -2 2 5 127 -1 1 4 260 -20 35 61 3086 -1 1 1 0 -1 1 1 15 -2 2 3 15 -1 1 1 15 -2 2 2 30 -5 6 14 3683 -42 47 53 914 -13 15 19 1156 -2 2 4 23 -1 1 2 39 -1 1 1 15 -1 1 33 677 -4 7 27 2015 -1 2 3 1438 -3 3 4 264 -3 3 16 441 -1 1 1 0 -1 1 1 0 -1 1 2 124 -1 1 1 14 -2 2 4 131 -1 1 1 15 -3 3 3 45 -1 1 2 24 -1 1 1 0 -1 1 1 0 -3 3 8 357 -1 1 1 0 -5 10 29 1548 -9 10 16 476 -1 1 2 8 -10 10 11 217 -1 1 1 23 -2 2 2 15 -1 1 1 16 -3 7 110 8681 -2 2 2 15 -1 1 1 13 -1 1 4 186 -96 132 319 18933 -8 9 12 125 -4 9 23 1490 -1 1 7 232 -2 2 2 29 -5 6 14 1130 -1 1 1 0 -1 1 1 15 -4 4 14 472 -1 1 1 15 -1 1 1 0 -1 1 19 224 -7 8 9 747 -1 1 4 1448 -1 1 1 15 -1 1 1 14 -10 10 16 474 -1 1 1 0 -1 1 4 51 -2 2 4 279 -4 8 82 6058 -1 1 1 0 -1 1 1 15 -3 3 3 15 -1 1 1 0 -1 1 1 16 -2 2 6 112 -1 1 2 17 -1 1 3 66 -5 5 11 429 -1 1 1 0 -1 1 1 130 -2 2 2 0 -2 3 77 1397 -2 2 9 458 -1 1 1 15 -2 3 3 15 -1 3 49 3825 -1 2 2 0 -3 3 3 0 -2 2 8 99 -2 2 2 31 -1 1 2 19 -3 3 3 15 -2 2 6 1218 -1 1 1 0 -2 2 11 108 -3 3 13 468 -1 1 2 7 -18 19 33 1067 -1 1 1 0 -1 1 4 10 -1 1 1 0 -5 5 6 15 -3 3 18 365 -1 1 3 74 -1 1 1 1528 -1 1 1 16 -1 4 5 249 -1 1 1 0 -2 3 36 1742 -2 2 2 44 -1 1 2 15 -1 1 2 14 -1 1 1 0 -6 10 50 1045 -1 1 1 0 -2 2 8 477 -1 1 2 150 -1 1 1 0 -1 1 1 0 -1 2 2 31 -1 1 44 2438 -1 1 6 525 -22 26 27 1709 -69 90 109 10068 -3 4 4 0 -1 1 4 101 -1 1 3 20 -1 1 1 14 -2 2 2 30 -1 1 1 0 -3 3 10 431 -2 2 15 1358 -1 1 1 0 -1 1 3 30 -1 1 1 0 -1 1 1 0 -126 137 259 23690 -1 1 3 161 -165 394 478 16574 -12 33 158 26562 -9 11 13 1543 1 1 1 11 -5 5 38 463 -55 109 818 86084 +6 8 22 1926 +1 1 3 4 +1 1 1 23 1 1 1 0 -1 1 7 148 -22 25 66 2914 -1 1 2 19 +1 3 31 1762 +1 2 9 2340 +4 4 4 0 +4 6 40 2088 +1 1 3 9 +1 1 1 11 +1 1 1 11 +3 3 3 0 +5 5 9 298 +2 2 2 58 +5 8 20 294 +82 93 250 25679 +12 12 66 1506 +1 2 12 1192 +1 1 1 11 +1 1 23 702 +1 1 1 0 +1 1 8 335 +1 1 1 0 +1 1 3 217 +3 3 4 278 +1 1 1 12 +7 12 25 594 +1 1 7 1496 +5 5 12 289 +1 1 2 60 +1 1 1 0 +1 1 1 11 +5 5 24 1871 +3 3 3 33 +3 3 7 65 +1 1 2 31 +1 1 11 448 +1 1 1 11 +7 9 33 1836 +1 1 7 301 +2 2 30 627 +7 7 9 158 +6 8 19 2809 +3 4 14 684 +1 1 1 0 +1 1 1 11 +1 1 1 0 +1 1 1 11 +2 2 4 47 +3 3 3 0 +2 2 28 1703 +7 7 13 94 +2 2 14 416 +2 2 2 32 +2 3 39 2194 +1 1 8 1002 +8 9 20 1033 +2 2 4 253 +3 3 5 986 +1 1 1 8 +1 1 1 0 +4 4 15 444 +3 3 3 45 +2 2 4 70 +1 1 7 139 +1 1 1 16 +6 7 38 1986 +6 13 21 1237 +26 34 49 3819 +27 29 40 882 +1 3 3 5 +1 1 1 11 +1 1 11 276 +1 2 2 22 +1 1 3 11 +1 2 6 189 +4 8 12 50 +1 1 3 30 +1 3 3 0 +1 1 1 11 +1 1 1 11 +1 1 1 11 +5 9 12 287 +1 1 1 0 +1 1 1 11 +2 2 6 848 +1 1 2 96 +1 1 1 0 +1 1 1 121 +2 2 2 0 +1 1 1 0 +2 2 2 19 +2 2 8 305 +2 2 2 19 +12 12 67 7827 +1 1 6 49 +1 1 1 11 +2 2 2 30 +13 48 162 5879 +1 1 1 0 +14 16 99 3428 +1 2 7 645 +4 5 14 1062 +1 1 1 11 +164 212 986 73006 +5 6 6 47 +1 1 2 20 +58 77 102 9209 +1 1 1 11 +7 7 12 187 +1 1 2 96 +1 1 1 11 +12 15 31 4746 +2 2 2 19 +1 2 2 23 +1 1 3 53 +1 1 1 11 +2 5 6 117 +2 2 4 49 +14 34 136 16428 +1 1 2 54 +1 1 5 1115 +1 1 1 0 +7 7 8 21 +1 1 1 0 +28 31 66 298 +3 5 5 0 +6 10 43 2347 +5 5 5 52 +4 4 46 2400 +5 5 5 0 +1 1 1 11 +1 1 1 17 +24 37 124 12370 +1 1 1 0 +1 1 1 11 +2 3 3 0 +3 3 9 39 +1 1 1 0 +1 1 2 45 +19 25 210 9807 +7 7 7 60 +1 1 1 19 +1 1 1 0 +1 1 1 0 +1 1 6 57 +2 3 5 369 +1 1 1 0 +1 4 4 38 +1 1 1 0 +9 24 82 5362 +2 2 2 0 +3 3 5 174 +2 3 4 1761 +2 2 4 24 +2 3 23 848 +21 28 60 5255 +4 4 6 112 +2 2 2 0 +125 148 303 22391 +689 2520 4204 800685 +6 6 6 0 +8 8 12 712 +1 1 6 108 +2 2 24 528 +5 7 94 5924 +1 1 1 0 +1 3 4 1822 +346 684 1504 146668 +11 14 33 1210 +1 1 1 0 +2 2 5 343 +1 1 6 277 +2 2 2 4 +3 4 4 37 +21 26 105 5345 +1 1 1 0 +1 1 1 0 +1 1 1 0 +1 1 1 0 +1 1 11 1202 +2 3 3 22 +3 3 3 0 +1 3 7 41 +6 6 29 2928 +1 1 1 0 +4 7 26 2243 +2 2 2 0 +1 1 2 42 +2 2 7 911 +1 1 2 92 +1 1 1 11 +121 272 1804 81031 +2 2 2 29 +1 1 1 0 +1 1 6 8 +1 1 1 0 +2 2 10 575 +1 1 1 0 +3 22 167 16008 +1 1 1 11 +3 3 3 33 +1 2 8 33 +1 1 1 22 +9 15 17 1289 +1 1 4 484 +1 1 1 0 +16 17 41 1687 +1 1 1 26 +1 1 6 143 +8 20 72 2685 +1 1 14 392 +1 1 1 11 +1 14 14 167 +11 11 22 894 +1 1 6 59 +1 1 1 8 +17 23 86 6438 +1 5 27 1613 +1 1 1 19 +17 26 44 5994 +1 1 1 11 +1 2 6 34 +1 1 3 12 +4 4 14 688 +2 2 20 384 +2 2 4 132 +14 37 95 5108 +4 5 7 193 +3 3 7 1324 +1 1 1 0 +1 2 4 1981 +2 2 4 66 +39 48 78 9745 +1 17 165 34059 +1 1 1 11 +4 35 539 46431 +1 1 1 19 +1 1 1 8 +1 1 3 275 +1 1 1 0 +1 1 1 11 +1 27 41 9920 +1 1 2 35 +1 1 1 253 +1 1 1 11 +4 5 12 1385 +1 1 2 216 +2 4 25 1332 +6 6 14 1193 +1 1 1 22 +3 5 5 11 +1 1 4 88 +1 1 1 0 +2 2 2 0 +1 1 1 19 +5 5 5 55 +1 1 1 0 +1 1 3 158 +1 1 3 5 +4 4 27 290 +1 1 1 11 +1 1 3 13 +2 2 2 0 +8 8 36 3345 +3 4 18 913 +1 1 1 11 +1 2 4 1637 +1 1 5 209 +1 4 19 3312 +1 1 3 23 +1 1 7 1990 +2 2 7 444 +2 2 2 119 +2 2 2 11 +4 6 44 3147 +2 2 2 11 +2 2 4 33 +1 1 1 18 +1 1 1 11 +31 83 230 25720 +30 38 98 6870 +2 2 3 29 +1 1 1 0 +1 1 6 178 +1 4 9 415 +32 37 45 4039 +1 1 1 11 +4 4 9 1568 +1 1 1 0 +38 44 81 2938 +5 14 25 655 +5 7 7 0 +5 5 7 1217 +1 1 3 121 +3 4 9 281 +24 34 106 9053 +5 7 9 56 +2 2 2 19 +10 10 10 97 +4 4 4 51 +1 1 6 165 +64 77 347 15480 +28 42 75 12045 +3 4 5 65 +1 1 1 11 +3 6 6 19 +1 1 1 0 +2 2 3 492 +6 6 12 120 +1 1 1 18 +6 6 8 118 +1 1 1 0 +21 31 119 4907 +1 1 3 371 +1 1 1 0 +1 1 3 110 +1 1 7 242 +1 2 28 343 +19 21 44 1143 +53 57 91 4678 +1 1 1 0 +29 33 252 11991 +2 2 2 11 +115 128 188 12022 +1 1 1 0 +5 7 26 1365 +1 1 10 59 +3 3 11 947 +1 1 4 226 +1 1 2 37 +30 48 80 3567 +1 1 1 0 +1 2 2 34 +38 58 165 13686 +3 3 25 1621 +1 1 12 955 +2 2 2 0 +16 17 31 1597 +3 3 3 0 +2 2 4 1025 +13 20 45 1533 +2 2 16 458 +3 5 11 789 +2 2 2 0 +4 5 5 78 +213 425 1385 117704 +13 15 26 2345 +2 2 3 66 +3 3 10 100 +1 1 1 0 +1 1 7 108 +1 1 11 882 +5 7 14 146 +1 1 1 0 +5 5 9 4035 +1 1 1 11 +1 2 2 22 +1 2 2 0 +16 17 35 4198 +3 3 3 0 +1 1 1 11 +1 1 1 0 +1 1 1 11 +3 3 9 438 +1 1 11 435 +38 50 150 24709 +3 6 28 667 +1 1 1 0 +6 9 38 1405 +45 57 358 35596 +2 2 2 42 +3 3 3 8 +1 1 1 11 +9 13 57 10251 +3 3 7 719 +9 10 12 90 +1 1 1 11 +6 6 9 476 +1 1 1 11 +1 2 3 234 +1 3 5 84 +1 1 1 11 +1 3 15 802 +1 1 1 0 +1 1 1 11 +1 1 1 11 +1 2 10 205 +1 1 1 0 +2 2 2 11 +3 3 5 98 +1 1 1 8 +1 1 3 54 +15 16 27 3419 +1 1 1 8 +3 6 22 2861 +1 1 7 76 +1 1 1 11 +1 2 2 0 +1 2 8 63 +6 8 12 484 +1 1 1 19 +12 24 113 6755 +3 5 39 2528 +1 1 1 0 +2 2 2 0 +2 5 143 2710 +2 2 2 22 +1 1 1 11 +3 3 5 78 +1 1 1 0 +3 3 3 11 +2 2 2 32 +81 87 124 42448 +1 1 1 11 +1 1 1 8 +1 1 1 0 +3 3 31 421 +1 1 2 117 +1 1 1 0 +1 27 38 5645 +1 1 1 11 +1 1 2 57 +1 1 2 49 +20 25 34 1305 +1 1 12 2165 +6 7 23 3702 +1 1 3 10 +35 35 126 10411 +5 8 66 3619 +1 2 6 253 +5 5 7 289 +3 3 5 538 +1 1 1 0 +1 1 1 0 +1 1 1 11 +1 1 3 18 +1 1 1 0 +3 3 4 57 +1 1 1 13 +5 5 7 117 +1 1 2 118 +5 5 5 21 +1 1 1 11 +1 1 1 0 +3 4 10 287 +4 4 9 2430 +1 1 2 30 +3 3 3 41 +19 20 48 1815 +1 1 1 21 +1 1 1 8 +1 3 11 1380 +1 1 1 0 +1 1 1 0 +3 3 25 764 +1 1 7 102 +1 1 1 11 +41 99 317 41164 +1 2 5 2468 +3 5 9 154 +1 1 6 107 +2 4 28 1212 +1 1 1 0 +12 17 19 277 +2 2 5 114 +1 1 9 798 +17 22 57 1481 +1 1 1 0 +2 2 7 2935 +3 4 10 1736 +4 4 26 1448 +1 1 3 91 +1 1 1 0 +1 1 1 0 +1 1 1 0 +1 1 3 47 +2 2 2 0 +1 1 2 83 +1 1 1 21 +2 2 11 763 +2 3 3 35 +33 39 84 4160 +1 1 1 11 +17 18 56 617 +1 1 6 110 +5 5 5 66 +1 1 1 8 +1 1 7 1333 +1 1 1 11 +4 4 12 1263 +5 5 20 4238 +1 1 1 11 +1 1 1 11 +3 4 11 1662 1 1 1 0 1 1 1 10 -1 1 1 17 -1 2 2 29 -1 1 1 0 -3 3 4 178 -1 1 2 99 -38 77 204 27681 -1 1 4 66 -1 1 3 499 -1 1 1 15 -7 8 18 272 -1 1 1 0 -5 5 7 73 -41 45 102 3500 -1 1 2 15 -1 1 1 0 -1 1 2 52 -1 2 34 1480 -1 1 1 0 -2 4 65 3948 -1 1 6 232 -1 1 2 54 -3 3 3 5 -1 1 1 16 -3 3 8 132 -4 4 8 466 -1 1 2 52 -7 14 49 2326 -1 1 1 0 -1 1 1 15 -1 1 3 421 -1 1 10 800 -1 1 1 0 -577 690 749 28625 -21 25 81 4843 -1 1 1 15 -2 2 3 89 -1 2 2 0 -9 11 13 656 -1 1 1 0 -3 3 5 39 -1 1 1 0 -1 1 4 76 -2 2 8 257 -21 21 41 593 -1 1 2 142 -16 16 25 726 -3 3 3 28 -8 8 13 400 -1 1 1 15 -11 11 55 1184 -1 1 1 0 -1 1 1 15 -3 3 7 269 -1 1 3 246 -10 11 36 3737 -1 1 3 211 -1 1 1 0 -1 1 5 223 -1 1 2 82 -1 1 8 320 -1 1 1 15 -1 1 1 0 -1 1 3 62 -2 2 11 82 -6 6 10 155 -1 1 2 15 -1 1 1 0 -1 1 6 182 -2 3 5 114 -1 1 1 15 -1 1 1 0 -1 1 1 0 -1 1 1 0 -1 1 2 132 -1 1 2 166 -1 1 2 46 -2 4 15 4149 -1 1 1 15 -2 3 26 518 -2 12 179 10619 -1 1 10 94 -1 1 1 15 -1 1 1 0 -1 1 9 385 -11 12 42 4487 -1 1 3 68 -5 5 21 1970 -2 2 13 374 -1 1 1 15 -1 1 11 218 -1 1 2 43 -1 1 1 0 -1 5 22 408 -4 23 61 22041 -1 2 2 0 -1 3 3 45 -1 1 1 15 -1 1 2 75 -1 1 1 14 -2 4 4 203 -8 8 9 71 -3 3 22 1545 -1 1 1 0 -12 13 30 1733 -1 1 1 15 -1 1 2 29 -42 48 69 3275 -7 8 8 146 -1 1 1 0 -1 1 1 16 -2 2 2 141 -1 2 20 320 -1 1 1 25 -1 1 1 15 -1 2 12 2196 -2 2 2 30 -1 1 6 245 -3 3 11 167 -22 34 283 11177 -2 2 2 32 -1 1 2 1460 -4 4 15 1200 -1 1 1 0 -1 1 1 15 -2 2 3 195 -3 21 38 9010 -1 1 1 15 -1 1 1 15 -2 2 2 0 -1 1 1 0 -3 3 7 365 -1 1 16 308 -15 34 212 7055 -1 1 1 0 -2 3 8 115 -1 1 1 15 -3 4 6 40 -1 1 1 15 -1 1 1 15 -1 1 2 23 3 3 3 30 -1 1 1 14 +1 1 1 11 +1 1 1 11 +5 5 19 4103 +2 4 4 11 +2 2 2 0 +6 7 45 4746 +1 1 3 809 +1 2 110 2211 +1 1 1 0 +1 1 1 0 +1 1 3 813 +1 1 1 11 +6 6 14 1117 +2 3 18 993 +3 3 14 2668 +4 4 15 1046 +55 56 56 829 +2 2 2 0 +1 2 2 40 +1 1 1 0 +1 1 1 11 +1 1 3 1407 +3 3 8 206 +1 1 1 0 +1 2 3 373 2 2 5 53 -3 3 8 1205 -5 5 5 60 -1 1 1 0 -7 8 18 2770 -2 2 9 192 -8 8 15 1420 -1 1 1 16 -18 19 20 28 -1 1 2 1575 +11 11 11 0 +13 32 80 10316 +1 1 1 8 2 2 2 0 -1 1 1 7 +14 21 129 6740 +1 2 31 1035 +18 22 182 6324 +14 15 73 2943 +1 1 13 254 +1 2 12 599 1 1 1 0 1 1 1 0 -1 1 1 15 -26 28 35 3975 -1 1 2 22 -2 2 2 107 -2 2 4 28 -67 89 147 8602 -1 1 1 26 -1 1 17 923 -4 4 4 45 1 1 1 0 -1 1 1 15 -6 8 19 646 -1 1 1 0 -1 1 3 319 -1 1 1 0 -12 48 137 10340 -1 1 2 416 -4 4 6 643 -1 1 14 388 -1 1 1 15 -1 1 2 13 -2 2 2 30 -2 2 3 68 -1 1 3 44 -1 1 42 1214 -1 1 1 26 -18 22 136 5471 -1 1 1 16 -1 1 17 3200 -1 1 1 0 -1 1 1 15 -1 1 1 14 -2 2 3 17 -1 1 4 119 -1 1 2 33 -1 1 3 143 -1 1 1 0 -37 42 106 55516 +3 3 8 106 +1 1 1 95 1 1 1 0 +5 20 1460 96399 +7 8 16 202 +16 18 43 1602 3 3 3 0 -1 1 1 15 -2 4 6 25 -4 8 34 1243 -12 16 52 2089 -25 26 29 983 -2 5 7 179 -3 4 9 1067 -5 5 8 89 -1 1 4 47 -4 4 4 0 -1 1 1 22 -1 1 7 462 -1 1 5 39 -1 1 1 25 -1 1 1 25 -2 2 9 242 -1 1 1 15 -2 7 8 109 -1 1 9 992 -2 3 3 38 +1 2 4 52 +1 1 1 11 +1 1 1 11 +3 19 266 27049 +1 1 1 8 +1 1 1 11 +17 25 30 3561 +1 1 1 8 +1 1 1 11 +1 2 9 165 +1 1 1 11 +4 5 5 49 +97 100 185 6082 +3 5 31 1387 +3 4 31 4777 1 1 1 0 -21 37 48 7793 -2 2 2 33 -7 11 16 1730 -2 3 10 3518 -1 1 1 0 -2 2 2 15 -1 1 1 15 -1 1 1 14 -2 2 3 28 -2 2 2 15 -2 2 2 15 -2 6 27 2204 -1 1 7 275 -1 1 1 16 -14 15 72 2898 -2 2 3 162 -1 1 2 24 -1 1 2 58 -4 10 19 1103 -2 2 13 271 -1 1 3 120 -1 1 1 16 -1 1 1 0 -1 1 1 15 -1 1 1 0 -1 1 1 0 -1 1 1 15 -1 1 1 0 -2 2 2 0 -3 3 7 236 -1 1 3 885 -4 5 7 135 -2 2 14 535 -3 5 15 2132 -3 3 4 48 -1 1 6 1685 -2 2 2 0 -2 2 13 2105 -1 1 5 75 -4 4 4 0 -1 1 1 16 -1 1 1 0 -2 2 3 837 -2 2 6 342 -1 2 27 2847 -1 1 2 6 -2 2 4 69 -2 2 2 0 -22 24 39 3421 -1 1 1 15 -1 1 1 15 -1 1 1 19 -4 6 6 62 -11 16 56 3186 -2 2 9 193 -1 1 1 0 -1 1 1 0 -1 2 3 138 -2 2 14 744 -1 1 1 15 -1 5 7 733 -1 1 1 0 -1 1 1 0 -4 7 18 260 -1 1 1 0 -1 1 1 14 -4 4 13 876 -19 62 248 15699 -2 3 10 100 -1 1 6 350 -1 3 3 46 -1 1 1 0 -1 1 1 0 -1 5 6 968 -3 3 3 0 -1 1 2 12 -1 1 1 0 -1 1 3 56 -1 4 20 355 -1 1 2 60 -1 1 1 14 -1 1 1 18 -2 2 7 260 -1 1 1 16 -4 4 7 240 -1 1 1 0 -1 1 1 0 -1 1 2 23 -3 3 4 827 -1 1 2 292 -1 1 2 19 -2 2 6 118 -1 1 1 0 -40 42 51 976 -96 106 146 10671 -1 1 1 0 -1 1 5 26 -1 1 3 93 -1 1 1 0 -1 1 1 15 -2 2 2 0 -3 3 3 29 -2 2 7 23 -1 1 1 15 -3 3 3 0 -1 1 1 0 -1 1 4 91 -2 3 3 0 -2 2 2 0 -1 1 1 0 -1 1 1 15 -2 2 4 83 -1 1 2 6 -1 1 1 15 -3 8 8 0 -4 6 13 1691 -1 2 60 2808 -9 9 11 638 -2 3 4 877 -2 2 4 101 -2 2 3 108 -1 23 31 313 -1 1 1 15 -1 1 1 0 -1 1 5 434 -1 1 1 0 -1 1 1 0 -2 2 2 0 -6 6 8 511 -1 1 1 0 -1 1 1 15 -4 4 6 145 -1 1 1 0 -1 1 1 15 -2 2 3 39 -1 1 1 0 -65 68 117 8892 -3 3 6 2113 -3 4 7 1121 -1 1 4 595 -1 2 10 420 -2 2 6 374 -1 2 2 0 -3 3 3 38 -2 2 3 166 -1 2 6 75 -1 1 2 5 -1 1 8 406 -1 1 4 817 -2 2 2 0 -1 1 1 15 -2 2 2 0 -1 1 1 0 -1 2 3 65 -1 1 2 11 -2 2 6 54 -3 3 3 44 -1 1 1 15 -1 1 2 133 -2 2 2 0 -6 6 7 580 -1 1 1 15 -3 3 3 43 -1 1 1 15 -1 1 16 1746 -17 19 37 3130 -1 2 7 116 -1 1 1 15 -1 1 1 520 -47 54 66 2628 -2 2 2 15 -3 4 18 1455 -1 1 1 15 -1 1 1 0 -1 1 5 104 -3 5 7 12 -1 1 1 0 -1 1 1 0 -1 1 1 0 -1 1 1 17 -1 1 2 48 -5 5 5 0 -1 1 1 16 -2 2 3 85 -1 1 1 15 -1 1 3 259 -1 1 1 15 -1 1 1 15 -1 1 1 0 -1 1 1 0 -1 1 1 0 -1 1 1 16 -1 1 1 29 -2 13 16 1090 -1 1 1 15 -2 2 2 0 -1 1 10 670 -1 1 6 75 -16 38 171 22052 -1 1 2 18 -1 2 3 127 -1 5 10 709 -21 22 23 935 -5 9 15 1779 -12 14 28 1721 -2 2 3 99 -1 1 1 0 -2 2 6 54 -1 1 1 15 -1 2 3 55 -1 1 1 0 -1 1 1 0 -3 4 13 1175 -1 1 7 363 -32 55 74 4937 -1 1 1 0 -2 3 3 0 -1 1 1 0 -1 2 2 30 -1 1 1 0 -1 1 1 0 -2 2 7 62 -1 1 2 40 -1 1 1 0 -2 3 35 2802 -1 1 1 0 -2 2 3 847 -1 2 2 0 -1 3 6 1448 -1 1 2 122 -1 1 6 2020 -2 2 2 0 -3 3 5 1168 -10 21 29 1050 -55 59 69 2209 -1 1 1 0 -1 1 3 51 -1 1 23 952 -27 32 326 11127 -3 3 6 646 -4 4 5 39 -5 5 10 668 -2 2 4 234 -2 2 2 0 -1 1 1 0 -1 1 2 141 -1 1 1 0 -1 1 1 0 -2 2 2 0 -2 4 6 517 -1 1 5 97 -1 1 1 0 -9 10 28 654 -2 2 2 0 -3 9 9 0 -2 2 4 223 -1 1 1 15 -1 1 2 14 -1 1 1 0 -1 2 2 0 -1 1 1 0 -1 2 2 15 -4 4 6 25 -1 1 1 16 -1 1 1 24 -1 1 1 18 -1 1 1 16 -1 1 1 0 -4 4 5 230 -2 3 3 30 -1 1 3 78 -1 1 1 15 -1 1 4 501 -1 1 3 99 -2 2 4 81 -2 2 9 750 -1 1 2 14 -1 1 1 14 -6 6 27 686 -1 1 1 0 -1 1 4 107 +2 2 8 273 1 1 1 20 -1 1 2 90 -1 1 61 827 -26 37 94 9174 -1 0 2 590 -1 1 1 0 -2 2 2 15 -1 2 4 11 -2 2 3 34 -2 2 2 51 -1 1 5 190 -1 1 1 0 -1 1 1 16 -1 1 1 0 -5 5 7 30 1 1 1 15 -1 1 2 15 -10 10 15 1749 -1 1 2 62 -6 6 6 0 -1 1 1 15 -3 3 3 0 +58 65 101 3789 +4 5 7 41 +1 1 7 847 +1 2 2 0 1 1 1 0 +1 1 3 4 +4 4 10 789 1 1 1 0 -5 5 5 0 -2 4 4 0 -2 2 3 128 -1 1 1 14 -3 5 46 1821 -1 1 3 21 -13 13 31 1959 -7 7 9 110 -42 44 56 5763 -1 1 1 17 +5 5 8 1200 +1 1 16 136 1 1 1 0 -1 1 18 481 -1 3 13 739 +1 1 1 11 +1 1 1 11 +2 2 5 193 +1 4 4 0 1 1 1 0 -1 1 6 82 -2 2 11 449 -1 1 1 15 -1 1 1 0 -3 3 8 1286 -19 19 27 2044 -1 1 1 15 -1 1 3 276 -1 1 1 0 -1 1 2 182 -1 1 8 205 -1 1 2 142 -1 1 2 77 -47 50 92 2135 -1 1 1 15 -7 21 81 4814 -2 2 2 30 -1 1 2 77 -1 1 1 15 -1 1 1 0 -3 3 3 0 -5 20 103 7059 -1 1 1 0 -8 8 10 46 -1 1 1 0 -1 1 1 0 -1 1 1 0 -1 1 1 4 -28 58 130 7134 -1 1 1 15 -1 1 1 0 -1 1 1 0 -4 4 39 2004 -1 1 2 416 -4 4 4 60 -1 1 1 0 -1 1 1 13 +1 1 3 110 +1 1 4 127 +6 7 10 2957 +2 2 4 52 +1 1 5 880 +2 2 3 168 +3 6 38 3866 +1 1 1 19 +1 1 1 11 +1 1 12 784 +1 1 3 20 +1 1 1 19 +1 1 1 9 +1 2 6 1026 +1 1 1 20 +1 1 1 11 +1 1 1 9 +1 1 8 281 +2 2 2 11 +1 3 3 33 +2 3 4 54 +2 2 2 38 +1 3 3 0 +1 1 1 19 +4 4 6 21 +1 4 4 50 +1 1 1 8 1 1 1 0 4 4 4 0 -1 1 4 1578 +1 1 1 19 +65 81 166 13564 +161 261 434 16985 +1 1 1 0 +1 7 7 63 +3 3 7 156 +2 5 5 0 +1 1 3 18 +1 1 1 0 +1 1 1 79 +35 36 424 8362 +1 1 7 689 +1 1 3 1994 +6 6 7 642 +3 3 3 28 +2 4 4 0 +1 7 48 1043 +2 3 3 11 +3 3 10 229 +1 1 1 0 +1 2 111 1932 +9 12 26 2788 +13 22 37 1697 +32 33 37 1452 +1 1 3 1507 +21 41 73 4716 +1 1 1 0 +1 1 1 0 +1 1 1 15 +1 1 1 18 +1 1 1 16 +4 4 4 173 +8 10 10 0 +4 4 6 63 +2 4 10 204 +2 2 2 19 +1 1 1 0 +1 1 1 0 +1 1 1 11 +1 1 1 0 +2 2 4 15 +2 3 10 123 +1 1 2 173 +1 1 1 11 +21 25 63 2482 +12 12 13 743 +40 82 182 10634 +1 1 2 173 +2 2 4 62 +149 159 243 12601 +2 2 3 129 +1 1 7 793 +2 11 13 20 +1 1 2 63 +1 1 1 11 +1 1 1 20 +1 1 1 11 +1 1 1 0 +3 3 8 1473 +1 2 4 51 +3 3 3 22 +1 1 10 1785 +8 9 10 67 +58 59 85 12924 +5 10 16 3338 +1 1 1 0 +1 2 2 44 +2 2 4 202 +1 1 7 291 +1 1 5 329 +3 3 7 1705 +1 1 1 11 +12 18 75 6272 +1 1 1 11 +4 4 4 63 +2 3 3 0 +1 3 7 135 +1 1 1 19 +1 1 1 63 +18 21 42 5809 +114 122 298 13665 +2 2 2 26 +1 1 1 21 +15 18 25 882 +1 1 1 0 +1 13 13 0 +8 9 29 2258 +20 22 26 189 +1 1 1 11 +5 7 7 0 +1 1 3 92 +1 1 6 321 +1 1 14 810 +4 5 14 287 +2 2 6 936 +5 5 5 61 +2 2 2 0 +1 1 1 19 +1 6 13 660 +2 2 2 0 +3 4 12 224 +1 1 1 0 +3 3 3 17 +4 4 7 65 +2 2 2 8 +26 30 39 844 +2 2 2 0 +3 3 5 37 +3 3 28 1226 +1 1 1 2 +1 1 3 259 +1 1 1 0 +3 4 26 477 +1 1 3 62 +3 3 38 1208 +37 45 201 6855 +1 1 2 110 +1 1 1 0 +1 1 3 1235 +1 1 1 21 +3 3 3 47 +1 1 6 1743 +1 1 1 11 +1 1 1 31 +31 39 176 8502 +1 1 1 0 +1 1 26 191 +1 1 1 0 +6 6 24 2502 +4 4 7 190 +3 3 7 1313 +1 1 2 21 +2 2 2 0 +1 1 2 57 +1 18 18 203 +1 1 1 8 +2 2 2 0 +2 2 2 79 +1 1 1 0 +1 1 1 11 +1 2 2 22 +2 2 3 2464 +1 1 3 55 +1 1 1 0 +1 1 1 11 +1 1 1 0 +1 1 3 99 +6 16 30 9372 +1 1 1 11 +1 1 1 0 +1 1 5 339 +1 1 1 11 +4 4 5 390 +24 101 608 51846 +3 3 12 599 +1 1 2 43 +1 1 1 28 +2 2 2 11 +1 1 1 0 +3 3 11 944 +2 6 11 3996 +2 2 5 102 +1 1 2 419 +1 1 1 0 +1 1 1 21 +1 1 1 11 +10 14 115 4031 +3 4 6 141 +6 10 69 1642 +7 8 26 435 +1 1 1 0 +1 1 1 0 +4 5 25 219 +1 1 7 157 +4 4 8 187 +1 1 5 505 +2 2 4 91 +1 4 19 3531 +1 2 2 0 +1 1 6 556 +1 1 1 11 +1 2 2 59 +1 1 6 35 +2 3 11 894 +2 2 4 68 +1 1 1 0 +1 1 1 0 +1 1 1 19 +2 3 6 1644 +1 1 1 11 +5 7 20 195 +57 75 422 46687 +1 1 1 11 +488 786 2802 174088 +1 1 1 32 +65 117 278 31034 +1 1 3 27 +131 136 174 6766 +7 9 15 1894 +1 1 5 293 +2 2 4 52 +3 3 5 16 +1 1 1 0 +1 1 3 102 +1 1 1 0 +1 1 7 127 +2 2 2 0 +1 1 3 641 +4 4 4 19 +2 2 4 122 +10 10 94 1765 +1 1 1 0 +3 3 6 185 +1 1 2 231 +1 1 1 11 +2 2 7 95 +1 1 1 23 +3 3 7 605 +2 3 11 55 +2 2 6 213 +5 10 26 2117 +1 1 4 77 +1 1 1 0 +7 10 19 1257 +3 3 3 0 +1 2 4 1798 +1 1 1 0 +1 1 1 11 +13 18 48 2205 +4 4 4 56 +1 1 2 164 +3 3 7 90 +1 1 1 11 +12 14 26 1101 +3 3 12 668 +1 1 6 205 +14 17 96 3247 +2 2 2 33 +3 3 3 0 +1 1 1 0 +6 8 21 1440 +1 1 3 151 +9 9 56 7312 +1 1 1 11 +1 1 14 59 +1 1 1 0 +2 2 2 0 +5 8 36 708 +1 1 1 19 +4 5 30 1111 +7 8 25 1843 +1 1 1 11 +6 12 38 3316 +2 2 2 0 +5 5 7 115 +1 1 3 10 +1 9 417 25980 +2 3 3 0 +12 12 12 410 +3 3 12 464 +1 1 3 30 +29 33 121 16070 +1 3 3 22 +1 1 1 19 +1 1 2 658 +1 1 1 8 +3 3 3 0 +1 1 1 11 +1 1 1 25 +2 2 8 751 +1 1 1 0 +2 2 4 590 +1 1 1 0 +1 1 1 11 +1 1 2 126 +20 21 31 3498 +4 4 4 30 +1 1 1 11 +1 1 3 35 +1 1 2 11 +1 1 4 50 +5 5 8 254 +3 3 3 0 +1 1 3 915 +2 3 3 50 +2 2 5 1097 +1 1 1 11 +1 2 11 510 +3 3 12 1310 +1 1 3 62 +3 3 3 34 +50 62 566 32493 +1 1 6 291 +2 2 6 1261 +2 2 6 95 +43 55 207 12664 +10 11 13 91 +1 1 2 458 +5 5 12 648 +1 1 1 11 +3 3 32 784 +2 2 2 0 +35 121 376 38864 +2 9 9 0 +1 1 10 191 +1 1 1 1 +1 1 1 0 +1 3 6 188 +1 1 1 11 +1 1 1 0 +1 1 1 11 +3 4 4 22 +2 2 2 0 +1 1 1 11 +5 6 11 805 +2 2 2 0 +1 2 97 1587 +2 3 29 2991 +2 2 4 113 +1 2 4 129 +2 3 10 224 +1 1 7 2870 +1 1 1 0 +19 37 54 3931 +1 1 1 0 +2 2 31 902 +1 1 1 11 +2 2 4 63 +3 3 3 0 +5 5 7 38 +2 2 9 137 +1 1 6 108 +1 1 3 22 +1 1 3 6 +34 46 491 18726 +6 6 12 1409 +3 4 20 1007 +1 3 61 9155 +1 1 1 0 +1 1 1 0 +114 292 1192 99949 +6 6 14 1317 +2 4 14 8244 +1 1 1 0 +2 2 5 76 +2 2 14 334 +1 1 1 0 +40 53 154 15716 +1 1 1 0 +6 7 10 1995 +3 3 6 226 +1 1 4 77 +1 2 7 3499 +1 1 1 11 +1 1 1 19 +3 4 8 188 +25 31 138 4193 +5 5 9 35 +9 24 38 6644 +1 1 1 11 +2 2 5 361 +16 34 43 5946 +1 1 1 0 +5 6 8 66 +23 30 70 4786 +3 4 22 2926 +1 1 3 28 +6 8 9 160 +8 11 68 6594 +1 1 2 253 +9 9 16 907 +3 3 3 22 +3 3 4 78 +1 1 14 411 +1 1 1 11 +5 6 15 390 +2 2 2 0 +1 1 1 0 +3 8 8 22 +2 2 2 0 +1 1 8 3835 +1 1 1 0 +2 2 2 0 +4 5 17 300 +57 63 166 5264 +1 1 6 944 +1 2 2 0 1 1 1 0 -6 18 37 1378 -1 4 4 60 -1 1 2 19 diff --git a/dbms/tests/queries/1_stateful/00024_random_counters.sql b/dbms/tests/queries/1_stateful/00024_random_counters.sql index 9411c106afe..12ae7d1b25c 100644 --- a/dbms/tests/queries/1_stateful/00024_random_counters.sql +++ b/dbms/tests/queries/1_stateful/00024_random_counters.sql @@ -1,992 +1,1000 @@ -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16447300; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 311849; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7227367; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21519412; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1010958; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6272416; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21888502; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21990949; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20331580; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 231276; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 100674; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20830528; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21099526; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 162896; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22424788; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20626921; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22734214; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21279544; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17870599; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18751324; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18264283; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19475803; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20477941; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22929382; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11299489; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16570483; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18391414; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21000451; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11369977; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9886051; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22617568; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 98054; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 970516; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21260029; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22622557; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14762728; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5482534; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10250275; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23344813; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23856562; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13223461; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1064845; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23453899; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22384789; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23982805; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15075358; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 322187; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23953330; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19325545; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21385060; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19965796; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18780277; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19454293; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2755; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4255870; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13723486; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20179597; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23510062; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16567711; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24067630; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10373758; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15871777; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21612955; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23441950; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5361274; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22348093; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23802799; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12903265; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24184564; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14005723; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 509416; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12879952; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6731659; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 148030; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21090781; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18679972; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18965725; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18882277; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20902840; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8542948; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14997619; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21554698; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21069571; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12146479; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1205378; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 771107; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22281868; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22178384; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17838109; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14603353; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10563076; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 724840; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21519640; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23695144; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16192327; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19444645; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10318822; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18910300; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1572259; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6401680; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 35054; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17775658; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23704582; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19154980; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21021124; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9070591; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1340709; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21384403; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19945789; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10049827; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 113943; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20827714; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20361523; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7068424; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5935387; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14371921; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9424240; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1683293; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22602985; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 244688; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12584464; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14582767; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11167036; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22451221; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12978052; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19790521; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15742261; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16908067; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2495578; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4380979; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1633467; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19866526; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21271384; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22486705; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21202006; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21070687; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11091565; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17985739; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22445365; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23772880; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20778388; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 230179; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16679578; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22065736; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3924838; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7212070; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22983808; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7384096; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2604; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 650932; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10900831; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24037393; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22596106; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22180217; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18547225; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1743991; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 956997; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22961593; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7082143; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15972304; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11266375; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 35774; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22151192; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 330149; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18494080; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 983810; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21804988; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23189848; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11064361; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1902205; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19899028; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20726233; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22751884; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2132557; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22594396; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 296158; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9946492; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19451524; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7763785; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16341391; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3796804; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1839; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23462068; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18318652; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11669965; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10501075; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20027482; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11583685; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 42586; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4131496; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6068563; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 70659; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1286379; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20402113; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16842985; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16461736; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14211247; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 183742; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16037572; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23579041; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6611935; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23591458; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24392791; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22161892; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13974904; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 160848; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7918660; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20085736; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1602419; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16923829; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11986633; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18207637; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12885928; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6375496; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10795105; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18557131; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17450179; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17800144; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15354772; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23144515; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15549385; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22716352; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21860068; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1940077; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23150329; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8049691; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11386126; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21803221; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21633664; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23955877; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20122396; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23727913; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13983208; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22711375; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13466764; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19050478; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23175136; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22450234; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1384335; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18792373; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7746310; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17369587; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12874756; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18735907; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12260821; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18635233; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17963329; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22934281; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17201476; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22418647; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13094539; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 752731; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18160555; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12757027; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18597361; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7433614; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5690881; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21118276; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20661916; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21953854; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20837995; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23306071; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 727709; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24061714; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 43861; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23931547; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12957628; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23807914; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 781569; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13607117; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20642545; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19101886; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7752361; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20350945; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22312993; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22495489; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1293793; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6601483; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11552092; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 864913; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3979828; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20662477; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7407445; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17868073; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21155629; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12989809; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 76719; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19830214; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17709220; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14661577; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23264026; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22911616; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6463993; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16677091; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21631570; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21111193; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5899678; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18159811; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1450043; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23534269; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1126596; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13574801; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12634069; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23823433; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14910490; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19860319; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 68737; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21617599; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21634810; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11430829; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 430818; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14317369; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22336622; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22912675; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18436540; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11961628; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 950406; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7178023; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23262754; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16829410; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 88731; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1606607; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14457415; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17923123; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13992169; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23736148; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20923675; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 142203; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20361103; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8205184; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22002982; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17587633; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20991970; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13051300; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19243744; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21625381; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1402053; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13104817; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5270176; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10227019; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 502139; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24065761; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23648725; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10145665; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14614057; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11614093; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21999106; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2040193; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11369239; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 153901; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6798544; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16963450; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21370864; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19999834; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11247409; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22176314; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18207316; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9458011; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21261418; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14765356; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23618191; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23444548; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13084939; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 115226; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22111729; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20570326; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23281543; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21768016; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17586049; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9347143; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23251951; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9837085; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22189843; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23836445; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1750735; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 48974; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 82604; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23429713; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 965907; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7941676; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1057301; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23599927; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18250675; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24033715; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20667007; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9721663; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13107145; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23406805; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15822511; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22922992; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10548637; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2437834; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16796794; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12040550; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4238398; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10793239; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21486094; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 180393; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17748835; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11223940; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1048199; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 404917; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1227383; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 150422; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22563523; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12328705; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13166551; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19248274; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20380597; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 207901; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20994013; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20533000; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21254380; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6531613; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 87499; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13607531; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12998860; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22963318; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24130984; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22186279; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6217882; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23694580; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11542936; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18403198; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20501191; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1839229; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13587707; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 337148; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 237922; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6695203; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23766385; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21893869; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22044565; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18356998; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22856353; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1940974; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21064438; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22491703; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10853794; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19208605; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7763506; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15633052; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12926980; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15619141; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21637108; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21421138; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16675555; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23824468; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23111200; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 977212; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1529177; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1757487; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17069044; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20982994; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15248224; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2958436; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20438797; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 241692; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 726991; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 187304; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22278727; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20699560; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1161170; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17143366; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3936901; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23147602; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 942211; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6360139; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22845019; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22262671; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21277948; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 658955; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21823405; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22222549; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 138960; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11868505; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7570210; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14717959; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4520284; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21945835; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20676250; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14875207; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20030020; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11187883; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 160366; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21047998; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16827436; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5243680; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1246330; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21091438; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17759953; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23344696; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4914007; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24238993; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19353595; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18776962; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11816812; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12775981; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 443080; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12073216; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21723667; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10503499; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23835697; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22324529; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9956137; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23427451; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23787157; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21645604; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12783328; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15502978; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24361354; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21571144; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18269953; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22764370; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23332939; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17378284; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24127903; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1659835; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 387614; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1813771; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24186424; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11454142; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 153960; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22881175; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16404439; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22564360; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21138574; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23913517; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10573612; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18990535; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16654072; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20622772; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18536539; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20430181; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15758722; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11805436; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15469597; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23249362; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16311706; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19907104; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 58364; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21300892; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16706377; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22226776; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11186215; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5629135; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8035720; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12074653; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12855724; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18727279; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23688403; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22348117; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22437676; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21731815; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22160845; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23529592; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21760885; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12896245; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 114634; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23317843; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23082139; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24280942; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23247967; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22473928; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23060461; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23778226; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15046105; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3236848; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7631659; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10507519; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17615887; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13046434; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20554606; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24049246; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18408787; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1497109; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17905573; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21754438; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5538790; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10829980; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13992391; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 66504; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22512982; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15652897; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21276235; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18770644; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15662662; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10161970; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18641509; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22280074; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21379552; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21264430; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23068549; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10772845; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2038669; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20626948; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 104064; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13087750; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4433956; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21344539; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16071850; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24005692; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20648488; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10638034; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 486867; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22998298; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13243405; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21667987; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13190524; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8192197; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21771235; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17734588; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23111911; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17761390; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12888601; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15711313; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21179218; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11486290; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18691054; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20880718; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21637201; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 169649; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21040945; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5734555; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21137548; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24005935; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20993179; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12570466; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10187791; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20212057; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19626166; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 81078; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17776366; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17087953; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20242894; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18246898; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23578141; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20801083; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20841418; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23750032; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12102745; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 201007; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21402886; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3063307; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23086156; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22628254; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21579151; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22474825; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22146962; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18094117; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9650656; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23760628; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24305014; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11900293; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18012577; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23059279; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17079028; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12899989; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21689104; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22357813; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17844745; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3243655; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16392643; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23500396; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11521879; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 794328; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6331864; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20151244; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 70595; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21229069; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10210294; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 461137; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20300068; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23946133; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21738727; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10631026; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 795837; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21329731; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6521590; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22407961; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22230415; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11333239; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21675262; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24131203; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22526434; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19025272; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21344743; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10787479; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14535805; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21879637; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5692288; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 105589; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20571772; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11376784; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14908795; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15896320; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1599005; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12751282; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16631110; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20863162; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20590756; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17304466; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11337502; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12787234; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20975842; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1067283; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15276505; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19962931; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 162300; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6444100; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19411474; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9790033; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22141253; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18325648; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23809642; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1354799; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23919961; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 644217; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20771050; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20695744; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6470599; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19296766; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1010887; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3025; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20735257; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22412722; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24300940; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22731619; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14886760; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18884518; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 159137; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7506880; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24278878; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15465016; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3170437; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11018566; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 65976; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1712285; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11839687; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24275806; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23469325; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21247126; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13037131; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 810429; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21335863; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24066841; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22365193; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24136081; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7518508; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24186001; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 168287; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9688087; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17368204; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9316975; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10008478; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9939709; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13130248; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 485384; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10588285; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13344565; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 755766; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17206873; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21464908; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10552819; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 513165; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 60844; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 193578; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23509153; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19839511; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21190726; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 145604; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4825291; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12910402; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17079070; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13436371; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 195125; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10275760; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21999427; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1284975; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23621008; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 285178; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13220617; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6796894; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17023966; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8123776; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17449006; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21668662; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20096968; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16708717; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10760635; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22733080; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23401273; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23510617; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14476834; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11308495; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24268282; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18114196; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20086945; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22009009; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6842632; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 550606; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22984000; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6651847; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23181460; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1073005; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24287764; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22624171; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13252402; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10718590; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9286879; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 143202; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12203728; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 211231; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11864926; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19677487; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21810544; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17221114; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10764424; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17899918; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 229129; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1098537; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 875109; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24175219; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19913062; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22563961; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19100509; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10032397; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14467660; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22418446; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19683055; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23531476; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1598765; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10355422; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3670951; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8048485; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5203336; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22183948; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23813230; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9813811; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10296184; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13266838; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23520373; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23705500; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23761111; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5693914; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24276121; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21962428; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16279855; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23125432; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22129157; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17607211; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13073128; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16297969; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1868509; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6990250; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8084032; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21613498; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6359830; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22080205; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20358655; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19484734; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2243911; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20661463; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22811053; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10448905; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21427147; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17219197; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18216478; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23375059; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16199908; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9936940; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22433392; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15615931; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17751211; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17921125; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21772687; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12905191; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5748610; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15028942; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17452492; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 934261; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20769670; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4256095; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9615136; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22405135; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5826757; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19455331; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11221384; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23065603; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1464787; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23255512; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21046240; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11152147; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19340260; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20790796; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22751983; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10663036; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12671212; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16979245; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16826713; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21908368; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 215363; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12887305; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 740471; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2006116; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24022195; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11873545; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23613532; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22024711; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15808192; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22510834; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10158082; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20062666; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2155900; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10996243; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23732275; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17657869; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12938056; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21366607; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23533687; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16908964; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21581755; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13146670; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13200580; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18423466; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22459213; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13949116; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17158843; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20406778; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21648643; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22846792; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20046997; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13194094; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19874278; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21553033; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20714647; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11354182; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10487221; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1618385; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 50359; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18610975; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20180476; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20858191; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15064975; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20795134; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24270307; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15478012; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22397041; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21325057; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11193616; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19598311; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19724464; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15102916; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16182112; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16384213; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22145798; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1367921; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 256297; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21791194; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2228362; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6101386; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20889169; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17157937; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19942327; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23165935; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7650367; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16051609; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23190688; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12277228; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18669307; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13831414; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21223684; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9615727; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23473615; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5967373; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4269664; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20641135; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24255571; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6999850; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5422; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23999812; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19762219; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21861691; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20138260; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24344110; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 698117; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21801520; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23957683; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2109130; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 214548; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10387462; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10972597; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32152608; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9627212; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25152951; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22202319; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13848191; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27855803; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27944638; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16513894; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4314057; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11878090; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23005927; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17205778; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21296650; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12068702; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8446208; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8439835; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30344780; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2881921; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1828473; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27620040; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14960013; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 103918; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9626742; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18370244; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 813903; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22176733; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17175454; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31608140; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11802602; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12577104; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 153437; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32240558; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27444870; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 79306; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15222279; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11782937; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1677; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9527330; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23580782; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33027895; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 199609; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29139484; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1700065; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30212873; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6773723; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21842879; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9460479; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16451704; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 51267; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30489182; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11947625; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18776987; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25762358; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 74905; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 877422; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3465045; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2084559; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13828281; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30299683; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 132115; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10919775; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12329250; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11525543; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32395537; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24537202; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2270964; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8518291; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11897183; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23805647; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22652078; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19363661; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32339088; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11394550; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1988179; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2135273; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14500371; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10463153; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18838936; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24492652; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26848923; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12495799; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12028938; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8934725; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18602951; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32404741; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19171705; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9831187; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20047182; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26690858; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 126413; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31244775; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15690176; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28374997; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12717244; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9152092; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5397339; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12452068; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13626118; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 46783; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11484344; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21453219; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7692388; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30879805; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27784549; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 665663; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30535786; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11685143; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13652647; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9880318; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30148588; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32745436; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27390924; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17470663; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 196859; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22123478; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 87021; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25264218; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24125574; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26099981; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1141558; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 220829; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15651875; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 182483; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28430678; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31384642; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1008241; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10462834; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26829659; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29130002; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17891770; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26531140; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15014338; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15375411; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7952204; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 41859; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21651593; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9527676; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 107394; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23409492; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31407407; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29312961; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9705505; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29848510; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10187274; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 112606; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15639744; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4375349; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1423039; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13933371; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20430236; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30679961; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 37094; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23197674; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 994587; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 437496; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3904733; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19200606; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 84668; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28581029; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11074306; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2470089; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12251899; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16996077; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12426411; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1034934; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4721601; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22026000; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21031300; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 559124; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15492463; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21419604; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25632271; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14446476; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12684903; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23292922; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26976782; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20269131; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18309978; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5305320; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30926629; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14816057; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19523905; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18775058; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32507411; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25535479; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24858652; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32420158; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4805894; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8157258; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5759745; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12626987; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5342591; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10951832; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9729032; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27999107; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7302193; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30447727; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15764416; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15727130; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15116605; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 527313; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16687935; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28304381; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17699739; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17339596; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29348067; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20861945; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12922065; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27019489; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18299445; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 108465; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 233447; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13042904; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31481509; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2267268; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26140306; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19094364; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25000943; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6860549; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30714288; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16289139; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1419182; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33436573; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30062358; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18167743; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27846382; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30148240; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32332238; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25129158; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14066924; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19832770; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29018190; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 852275; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11328399; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28179212; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20155907; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30685297; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32783957; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1552720; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28110991; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4814424; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20171153; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14920591; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 65690; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14357916; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26533001; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17014738; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11977336; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30142464; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14082365; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18851419; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27638649; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8798932; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 717825; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6912378; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26898048; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5992218; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13422462; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21204372; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17845298; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6933004; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21627605; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3395439; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22315068; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24973444; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27751340; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6022884; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32417601; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18087198; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21940806; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23809389; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9510424; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30651933; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17818815; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9038457; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9153497; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29938964; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10471118; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12913162; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14933629; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7173707; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28680585; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1279785; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33276693; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 573557; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27753414; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22968595; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25211823; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32687774; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6062762; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18866703; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5164840; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6462629; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25039797; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10789598; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33076990; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28960547; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32723171; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17888313; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29810654; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21760643; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16678170; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 368520; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12506284; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9802670; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18488016; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 227003; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15254606; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32580177; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30313645; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20879524; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27222776; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11266528; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17018146; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19902143; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19469853; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22823497; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 56768; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 136798; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16554922; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20627728; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6551053; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 124145; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10881152; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17271030; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28213281; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15665842; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28264219; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29277533; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22926441; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31057728; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8027311; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14229492; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14782220; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29099258; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 99953; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9334015; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16156945; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 124031; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1670442; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21036594; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22954047; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16054043; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 121765; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1482385; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25977258; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24596247; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 550092; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1579438; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1205; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 126296; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 177248; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27523607; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15873699; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11971473; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18965085; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19035683; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29640643; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11929806; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9352219; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18492653; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7967264; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11391453; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4289; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3567; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13575826; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2566437; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21042675; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26498330; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23764459; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32664413; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10116935; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24572551; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26788657; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12830859; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 530033; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6764575; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25219472; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10721285; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26254035; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15486693; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10323514; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23578364; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25449880; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13428298; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17679279; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27610140; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15346859; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 535736; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 513828; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20411888; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13595045; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33221835; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 97601; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12819274; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18047205; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19900235; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27830172; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20839743; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29980468; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27417156; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17908689; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24471592; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32147490; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22966030; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21060870; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 238185; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10152551; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11255139; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 982334; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15199978; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15678357; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18206303; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10902608; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22494906; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22204221; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13097211; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30998656; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26656294; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 922545; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9428510; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15137339; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15578624; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31695129; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15791360; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29571338; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5371768; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15163979; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16312681; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6126176; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16061128; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8528634; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 136544; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3093873; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3994698; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8302978; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16115563; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21804036; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9785708; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10847072; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30692218; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15582824; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19802155; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20835290; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 204284; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25636491; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30446517; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16761451; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 456303; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17301839; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27472581; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24078399; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26345482; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 451381; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8576994; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19418898; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10068353; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3767138; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 758020; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13521375; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25968099; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26805240; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13051011; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 901894; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33097016; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12545080; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29944288; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8250825; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12499373; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22535728; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11929724; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3615273; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24172869; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 116132; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12002817; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23681158; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3938; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8468701; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1295067; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27469232; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32708119; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 122578; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12139400; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12219626; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9262336; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9269892; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 122701; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19589931; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29539889; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31115640; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6283044; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30642040; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18065262; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26714391; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15351586; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13090710; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16201652; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31960256; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2658509; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 467277; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1274110; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23640128; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16197014; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28228612; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11659509; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24981440; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 52285; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30583892; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31467341; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25512316; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2908472; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 422752; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32718035; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14213540; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14951444; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6819113; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9532880; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4102488; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19537427; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7078160; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29521616; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5045377; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23131467; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22383622; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22079706; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29466380; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12045654; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30178011; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20821588; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21966434; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29390311; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19370159; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24857158; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31982180; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11990254; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3841725; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13993951; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31252290; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26398773; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 891512; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27087947; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2097095; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26252354; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13928858; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4331960; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30552074; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27905732; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30049284; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2118697; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20849218; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11338538; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3348692; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17693905; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23502543; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8905975; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18343399; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15235863; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20356153; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10552704; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28875831; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1488561; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15012941; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25726446; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2601050; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27426912; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11269650; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14880200; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 362337; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23533327; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26381021; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17522450; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31868526; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18276314; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1841289; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22234319; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11463222; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15251006; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24841412; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28755796; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9087442; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13734462; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9285105; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13289061; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29890926; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30509694; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17698850; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 46229; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16541087; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11305551; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 429238; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7583796; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8604476; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29759280; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1388922; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10884907; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18220244; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 122157; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18069840; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6707469; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26818794; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14770800; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16652737; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25497243; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14747538; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21371935; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1681601; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5343898; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22040058; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 752596; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9377867; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1848946; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1449313; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31332002; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10829982; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22431161; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29172033; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7631750; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 898844; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21460344; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25387068; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30980374; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13021547; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27715925; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30292547; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18666245; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18954194; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29070192; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 914290; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14807517; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23062682; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5132969; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15094854; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 622095; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1244323; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14804701; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11656845; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17167258; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8959523; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23121135; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4339624; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22679035; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13127067; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18362622; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4189114; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18776826; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26792263; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13409810; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22183039; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16132723; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3925258; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14248840; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18135589; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11234961; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11179577; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 178965; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10138078; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21048048; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8001235; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32833016; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32275374; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1430786; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12969140; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25529912; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18395861; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27380554; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16653574; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16372034; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28050494; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6886254; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7472729; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12646802; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6589761; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19556032; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10261903; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4389; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2415202; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20007939; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17957094; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9920354; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24840314; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5077718; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11650674; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19766470; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7854638; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9169290; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22873394; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30838169; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 79894; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25792494; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25326672; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33123311; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33237554; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15130284; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18811870; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25418177; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17202302; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31836505; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28671820; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25643858; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16338596; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27288074; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9458517; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25163573; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15680967; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20413991; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19332304; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23159444; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24708786; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 250297; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29944728; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14582542; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 512441; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31273184; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30255145; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 89813; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14959234; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26621829; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 279206; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13041403; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33392742; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10895948; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20804625; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10129067; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13855355; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31007051; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4109301; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29492024; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28963180; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11530154; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31889101; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1713672; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16069992; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9075873; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14512529; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8632591; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33056094; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28349520; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26806792; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11496875; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11797321; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25795940; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33196708; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13243216; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25096876; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26974949; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27061789; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29686454; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5045092; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2893170; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21528033; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16980819; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30854698; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1041468; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 215125; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 91347; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22706469; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33038294; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1446406; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 183702; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10246325; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13754526; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6854006; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26686232; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29345198; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15956574; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8558022; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14066782; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31710428; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6750831; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14832055; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29613113; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15159107; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6309003; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4311581; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28180829; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15131841; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20458889; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26250664; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31737265; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 802571; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25064649; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21183784; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3218637; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3375471; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1690000; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18602620; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29918973; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8555235; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32152623; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19670163; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25856874; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6142197; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27822106; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8944163; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7596672; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 129436; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33541084; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5199217; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10337246; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12718765; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10729131; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28049397; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1410155; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24924437; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16706889; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 54647; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29407271; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1575071; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6861225; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30114382; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 129970; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21103497; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3433579; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14174715; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8450741; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30033987; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11474175; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9601520; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7377941; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15646334; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18305797; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2057218; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17121933; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6870927; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19743903; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9019159; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21251610; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 239704; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16170940; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31857931; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25174672; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31546315; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 811438; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33135020; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28325470; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1196502; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 117339; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19198214; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28046111; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27663162; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3651; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8443242; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6773651; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28957858; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15586212; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 155469; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 731800; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13198917; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2080118; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17987407; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1832110; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32960999; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13858070; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2800568; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 381151; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26724412; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 238149; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20458616; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16847984; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14870120; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4729620; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12886810; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 109350; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17512881; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5250020; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 184094; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3071553; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18940958; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16166873; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13648378; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32750584; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31167464; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21597707; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21992900; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16695153; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12272303; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18958518; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11827733; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12495926; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21022681; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32262727; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12082756; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15636497; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20081370; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26349655; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32832383; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18190567; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 61749; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28596915; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28835938; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32924951; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15835912; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22905942; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12295903; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12461093; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27568271; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33525856; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10351138; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16804486; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24506501; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1336365; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19178381; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17921720; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25396786; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22031463; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19624501; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28665905; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14851585; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27554706; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14188052; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33301471; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32896955; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1134828; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27050219; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23641604; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22935857; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29805516; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25890338; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20710225; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3925036; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31404180; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25888177; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11074293; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30922753; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11403908; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25615656; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17652214; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16155802; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5565120; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5508217; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33281735; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11619273; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 67148; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22687534; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17887682; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18506413; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1443226; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13761576; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30941622; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17681363; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 187532; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 95405; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31073741; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9706801; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12504322; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31779591; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18781661; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18284607; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10633383; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3249127; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17567300; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8789986; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30073024; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26477401; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32222832; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23098807; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 50708; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25067039; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29132588; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22947337; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27778601; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25325678; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12822401; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8876685; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31096269; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17466070; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26058342; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1468384; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22665021; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24895973; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15423066; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26091197; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12103346; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15917190; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31527060; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3944; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24572480; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 229185; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17038391; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27368675; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26899897; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13257515; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19531252; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21048946; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33104049; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20824535; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15014380; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25235392; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29560548; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2599836; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32842358; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5795232; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29588193; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19019850; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29580949; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15335748; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15094099; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6308405; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20762370; +SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14121177; diff --git a/dbms/tests/queries/1_stateful/00030_array_enumerate_uniq.reference b/dbms/tests/queries/1_stateful/00030_array_enumerate_uniq.reference index ec635144f60..45a4fb75db8 100644 --- a/dbms/tests/queries/1_stateful/00030_array_enumerate_uniq.reference +++ b/dbms/tests/queries/1_stateful/00030_array_enumerate_uniq.reference @@ -1 +1 @@ -9 +8 diff --git a/dbms/tests/queries/1_stateful/00031_array_enumerate_uniq.reference b/dbms/tests/queries/1_stateful/00031_array_enumerate_uniq.reference index 80d8ffb01ea..d8d46c1a8ce 100644 --- a/dbms/tests/queries/1_stateful/00031_array_enumerate_uniq.reference +++ b/dbms/tests/queries/1_stateful/00031_array_enumerate_uniq.reference @@ -1,20 +1,20 @@ -9423131355037180 [1,1,1,1,2,1] -99069201391534782 [1,1] -126427371393908088 [1,1,1,2,1] -129406171394116372 [1,2] -137604511395271977 [1,1] -187815321395062055 [1,1,1] -194264491393666756 [1,1] -263245441387901769 [1,2,3,1,4,5,6,2,3,4,7,5,8] -312403081390726793 [1,1,1,1,1,1,1,1,1,1,1,1] -318522451358917158 [1,2] -324118291394008684 [1,1,2,3,4,5,6] -355608731360247322 [1,1,1] -408811531393490978 [1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30] -410895631386491691 [1,1,1] -465280661362859686 [1,2,3,4,5] -536058931389612176 [1,1,1] -604260701390284018 [1,2] -644639891379925126 [1,1] -658636461391365106 [1,1] -693047471394026191 [1,2,1] +5195547394193628 [1,2,1] +7118964044279372 [1,1,1,1,1,1] +14500856436384521 [1,1] +15557446916708907 [1,1,2] +21035116119959325 [1,2,1] +40726131308308591 [1,2] +43038787220788512 [1,1] +54360252114780584 [1,2,1] +59921951138443377 [1,1,1,1] +78201026678300564 [1,2,1] +95366733472363956 [1,1,1] +98979818331716939 [1,2,1,2,3,4,5,3,1,6,7,4,5,1,1,6,7] +109918405752632333 [1,1] +111844388672756531 [1,2] +137703578677775957 [1,1,1,1,2] +191075961988033432 [1,1] +238817382055499481 [1,2,1,1,3,1] +245632813703984015 [1,1,1,1] +251629483054618248 [1,1] +256951016601595607 [1,2] diff --git a/dbms/tests/queries/1_stateful/00031_array_enumerate_uniq.sql b/dbms/tests/queries/1_stateful/00031_array_enumerate_uniq.sql index 158095c4333..1898fc20579 100644 --- a/dbms/tests/queries/1_stateful/00031_array_enumerate_uniq.sql +++ b/dbms/tests/queries/1_stateful/00031_array_enumerate_uniq.sql @@ -3,11 +3,11 @@ FROM ( SELECT UserID, SearchPhrase FROM test.hits - WHERE CounterID = 731962 AND UserID IN + WHERE CounterID = 1704509 AND UserID IN ( SELECT UserID FROM test.hits - WHERE notEmpty(SearchPhrase) AND CounterID = 731962 + WHERE notEmpty(SearchPhrase) AND CounterID = 1704509 GROUP BY UserID HAVING count() > 1 ) diff --git a/dbms/tests/queries/1_stateful/00032_aggregate_key64.reference b/dbms/tests/queries/1_stateful/00032_aggregate_key64.reference index dc47fead232..0b362bf0bee 100644 --- a/dbms/tests/queries/1_stateful/00032_aggregate_key64.reference +++ b/dbms/tests/queries/1_stateful/00032_aggregate_key64.reference @@ -1,10 +1,10 @@ 0 8040919 -2 498635 -3 229872 -6 38617 -13 20490 -181 4786 -68 4735 -10 4675 -85 4494 +3 498635 +2 229872 +4 38617 +15 20490 +151 4786 +112 4735 +12 4675 +65 4494 1 3741 diff --git a/dbms/tests/queries/1_stateful/00033_aggregate_key_string.reference b/dbms/tests/queries/1_stateful/00033_aggregate_key_string.reference index 1de4721d6e1..acee3a1724c 100644 --- a/dbms/tests/queries/1_stateful/00033_aggregate_key_string.reference +++ b/dbms/tests/queries/1_stateful/00033_aggregate_key_string.reference @@ -1,10 +1,10 @@ 8267016 -интерьер ванной комнаты 2166 -яндекс 1655 -весна 2014 мода 1549 -фриформ фото 1480 -анджелина джоли 1245 -омск 1112 -фото собак разных пород 1091 -дизайн штор 1064 -баку 1000 +ст 12.168.0.1 3567 +orton 2402 +игры лица и гым чан дизайн 2166 +imgsrc 1848 +брызговик 1659 +индийский афтозный 1549 +ооооотводка и 1480 +выступная мужчин 1247 +юность 1112 diff --git a/dbms/tests/queries/1_stateful/00034_aggregate_key_fixed_string.reference b/dbms/tests/queries/1_stateful/00034_aggregate_key_fixed_string.reference index 8672453b66a..33387a94279 100644 --- a/dbms/tests/queries/1_stateful/00034_aggregate_key_fixed_string.reference +++ b/dbms/tests/queries/1_stateful/00034_aggregate_key_fixed_string.reference @@ -1,10 +1,10 @@ \0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0 8267016 -картинки 6148 -смотреть 5055 -интерьер 2417 -игры для 2284 -красивые 1756 -яндекс\0\0\0\0\0 1655 -весна 2014 1549 -фриформ ф 1480 -свадебны 1432 +смотреть 6253 +ст 12.168.0.1\0\0 3567 +картинки 2496 +orton\0\0\0\0\0\0\0\0\0\0\0\0 2402 +игры лица 2173 +индийски 2084 +скачать в 1959 +интерьер 1896 +imgsrc\0\0\0\0\0\0\0\0\0\0\0 1848 diff --git a/dbms/tests/queries/1_stateful/00035_aggregate_keys128.reference b/dbms/tests/queries/1_stateful/00035_aggregate_keys128.reference index fe6adc72c92..643272037a0 100644 --- a/dbms/tests/queries/1_stateful/00035_aggregate_keys128.reference +++ b/dbms/tests/queries/1_stateful/00035_aggregate_keys128.reference @@ -1,10 +1,10 @@ 0 0 8019359 -2 0 498608 -3 0 220903 -6 0 38616 -13 0 20489 -0 2 13924 -3 2 8944 -181 0 4786 -68 0 4700 -10 0 4673 +3 0 498608 +2 0 220903 +4 0 38616 +15 0 20489 +0 3 13924 +2 3 8944 +151 0 4786 +112 0 4700 +12 0 4673 diff --git a/dbms/tests/queries/1_stateful/00036_aggregate_hashed.reference b/dbms/tests/queries/1_stateful/00036_aggregate_hashed.reference index 7bd3aef3079..4c69e90528c 100644 --- a/dbms/tests/queries/1_stateful/00036_aggregate_hashed.reference +++ b/dbms/tests/queries/1_stateful/00036_aggregate_hashed.reference @@ -1,10 +1,10 @@ 0 8040919 -3 194175 -2 22288 -125 3327 -126 2188 -2 интерьер ванной комнаты 2166 -12 1934 -2 весна 2014 мода 1549 -2 фриформ фото 1442 -2 анджелина джоли 1245 +2 194175 +3 22288 +3 ст 12.168.0.1 3490 +101 3327 +110 2188 +3 игры лица и гым чан дизайн 2166 +8 1934 +3 orton 1599 +3 индийский афтозный 1549 diff --git a/dbms/tests/queries/1_stateful/00037_uniq_state_merge1.reference b/dbms/tests/queries/1_stateful/00037_uniq_state_merge1.reference index b456459c189..d9ca7e3be21 100644 --- a/dbms/tests/queries/1_stateful/00037_uniq_state_merge1.reference +++ b/dbms/tests/queries/1_stateful/00037_uniq_state_merge1.reference @@ -1,100 +1,100 @@ -yandex.ru 23162 23162 -e.mail.ru 7460 7460 -rutube.ru 5567 5567 -images.yandex.ru 4158 4158 -news.yandex.ru 4099 4099 -mail.yandex.ru 3571 3571 -avito.ru 3334 3334 - 3281 3281 -yabs.yandex.ru 3274 3274 -news.mail.ru 3255 3255 -hurpass.com 3066 3066 -youtube.com 2845 2845 -pogoda.yandex.ru 2816 2816 -yandex.ua 2641 2641 -maps.yandex.ru 2489 2489 -yandex.com.tr 2301 2301 -clck.yandex.ru 2238 2238 -my.mail.ru 2186 2186 -ria.ru 2155 2155 -rambler.ru 2082 2082 -tvigle.ru 1980 1980 -hugesex.tv 1930 1930 -mynet.com 1920 1920 -top.rbc.ru 1790 1790 -images.rambler.ru 1746 1746 -megogo.net 1717 1717 -kinopoisk.ru 1669 1669 -gismeteo.ru 1655 1655 -fucked-tube.com 1631 1631 -vesti.ru 1611 1611 -warthunder.ru 1562 1562 -wotlauncher.exe 1513 1513 -content.directadvert.ru 1454 1454 -tubecup.com 1418 1418 -market.yandex.ru 1415 1415 -sozcu.com.tr 1300 1300 -an.yandex.ru 1279 1279 -otvet.mail.ru 1255 1255 -code.directadvert.ru 1220 1220 -searcher.takataka.coccoc.com 1202 1202 -loginza.ru 1190 1190 -rg.ru 1183 1183 -liveinternet.ru 1173 1173 -tabs.ultimate-guitar.com 1155 1155 -news.rambler.ru 1147 1147 -vk.com 1146 1146 -gazeta.ru 1132 1132 -kinogo.net 1125 1125 -turbobit.net 1119 1119 -tv.yandex.ru 1080 1080 -bigcinema.tv 1070 1070 -itar-tass.com 1042 1042 -www.avito.ru 1036 1036 -sprashivai.ru 1034 1034 -eksisozluk.com 1018 1018 -coccoc.com 1010 1010 -haberler.com 1010 1010 -money.yandex.ru 1008 1008 -online.sberbank.ru 1008 1008 -utro.ru 999 999 -woman.ru 999 999 -wildberries.ru 959 959 -afisha.mail.ru 954 954 -regnum.ru 949 949 -kp.ru 948 948 -privatehomeclips.com 946 946 -znanija.com 946 946 -browser.yandex.ru 943 943 -marketgid.com 936 936 -lenta.ru 912 912 -video.yandex.ru 900 900 -ivi.ru 849 849 -segodnya.ua 831 831 -video.mail.ru 821 821 -interfax.ru 819 819 -worldoftanks.ru 816 816 -games.mail.ru 802 802 -disk.yandex.ru 801 801 -docviewer.yandex.ru 796 796 -cum.mobi 782 782 -market-click2.yandex.ru 781 781 -syndication.exoclick.com 778 778 -odnoklassniki.ru 772 772 -echo.msk.ru 769 769 -deti.mail.ru 765 765 -censor.net.ua 757 757 -sberbank.ru 753 753 -passport.yandex.ru 750 750 -spor.mynet.com 749 749 -bolshoyvopros.ru 732 732 -yandex.by 724 724 -korrespondent.net 723 723 -zoomby.ru 723 723 -ntv.ru 716 716 -1tv.ru 714 714 -music.yandex.ru 714 714 -yobt.com 713 713 -seasonvar.ru 704 704 -prntscr.com 699 699 -rbc.ru 697 697 +yandex.ru 25107 25107 + 21999 21999 +public_search 16749 16749 +avito.ru 16523 16523 +public 15429 15429 +mail.yandex.ru 13663 13663 +yandsearch 10039 10039 +news 8827 8827 +mail.ru 7643 7643 +doc 7537 7537 +auto.ru 7350 7350 +hurpass.com 6395 6395 +best.ru 5477 5477 +tv.yandex.ru 5341 5341 +korer.ru 4967 4967 +mail.yandsearch 4246 4246 +cars 4077 4077 +publ 3970 3970 +yandex 3845 3845 +main=hurriyet.com 3806 3806 +yandex.ua 3803 3803 +korablitz.ru 3717 3717 +uyelik.hurriyet.com 3584 3584 +e.mail.ru 3508 3508 +images.yandex.ru 3220 3220 +kinopoisk.ru 2921 2921 +coccoc.com 2707 2707 +rutube.ru 2699 2699 +rbc.ru 2644 2644 +mamba.ru 2598 2598 +video 2558 2558 +mail.yandex 2447 2447 +wot 2253 2253 +pikabu.ru 2130 2130 +yandex.php 2057 2057 +e.mail.yandex.ru 1971 1971 +brandex.ru 1969 1969 +bravoslava-230v 1942 1942 +search 1933 1933 +market.ru 1913 1913 +mynet.ru 1881 1881 +mail 1845 1845 +mail.yandex.ua 1825 1825 +rutube.com 1821 1821 +images 1812 1812 +news.rambler.com 1787 1787 +hurpass.com.tr 1763 1763 +ads.search 1742 1742 +marina_2_sezon 1680 1680 +cars.auto.ru 1628 1628 +cian.ru 1620 1620 +ivi.ru 1617 1617 +av.by 1598 1598 +world 1596 1596 +news.yandex.ru 1495 1495 +vk.com 1474 1474 +pub 1469 1469 +forum 1414 1414 +wow-girls.ru 1399 1399 +kinogo-dhpWXEdIcgoxWUZ6fgdTWw.. 1338 1338 +uyelik.hurriyet.com.tr 1330 1330 +aukro.ua 1314 1314 +plugins 1244 1244 +images.yandsearch 1235 1235 +ondom.ru 1221 1221 +korablitz.com 1189 1189 +videovol-9-sezon 1187 1187 +kerl.org 1155 1155 +mail.yandex.php 1148 1148 +file 1147 1147 +love.mail.yandex.ru 1136 1136 +yandex.kz 1124 1124 +coccoc.com.tr 1113 1113 +2gis.ru 1092 1092 +mynet.com 1073 1073 +sprashivai.ru 1072 1072 +market.yandex.ru 1064 1064 +spb-n.ru 1056 1056 +sz.spaces.ru 1055 1055 +xofx.net%2F63857&secret-oper=reply&id=0&extras] 1054 1054 +marinance.ua 1050 1050 +tube.ru 1044 1044 +haber.com 1043 1043 +image&img_url=http 1042 1042 +sport 1040 1040 +megogo.net 993 993 +sozcu.com 991 991 +yandex.by 938 938 +image&uinfo 936 936 +fast-golove.mail.ru_Mobile=0&at=35&text=производств 927 927 +linka 901 901 +gazeta.ru 892 892 +yandex.ru;yandex.ru 892 892 +kinogo-dhpWXEdIcgoxWUZ6fgdTXA.. 890 890 +fotki.yandex.ru 875 875 +fast-golove.mail.yandex.php 842 842 +news=previews 839 839 +faber 833 833 +lenta.ru 820 820 +publicdaroglundai_anketa.ru 813 813 diff --git a/dbms/tests/queries/1_stateful/00038_uniq_state_merge2.reference b/dbms/tests/queries/1_stateful/00038_uniq_state_merge2.reference index 5fc3b5a6fdb..926cb1911ba 100644 --- a/dbms/tests/queries/1_stateful/00038_uniq_state_merge2.reference +++ b/dbms/tests/queries/1_stateful/00038_uniq_state_merge2.reference @@ -1,100 +1,100 @@ -ru 458921 60255 -com 138290 60492 -ua 57079 12203 -net 40851 20955 -tr 14141 7851 -tv 13603 9771 -org 12599 9089 -by 10983 2885 -kz 6594 2571 - 6570 5540 -info 6255 4811 -su 4418 3643 -mobi 3125 2387 -biz 3008 2189 -me 2180 1829 -exe 1516 1516 -ws 1079 995 -to 988 860 -eu 979 853 -cc 920 836 -pl 819 673 -fm 745 702 -pro 692 644 -am 641 367 -vc 600 599 -lv 593 475 -az 591 390 -in 498 473 -md 467 256 -vn 459 132 -hu 426 342 -co 422 419 -de 409 390 -uz 406 256 -sx 386 277 -name 378 373 -im 361 300 -asia 359 352 -br 355 317 -kg 343 122 -us 327 203 -il 262 193 -travel 243 214 -sk 194 193 -cu 188 188 -cz 188 162 -ee 174 144 -uk 169 165 -ge 160 114 -tk 159 136 -my 138 138 -cn 131 121 -nu 130 128 -fr 110 105 -es 107 95 -sg 102 102 -io 92 59 -tl 92 92 -ro 91 87 -ca 87 86 -tj 87 44 -xxx 84 76 -be 81 80 -nl 78 63 -ly 76 76 -lt 75 71 -bg 73 66 -ae 70 53 -gl 59 59 -bz 57 49 -it 56 52 -li 55 55 -dj 51 39 -fi 50 44 -mx 41 41 -gr 40 34 -pw 37 37 -edu 36 34 -ch 35 35 -ec 35 35 -ar 34 33 -se 34 33 -so 33 29 -aero 30 25 -gov 29 28 -cl 27 27 -au 26 26 -at 25 24 -kr 24 20 -re 23 17 -hr 22 22 -pt 22 19 -tc 22 22 -za 21 21 -jp 20 20 -guru 17 17 -tm 16 15 -rs 15 14 -ms 14 14 -gg 12 12 + 582035 80248 +ru 299420 71339 +com 78253 34500 +html 40288 19569 +ua 33160 18847 +tr 19570 13117 +net 19003 12908 +php 17817 12011 +yandsearch 13598 10329 +by 9349 7695 +yandex 8946 7282 +org 5897 5320 +tv 5371 4660 +kz 5175 4588 +aspx 3084 2800 +phtml 3012 2725 +xml 2993 2726 +tr&callback_url=http 2897 2681 +su 2833 2587 +shtml 2442 2218 +hurriyet 2030 1907 +search 1915 1904 +tr&user 1556 1494 +jpg 1531 1427 +tr&users 1449 1373 +tr&callback 1294 1244 +jsp 1083 1048 +net%2F63857&secret-oper=reply&id=0&extras] 1054 1054 +htm 957 921 +ru_Mobile=0&at=35&text=производств 927 927 +lv 916 910 +tr&user_page 916 885 +exe 911 891 +me 911 864 +tr&user_page=http 900 868 +do 864 838 +tr&used 782 768 +pro 778 772 +airway 724 724 +biz 685 672 +mail 677 660 +info 593 575 +tr&callback_url=https 534 526 +tr%2Fgaleri 533 522 +bstatistik_dlja-dlya-naches 521 521 +sx 498 496 +ru%2Fupload 497 492 +news 492 487 +hu 486 479 +aspx&referer 473 459 +pogoda 460 460 +auto 438 429 +az 434 425 +net%2F63857&secret=506d9e3dfbd268e6b6630e58 432 432 +sportlibrary 431 431 +jpg,http 411 397 +tr&callbusiness 410 407 +fm 405 400 +online 401 399 +tr&callbusines 388 384 +ru%2Fnews 387 382 +bstatistic 366 366 +wbp 346 346 +am 336 333 +ru;yandsearch 330 328 +tr&user_page=https 330 328 +tr&callback_url 329 319 +html&lang=ru&lr=110&category=dressages%2Fcs306755 328 328 +pl 328 326 +blog 327 326 +jpg&pos 307 302 +bstana 305 305 +ru;yandex 287 284 +im 283 278 +diary 277 275 +slando 276 274 +eu 274 269 +to 271 269 +asp 253 250 +html&lang 253 248 +mynet 253 251 +tj 242 241 +sberbank 241 238 +haber 234 227 +jpg,https 232 232 +cc 226 221 +_2544 222 222 +ws 221 219 +mamba 220 220 +liveinteria 218 218 +tr%2Fanasayfa 215 210 +tr&user_pts=&states 213 213 +yandsearchplus 212 211 +jpg","photo 211 209 +ru%2Fwww 211 211 +com&callback_url=http 209 208 +auto-supers 208 208 +co 206 205 +kg 206 205 +ru%2Fuploads 206 205 diff --git a/dbms/tests/queries/1_stateful/00040_aggregating_materialized_view.reference b/dbms/tests/queries/1_stateful/00040_aggregating_materialized_view.reference index 61c98c6a49d..036f38c51f7 100644 --- a/dbms/tests/queries/1_stateful/00040_aggregating_materialized_view.reference +++ b/dbms/tests/queries/1_stateful/00040_aggregating_materialized_view.reference @@ -2,20 +2,20 @@ 2014-03-18 258723 36085 2014-03-19 261624 36479 2014-03-20 255328 36065 -2014-03-21 236232 35120 +2014-03-21 236232 35119 2014-03-22 197354 31256 2014-03-23 202212 31075 -2014-03-17 15797 7121 -2014-03-18 15704 7099 -2014-03-19 15599 7082 -2014-03-20 15081 6962 -2014-03-21 14409 6691 -2014-03-22 12875 5682 -2014-03-23 13199 5767 -2014-03-17 15797 7121 -2014-03-18 15704 7099 -2014-03-19 15599 7082 -2014-03-20 15081 6962 -2014-03-21 14409 6691 -2014-03-22 12875 5682 -2014-03-23 13199 5767 +2014-03-17 16854 7227 +2014-03-18 16502 7131 +2014-03-19 16606 7189 +2014-03-20 16137 7078 +2014-03-21 15638 6896 +2014-03-22 13099 5701 +2014-03-23 13297 5715 +2014-03-17 16854 7227 +2014-03-18 16502 7131 +2014-03-19 16606 7189 +2014-03-20 16137 7078 +2014-03-21 15638 6896 +2014-03-22 13099 5701 +2014-03-23 13297 5715 diff --git a/dbms/tests/queries/1_stateful/00040_aggregating_materialized_view.sql b/dbms/tests/queries/1_stateful/00040_aggregating_materialized_view.sql index 460e5f62371..909c4325a75 100644 --- a/dbms/tests/queries/1_stateful/00040_aggregating_materialized_view.sql +++ b/dbms/tests/queries/1_stateful/00040_aggregating_materialized_view.sql @@ -26,7 +26,7 @@ SELECT sumMerge(Visits) AS Visits, uniqMerge(Users) AS Users FROM test.basic -WHERE CounterID = 731962 +WHERE CounterID = 942285 GROUP BY StartDate ORDER BY StartDate; @@ -36,7 +36,7 @@ SELECT sum(Sign) AS Visits, uniq(UserID) AS Users FROM test.visits -WHERE CounterID = 731962 +WHERE CounterID = 942285 GROUP BY StartDate ORDER BY StartDate; diff --git a/dbms/tests/queries/1_stateful/00041_aggregating_materialized_view.reference b/dbms/tests/queries/1_stateful/00041_aggregating_materialized_view.reference index 61c98c6a49d..036f38c51f7 100644 --- a/dbms/tests/queries/1_stateful/00041_aggregating_materialized_view.reference +++ b/dbms/tests/queries/1_stateful/00041_aggregating_materialized_view.reference @@ -2,20 +2,20 @@ 2014-03-18 258723 36085 2014-03-19 261624 36479 2014-03-20 255328 36065 -2014-03-21 236232 35120 +2014-03-21 236232 35119 2014-03-22 197354 31256 2014-03-23 202212 31075 -2014-03-17 15797 7121 -2014-03-18 15704 7099 -2014-03-19 15599 7082 -2014-03-20 15081 6962 -2014-03-21 14409 6691 -2014-03-22 12875 5682 -2014-03-23 13199 5767 -2014-03-17 15797 7121 -2014-03-18 15704 7099 -2014-03-19 15599 7082 -2014-03-20 15081 6962 -2014-03-21 14409 6691 -2014-03-22 12875 5682 -2014-03-23 13199 5767 +2014-03-17 16854 7227 +2014-03-18 16502 7131 +2014-03-19 16606 7189 +2014-03-20 16137 7078 +2014-03-21 15638 6896 +2014-03-22 13099 5701 +2014-03-23 13297 5715 +2014-03-17 16854 7227 +2014-03-18 16502 7131 +2014-03-19 16606 7189 +2014-03-20 16137 7078 +2014-03-21 15638 6896 +2014-03-22 13099 5701 +2014-03-23 13297 5715 diff --git a/dbms/tests/queries/1_stateful/00041_aggregating_materialized_view.sql b/dbms/tests/queries/1_stateful/00041_aggregating_materialized_view.sql index 2aa97332e49..8ac1ef0f405 100644 --- a/dbms/tests/queries/1_stateful/00041_aggregating_materialized_view.sql +++ b/dbms/tests/queries/1_stateful/00041_aggregating_materialized_view.sql @@ -42,7 +42,7 @@ SELECT sumMerge(Visits) AS Visits, uniqMerge(Users) AS Users FROM test.basic -WHERE CounterID = 731962 +WHERE CounterID = 942285 GROUP BY StartDate ORDER BY StartDate; @@ -52,7 +52,7 @@ SELECT sum(Sign) AS Visits, uniq(UserID) AS Users FROM test.visits -WHERE CounterID = 731962 +WHERE CounterID = 942285 GROUP BY StartDate ORDER BY StartDate; diff --git a/dbms/tests/queries/1_stateful/00042_any_left_join.reference b/dbms/tests/queries/1_stateful/00042_any_left_join.reference index ba19dab8ba9..78cf96b8269 100644 --- a/dbms/tests/queries/1_stateful/00042_any_left_join.reference +++ b/dbms/tests/queries/1_stateful/00042_any_left_join.reference @@ -1,10 +1,7 @@ -1143050 523264 13665 -731962 475698 102664 -722545 337212 108133 -722889 252197 10545 -2237260 196036 9522 -23057320 147211 7685 -722818 90109 17837 -48221 85379 4652 -19762435 77807 7024 -722884 77492 11050 +2014-03-17 1406958 265108 +2014-03-19 1405797 261624 +2014-03-18 1383658 258723 +2014-03-20 1353623 255328 +2014-03-21 1245779 236232 +2014-03-23 1046491 202212 +2014-03-22 1031592 197354 diff --git a/dbms/tests/queries/1_stateful/00042_any_left_join.sql b/dbms/tests/queries/1_stateful/00042_any_left_join.sql index 6ff0c5d4feb..3d94ddede37 100644 --- a/dbms/tests/queries/1_stateful/00042_any_left_join.sql +++ b/dbms/tests/queries/1_stateful/00042_any_left_join.sql @@ -1,21 +1,21 @@ SELECT - CounterID, + EventDate, hits, visits FROM ( SELECT - CounterID, + EventDate, count() AS hits FROM test.hits - GROUP BY CounterID + GROUP BY EventDate ) ANY LEFT JOIN ( SELECT - CounterID, + StartDate AS EventDate, sum(Sign) AS visits FROM test.visits - GROUP BY CounterID -) USING CounterID + GROUP BY EventDate +) USING EventDate ORDER BY hits DESC LIMIT 10; diff --git a/dbms/tests/queries/1_stateful/00043_any_left_join.reference b/dbms/tests/queries/1_stateful/00043_any_left_join.reference index ba19dab8ba9..78cf96b8269 100644 --- a/dbms/tests/queries/1_stateful/00043_any_left_join.reference +++ b/dbms/tests/queries/1_stateful/00043_any_left_join.reference @@ -1,10 +1,7 @@ -1143050 523264 13665 -731962 475698 102664 -722545 337212 108133 -722889 252197 10545 -2237260 196036 9522 -23057320 147211 7685 -722818 90109 17837 -48221 85379 4652 -19762435 77807 7024 -722884 77492 11050 +2014-03-17 1406958 265108 +2014-03-19 1405797 261624 +2014-03-18 1383658 258723 +2014-03-20 1353623 255328 +2014-03-21 1245779 236232 +2014-03-23 1046491 202212 +2014-03-22 1031592 197354 diff --git a/dbms/tests/queries/1_stateful/00043_any_left_join.sql b/dbms/tests/queries/1_stateful/00043_any_left_join.sql index 9e61e69c547..495a776045f 100644 --- a/dbms/tests/queries/1_stateful/00043_any_left_join.sql +++ b/dbms/tests/queries/1_stateful/00043_any_left_join.sql @@ -1,15 +1,15 @@ SELECT - CounterID, + EventDate, count() AS hits, any(visits) FROM test.hits ANY LEFT JOIN ( SELECT - CounterID, + StartDate AS EventDate, sum(Sign) AS visits FROM test.visits - GROUP BY CounterID -) USING CounterID -GROUP BY CounterID + GROUP BY EventDate +) USING EventDate +GROUP BY EventDate ORDER BY hits DESC LIMIT 10; diff --git a/dbms/tests/queries/1_stateful/00044_any_left_join_string.reference b/dbms/tests/queries/1_stateful/00044_any_left_join_string.reference index f346dda0f2a..a96e3c9f457 100644 --- a/dbms/tests/queries/1_stateful/00044_any_left_join_string.reference +++ b/dbms/tests/queries/1_stateful/00044_any_left_join_string.reference @@ -1,10 +1,10 @@ -yandex.ru 346544 84669 -yandex.ua 46242 11768 -yandex.by 14989 3861 -images.yandex.ru 11635 0 -yabs.yandex.ru 10469 0 - 9754 169 -yandex.kz 8527 2085 -clck.yandex.ru 2605 0 -maps.yandex.ru 1809 0 -images.yandex.ua 1476 0 +auto.ru 576845 8935 +yandex.ru 410788 111278 +public 328528 23 + 313516 26015 +public_search 311125 0 +korer.ru 277987 0 +avito.ru 163820 15556 +mail.yandex.ru 152469 1046 +main=hurriyet.com 152096 259 +wot 116912 6682 diff --git a/dbms/tests/queries/1_stateful/00044_any_left_join_string.sql b/dbms/tests/queries/1_stateful/00044_any_left_join_string.sql index 06e2873c2cb..ce5236a2708 100644 --- a/dbms/tests/queries/1_stateful/00044_any_left_join_string.sql +++ b/dbms/tests/queries/1_stateful/00044_any_left_join_string.sql @@ -8,7 +8,6 @@ FROM domain(URL) AS domain, count() AS hits FROM test.hits - WHERE CounterID = 731962 GROUP BY domain ) ANY LEFT JOIN ( @@ -16,7 +15,6 @@ FROM domain(StartURL) AS domain, sum(Sign) AS visits FROM test.visits - WHERE CounterID = 731962 GROUP BY domain ) USING domain ORDER BY hits DESC diff --git a/dbms/tests/queries/1_stateful/00047_bar.reference b/dbms/tests/queries/1_stateful/00047_bar.reference index 7ef3fab92d9..61999ae73c9 100644 --- a/dbms/tests/queries/1_stateful/00047_bar.reference +++ b/dbms/tests/queries/1_stateful/00047_bar.reference @@ -1,100 +1,100 @@ -1143050 523264 ████████████████████████████████████████████████████████████████████████████████ -731962 475698 ████████████████████████████████████████████████████████████████████████▋ -722545 337212 ███████████████████████████████████████████████████▌ -722889 252197 ██████████████████████████████████████▌ -2237260 196036 █████████████████████████████▊ -23057320 147211 ██████████████████████▌ -722818 90109 █████████████▋ -48221 85379 █████████████ -19762435 77807 ███████████▊ -722884 77492 ███████████▋ -20860117 73213 ███████████▏ -21211267 68945 ██████████▌ -22753222 67570 ██████████▎ -12725416 64174 █████████▋ -23910442 60456 █████████▏ -23414332 58389 ████████▊ -160656 57017 ████████▋ -14545480 52345 ████████ -64174 52142 ███████▊ -24142063 47758 ███████▎ -10288858 44080 ██████▋ -19765189 43395 ██████▋ -15553948 43279 ██████▌ -13837519 40581 ██████▏ -17956927 37562 █████▋ -23969905 34301 █████▏ -21137731 32776 █████ -23723584 28788 ████▍ -16443139 26603 ████ -15068284 25733 ███▊ -64539 25595 ███▊ -24201256 25585 ███▊ -16949086 25496 ███▊ -805556 25270 ███▋ -815578 24744 ███▋ -16037275 23349 ███▌ -14645857 21270 ███▎ -16443178 20825 ███▏ -13205491 20788 ███▏ -146686 20785 ███▏ -14231362 19897 ███ -94020 19724 ███ -38230 19717 ███ -15011071 19402 ██▊ -9325861 18557 ██▋ -7234936 18370 ██▋ -115080 17443 ██▋ -22131482 17390 ██▋ -15987325 17302 ██▋ -10652611 17279 ██▋ -114208 16959 ██▌ -23427556 16849 ██▌ -21407305 16175 ██▍ -21270109 16017 ██▍ -7692568 15340 ██▎ -9927988 15171 ██▎ -18746557 15146 ██▎ -20008321 15104 ██▎ -18274111 14719 ██▎ -13227769 14584 ██▏ -23474449 14540 ██▏ -91244 14199 ██▏ -17969140 13972 ██▏ -149814 13930 ██▏ -230672 13792 ██ -22663942 13615 ██ -1911064 13509 ██ -79376 13308 ██ -10065061 13181 ██ -20424475 13181 ██ -23745772 12922 █▊ -67763 12520 █▊ -15748243 12352 █▊ -1605811 12283 █▊ -11492179 12183 █▋ -24327397 12170 █▋ -23878183 12158 █▋ -7941022 12049 █▋ -10849243 11818 █▋ -14544658 11733 █▋ -21374125 11658 █▋ -12929146 11514 █▋ -1276757 11452 █▋ -960630 11444 █▋ -10538560 11358 █▋ -62180 11193 █▋ -24056266 11047 █▋ -20793439 10936 █▋ -6414910 10747 █▋ -23194783 10738 █▋ -14131513 10656 █▋ -1598595 10655 █▋ -158751 10625 █▌ -150748 10522 █▌ -3228 10503 █▌ -18520561 10128 █▌ -1252825 10098 █▌ -4308403 10022 █▌ -199955 9997 █▌ -23214754 9780 █▍ +1704509 523264 ████████████████████████████████████████████████████████████████████████████████ +732797 475698 ████████████████████████████████████████████████████████████████████████▋ +598875 337212 ███████████████████████████████████████████████████▌ +792887 252197 ██████████████████████████████████████▌ +3807842 196036 █████████████████████████████▊ +25703952 147211 ██████████████████████▌ +716829 90109 █████████████▋ +59183 85379 █████████████ +33010362 77807 ███████████▊ +800784 77492 ███████████▋ +20810645 73213 ███████████▏ +25843850 68945 ██████████▌ +23447120 67570 ██████████▎ +14739804 64174 █████████▋ +32077710 60456 █████████▏ +22446879 58389 ████████▊ +170282 57017 ████████▋ +11482817 52345 ████████ +63469 52142 ███████▊ +29103473 47758 ███████▎ +10136747 44080 ██████▋ +27528801 43395 ██████▋ +10581377 43279 ██████▌ +9841201 40581 ██████▏ +20310963 37562 █████▋ +17337667 34301 █████▏ +28600281 32776 █████ +32046685 28788 ████▍ +10130880 26603 ████ +8676831 25733 ███▊ +53230 25595 ███▊ +20271226 25585 ███▊ +17420663 25496 ███▊ +631207 25270 ███▋ +633130 24744 ███▋ +14324015 23349 ███▌ +8537965 21270 ███▎ +11285298 20825 ███▏ +14937615 20788 ███▏ +185050 20785 ███▏ +16368233 19897 ███ +81602 19724 ███ +62896 19717 ███ +12967664 19402 ██▊ +15996597 18557 ██▋ +4379238 18370 ██▋ +90982 17443 ██▋ +18211045 17390 ██▋ +14625884 17302 ██▋ +12864910 17279 ██▋ +126096 16959 ██▌ +30296134 16849 ██▌ +26360482 16175 ██▍ +17788950 16017 ██▍ +5928716 15340 ██▎ +15469035 15171 ██▎ +29732125 15146 ██▎ +32946244 15104 ██▎ +20957241 14719 ██▎ +9495695 14584 ██▏ +29241146 14540 ██▏ +109805 14199 ██▏ +26905788 13972 ██▏ +212019 13930 ██▏ +171509 13792 ██ +23913162 13615 ██ +1861993 13509 ██ +125776 13308 ██ +11312316 13181 ██ +32667326 13181 ██ +28628973 12922 █▊ +122804 12520 █▊ +12322758 12352 █▊ +1301819 12283 █▊ +10769545 12183 █▋ +21566939 12170 █▋ +28905364 12158 █▋ +4250765 12049 █▋ +15009727 11818 █▋ +12761932 11733 █▋ +26995888 11658 █▋ +12759346 11514 █▋ +1507911 11452 █▋ +968488 11444 █▋ +15736172 11358 █▋ +54310 11193 █▋ +17027391 11047 █▋ +17439919 10936 █▋ +4480860 10747 █▋ +26738469 10738 █▋ +9986231 10656 █▋ +1539995 10655 █▋ +214556 10625 █▌ +219339 10522 █▌ +3266 10503 █▌ +30563429 10128 █▌ +1960469 10098 █▌ +7901143 10022 █▌ +194599 9997 █▌ +21052498 9780 █▍ diff --git a/dbms/tests/queries/1_stateful/00049_max_string_if.reference b/dbms/tests/queries/1_stateful/00049_max_string_if.reference index 51f17c87649..f87bc6d1fd2 100644 --- a/dbms/tests/queries/1_stateful/00049_max_string_if.reference +++ b/dbms/tests/queries/1_stateful/00049_max_string_if.reference @@ -1,20 +1,20 @@ -1143050 523264 яндекс почта яндекс почта -731962 475698 • ротация товара по принципу fifo; -722545 337212 індія вавілон египет -722889 252197 ↵ ↵ ↵ анна↵↵ ↵ ↵ ↵ -2237260 196036 ярмарка калининград авто с пробегом -23057320 147211 -722818 90109 яровая -48221 85379 уссурийский аграрный техникум -19762435 77807 электромобиль купить в красноярске -722884 77492 ян женчак лявоны -20860117 73213 ярмарка калининград авто с пробегом -21211267 68945 マジックかいと -22753222 67570 ютуб видео -12725416 64174 эдгар по фильм ворон -23910442 60456 -23414332 58389 №18-52-857 от 19.03.2002 -160656 57017 яндекс маркет -14545480 52345 şu polarmi apolarmi -64174 52142 яндекс почта -24142063 47758 +1704509 523264 نيك امريكي نيك افلام سكس جامد +732797 475698 نيك سكس سيحاق +598875 337212 سکس باصات +792887 252197 №2267 отзыв +3807842 196036 ярмаркетовара 200кг купить по неделю тебелье +25703952 147211 +716829 90109 яндекс повыш +59183 85379 франция машину угловы крузер из кофе +33010362 77807 ярмаркетовара 200кг купить по неделю тебелье +800784 77492 ярмаркур смерти теплицы из чего +20810645 73213 ярмаркетовара 200кг купить по неделю тебе перево метиков детский +25843850 68945 электросчет-фактура +23447120 67570 южная степанов +14739804 64174 штангал волк +32077710 60456 +22446879 58389 فیلم سكس امريكي نيك +170282 57017 ل افلام السكس +11482817 52345 я скачать игры +63469 52142 яндекс марте рокус надписями я любимому у полосы фото минск +29103473 47758 diff --git a/dbms/tests/queries/1_stateful/00050_min_max.reference b/dbms/tests/queries/1_stateful/00050_min_max.reference index 83c625ab0cc..ab47fd7a69a 100644 --- a/dbms/tests/queries/1_stateful/00050_min_max.reference +++ b/dbms/tests/queries/1_stateful/00050_min_max.reference @@ -1,20 +1,20 @@ -1143050 5713920003519694728 5716397076045616998 -731962 5713920015071971218 5716397257116862311 -722545 5713920015651149659 5716397256090216288 -722889 5713920017089177471 5716397260516085458 -2237260 5713920016994238167 5716397237297002364 -23057320 5713949520802533074 5716413447410800488 -722818 5713920043014670169 5716397236420603606 -48221 5713920004276864894 5716397210278713206 -19762435 5713920017181982418 5716397246466066301 -722884 5713920011998402414 5716397246767304399 -20860117 5713920017370074831 5716397208719681371 -21211267 5713920045935435469 5716397248483389135 -22753222 5713920007612184431 5716397151789805258 -12725416 5713920109650941822 5716397254591100797 -23910442 5714042239335131000 5716386006459287388 -23414332 5713920019599158998 5716397247414163312 -160656 5713921425816235729 5716396958603462522 -14545480 5713949495531890580 5716413422480215759 -64174 5713920018026575709 5716397258157885302 -24142063 5713875910905472884 5716353012118216394 +1704509 4611700827100483880 9223360787015464643 +732797 4611701940806302259 9223355550934604746 +598875 4611701407242345792 9223362250391155632 +792887 4611699550286611812 9223290551912005343 +3807842 4611710821592843606 9223326163906184987 +25703952 4611709443519524003 9223353913449113943 +716829 4611852156092872082 9223361623076951140 +59183 4611730685242027332 9223354909338698162 +33010362 4611704682869732882 9223268545373999677 +800784 4611752907938305166 9223340418389788041 +20810645 4611712185532639162 9223218900001937412 +25843850 4611690025407720929 9223346023778617822 +23447120 4611796031755620254 9223329309291309758 +14739804 4611692230555590277 9223313509005166531 +32077710 4611884228437061959 9223352444952988904 +22446879 4611846229717089436 9223124373140579096 +170282 4611833225706935900 9223371583739401906 +11482817 4611708000353743073 9223337838355779113 +63469 4611695097019173921 9223353530156141191 +29103473 4611744585914335132 9223333530281362537 diff --git a/dbms/tests/queries/1_stateful/00051_min_max_array.reference b/dbms/tests/queries/1_stateful/00051_min_max_array.reference index fac952200a9..a5f1b6cdfef 100644 --- a/dbms/tests/queries/1_stateful/00051_min_max_array.reference +++ b/dbms/tests/queries/1_stateful/00051_min_max_array.reference @@ -1,20 +1,20 @@ -1143050 523264 [353023] [] [353023] -731962 475698 [] [] [] -722545 337212 [] [] [] -722889 252197 [1698655] [] [1545418] -2237260 196036 [1096990] [] [1095625] -23057320 147211 [] [] [] -722818 90109 [4187887] [] [1418056] -48221 85379 [] [] [] -19762435 77807 [] [] [] -722884 77492 [3440203] [] [1784959] -20860117 73213 [] [] [] -21211267 68945 [3658087] [] [3658087] -22753222 67570 [4249348,3287725] [] [3287395,3287425] -12725416 64174 [2645938] [] [939124] -23910442 60456 [] [] [] -23414332 58389 [] [] [] -160656 57017 [4118521,4067026,2509678] [] [56598,1084654,1113568,1960378] -14545480 52345 [] [] [] -64174 52142 [] [] [] -24142063 47758 [4211836] [] [4211836] +1704509 523264 [271264] [] [271264] +732797 475698 [] [] [] +598875 337212 [] [] [] +792887 252197 [2094893,2028343] [] [1272031] +3807842 196036 [1710269] [] [1134660] +25703952 147211 [] [] [] +716829 90109 [4186138] [] [1800405] +59183 85379 [] [] [] +33010362 77807 [] [] [] +800784 77492 [4002316] [] [1270480] +20810645 73213 [] [] [] +25843850 68945 [4028285] [] [4028285] +23447120 67570 [6503091,2762273] [] [2098132] +14739804 64174 [4180720] [] [664490] +32077710 60456 [] [] [] +22446879 58389 [] [] [] +170282 57017 [4166114] [] [34386,1240412,1248634,1616213,2928740,1458582] +11482817 52345 [] [] [] +63469 52142 [] [] [] +29103473 47758 [6185451] [] [6185451] diff --git a/dbms/tests/queries/1_stateful/00052_group_by_in.reference b/dbms/tests/queries/1_stateful/00052_group_by_in.reference index 46bd3c48aa6..8536c8df413 100644 --- a/dbms/tests/queries/1_stateful/00052_group_by_in.reference +++ b/dbms/tests/queries/1_stateful/00052_group_by_in.reference @@ -1,14 +1,14 @@ -2014-03-17 other 306 -2014-03-17 type_in 52 -2014-03-18 other 307 -2014-03-18 type_in 50 -2014-03-19 other 332 -2014-03-19 type_in 42 -2014-03-20 other 276 -2014-03-20 type_in 31 -2014-03-21 other 262 -2014-03-21 type_in 48 -2014-03-22 other 232 -2014-03-22 type_in 52 -2014-03-23 other 329 -2014-03-23 type_in 34 +2014-03-17 other 3127 +2014-03-17 type_in 93 +2014-03-18 other 2727 +2014-03-18 type_in 83 +2014-03-19 other 2716 +2014-03-19 type_in 100 +2014-03-20 other 2571 +2014-03-20 type_in 101 +2014-03-21 other 2500 +2014-03-21 type_in 84 +2014-03-22 other 1858 +2014-03-22 type_in 64 +2014-03-23 other 1758 +2014-03-23 type_in 55 diff --git a/dbms/tests/queries/1_stateful/00052_group_by_in.sql b/dbms/tests/queries/1_stateful/00052_group_by_in.sql index 17fb4acaaae..35d3b4bac91 100644 --- a/dbms/tests/queries/1_stateful/00052_group_by_in.sql +++ b/dbms/tests/queries/1_stateful/00052_group_by_in.sql @@ -1,4 +1,4 @@ select StartDate, TraficSourceID in (0) ? 'type_in' : 'other' as traf_type, sum(Sign) from test.visits -where CounterID = 160656 +where CounterID = 842440 group by StartDate, traf_type ORDER BY StartDate, traf_type diff --git a/dbms/tests/queries/1_stateful/00054_merge_tree_partitions.reference b/dbms/tests/queries/1_stateful/00054_merge_tree_partitions.reference index 43960608828..330b046991e 100644 --- a/dbms/tests/queries/1_stateful/00054_merge_tree_partitions.reference +++ b/dbms/tests/queries/1_stateful/00054_merge_tree_partitions.reference @@ -1,12 +1,12 @@ -475698 -42917 -432781 -432781 +523264 +29359 +493905 +493905 0 -432781 -475698 -42917 -432781 -458792 -475698 -492604 +493905 +523264 +29359 +493905 +491221 +523264 +555307 diff --git a/dbms/tests/queries/1_stateful/00054_merge_tree_partitions.sql b/dbms/tests/queries/1_stateful/00054_merge_tree_partitions.sql index 8e8ea29a9db..73020952977 100644 --- a/dbms/tests/queries/1_stateful/00054_merge_tree_partitions.sql +++ b/dbms/tests/queries/1_stateful/00054_merge_tree_partitions.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS test.partitions; CREATE TABLE test.partitions (EventDate Date, CounterID UInt32) ENGINE = MergeTree(EventDate, CounterID, 8192); -INSERT INTO test.partitions SELECT EventDate + UserID % 365 AS EventDate, CounterID FROM test.hits WHERE CounterID = 731962; +INSERT INTO test.partitions SELECT EventDate + UserID % 365 AS EventDate, CounterID FROM test.hits WHERE CounterID = 1704509; SELECT count() FROM test.partitions; @@ -24,7 +24,7 @@ ALTER TABLE test.partitions DETACH PARTITION 201403; SELECT count() FROM test.partitions; -INSERT INTO test.partitions SELECT EventDate + UserID % 365 AS EventDate, CounterID FROM test.hits WHERE CounterID = 731962 AND toStartOfMonth(EventDate) = toDate('2014-03-01'); +INSERT INTO test.partitions SELECT EventDate + UserID % 365 AS EventDate, CounterID FROM test.hits WHERE CounterID = 1704509 AND toStartOfMonth(EventDate) = toDate('2014-03-01'); SELECT count() FROM test.partitions; diff --git a/dbms/tests/queries/1_stateful/00056_view.reference b/dbms/tests/queries/1_stateful/00056_view.reference index 16cf0436b1c..30b3a449c1e 100644 --- a/dbms/tests/queries/1_stateful/00056_view.reference +++ b/dbms/tests/queries/1_stateful/00056_view.reference @@ -9,23 +9,23 @@ 8 3083 9 2347 10 2070 -1143050 523264 -731962 475698 -722545 337212 -722889 252197 -2237260 196036 -23057320 147211 -722818 90109 -48221 85379 -19762435 77807 -722884 77492 -1143050 523264 -731962 475698 -722545 337212 -722889 252197 -2237260 196036 -23057320 147211 -722818 90109 -48221 85379 -19762435 77807 -722884 77492 +1704509 523264 +732797 475698 +598875 337212 +792887 252197 +3807842 196036 +25703952 147211 +716829 90109 +59183 85379 +33010362 77807 +800784 77492 +1704509 523264 +732797 475698 +598875 337212 +792887 252197 +3807842 196036 +25703952 147211 +716829 90109 +59183 85379 +33010362 77807 +800784 77492 diff --git a/dbms/tests/queries/1_stateful/00059_merge_sorting_empty_array_joined.sql b/dbms/tests/queries/1_stateful/00059_merge_sorting_empty_array_joined.sql index 9ff482e4b46..8781c86b622 100644 --- a/dbms/tests/queries/1_stateful/00059_merge_sorting_empty_array_joined.sql +++ b/dbms/tests/queries/1_stateful/00059_merge_sorting_empty_array_joined.sql @@ -1 +1 @@ -SELECT CounterID FROM test.visits ARRAY JOIN Goals.ID WHERE CounterID = 731962 ORDER BY CounterID +SELECT CounterID FROM test.visits ARRAY JOIN Goals.ID WHERE CounterID = 942285 ORDER BY CounterID diff --git a/dbms/tests/queries/1_stateful/00060_move_to_prewhere_and_sets.reference b/dbms/tests/queries/1_stateful/00060_move_to_prewhere_and_sets.reference index 60a2c1d1bb7..b8639573348 100644 --- a/dbms/tests/queries/1_stateful/00060_move_to_prewhere_and_sets.reference +++ b/dbms/tests/queries/1_stateful/00060_move_to_prewhere_and_sets.reference @@ -1 +1 @@ -4487 +476 diff --git a/dbms/tests/queries/1_stateful/00061_storage_buffer.reference b/dbms/tests/queries/1_stateful/00061_storage_buffer.reference index 50a5f6923ea..70630b184cb 100644 --- a/dbms/tests/queries/1_stateful/00061_storage_buffer.reference +++ b/dbms/tests/queries/1_stateful/00061_storage_buffer.reference @@ -1,4 +1,4 @@ -2417 +77492 0 -2417 -2417 +77492 +77492 diff --git a/dbms/tests/queries/1_stateful/00061_storage_buffer.sql b/dbms/tests/queries/1_stateful/00061_storage_buffer.sql index 4e3f4b81394..e1f67abda20 100644 --- a/dbms/tests/queries/1_stateful/00061_storage_buffer.sql +++ b/dbms/tests/queries/1_stateful/00061_storage_buffer.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS test.hits_buffer; CREATE TABLE test.hits_dst AS test.hits; CREATE TABLE test.hits_buffer AS test.hits_dst ENGINE = Buffer(test, hits_dst, 8, 1, 10, 10000, 100000, 10000000, 100000000); -INSERT INTO test.hits_buffer SELECT * FROM test.hits WHERE CounterID = 101500; +INSERT INTO test.hits_buffer SELECT * FROM test.hits WHERE CounterID = 800784; SELECT count() FROM test.hits_buffer; SELECT count() FROM test.hits_dst; diff --git a/dbms/tests/queries/1_stateful/00062_loyalty.reference b/dbms/tests/queries/1_stateful/00062_loyalty.reference index 266295d996d..605e4881dd4 100644 --- a/dbms/tests/queries/1_stateful/00062_loyalty.reference +++ b/dbms/tests/queries/1_stateful/00062_loyalty.reference @@ -1,12 +1,12 @@ --10 4291 ████████████████████████████████████████████████████████████████████████████▊ --9 222 █████████████████████████████████████████████████▋ --8 156 ██████████████████████████████████████████████▍ --7 102 ██████████████████████████████████████████▌ --6 91 █████████████████████████████████████████▌ --5 100 ██████████████████████████████████████████▍ -5 87 █████████████████████████████████████████▏ -6 123 ████████████████████████████████████████████▎ -7 133 █████████████████████████████████████████████ -8 236 ██████████████████████████████████████████████████▎ -9 603 ██████████████████████████████████████████████████████████▊ -10 5604 ███████████████████████████████████████████████████████████████████████████████▎ +-10 5604 ███████████████████████████████████████████████████████████████████████████████▎ +-9 603 ██████████████████████████████████████████████████████████▊ +-8 236 ██████████████████████████████████████████████████▎ +-7 133 █████████████████████████████████████████████ +-6 123 ████████████████████████████████████████████▎ +-5 105 ██████████████████████████████████████████▊ +5 82 ████████████████████████████████████████▋ +6 91 █████████████████████████████████████████▌ +7 102 ██████████████████████████████████████████▌ +8 156 ██████████████████████████████████████████████▍ +9 222 █████████████████████████████████████████████████▋ +10 4291 ████████████████████████████████████████████████████████████████████████████▊ diff --git a/dbms/tests/queries/1_stateful/00063_loyalty_joins.reference b/dbms/tests/queries/1_stateful/00063_loyalty_joins.reference index f43569bbd7a..e4c3619bf5a 100644 --- a/dbms/tests/queries/1_stateful/00063_loyalty_joins.reference +++ b/dbms/tests/queries/1_stateful/00063_loyalty_joins.reference @@ -1,51 +1,51 @@ --10 1244506 --9 145771 --8 74083 --7 52819 --6 32860 --5 54350 +-10 2932018 +-9 472052 +-8 136048 +-7 73688 +-6 56766 +-5 55691 0 3550514 -5 48423 -6 56766 -7 73688 -8 136048 -9 472052 -10 2932018 --10 1244506 --9 145771 --8 74083 --7 52819 --6 32860 --5 54350 +5 47082 +6 32860 +7 52819 +8 74083 +9 145771 +10 1244506 +-10 2932018 +-9 472052 +-8 136048 +-7 73688 +-6 56766 +-5 55691 0 3550514 -5 48423 -6 56766 -7 73688 -8 136048 -9 472052 -10 2932018 --10 1244506 --9 145771 --8 74083 --7 52819 --6 32860 --5 54350 +5 47082 +6 32860 +7 52819 +8 74083 +9 145771 +10 1244506 +-10 2932018 +-9 472052 +-8 136048 +-7 73688 +-6 56766 +-5 55691 0 3550514 -5 48423 -6 56766 -7 73688 -8 136048 -9 472052 -10 2932018 --10 1244506 ███████████████████████████████████████████████████████████████████████████▎ --9 145771 ███████████████████████████████████████████████████████████████▋ --8 74083 ████████████████████████████████████████████████████████████▏ --7 52819 ██████████████████████████████████████████████████████████▎ --6 32860 ███████████████████████████████████████████████████████▋ --5 54350 ██████████████████████████████████████████████████████████▍ -5 48423 █████████████████████████████████████████████████████████▋ -6 56766 ██████████████████████████████████████████████████████████▋ -7 73688 ████████████████████████████████████████████████████████████ -8 136048 ███████████████████████████████████████████████████████████████▍ -9 472052 ██████████████████████████████████████████████████████████████████████ -10 2932018 ███████████████████████████████████████████████████████████████████████████████▊ +5 47082 +6 32860 +7 52819 +8 74083 +9 145771 +10 1244506 +-10 2932018 ███████████████████████████████████████████████████████████████████████████████▊ +-9 472052 ██████████████████████████████████████████████████████████████████████ +-8 136048 ███████████████████████████████████████████████████████████████▍ +-7 73688 ████████████████████████████████████████████████████████████ +-6 56766 ██████████████████████████████████████████████████████████▋ +-5 55691 ██████████████████████████████████████████████████████████▌ +5 47082 █████████████████████████████████████████████████████████▋ +6 32860 ███████████████████████████████████████████████████████▋ +7 52819 ██████████████████████████████████████████████████████████▎ +8 74083 ████████████████████████████████████████████████████████████▏ +9 145771 ███████████████████████████████████████████████████████████████▋ +10 1244506 ███████████████████████████████████████████████████████████████████████████▎ diff --git a/dbms/tests/queries/1_stateful/00065_loyalty_with_storage_join.reference b/dbms/tests/queries/1_stateful/00065_loyalty_with_storage_join.reference index 6ce4ca04eea..a5cf625a288 100644 --- a/dbms/tests/queries/1_stateful/00065_loyalty_with_storage_join.reference +++ b/dbms/tests/queries/1_stateful/00065_loyalty_with_storage_join.reference @@ -1,24 +1,24 @@ --10 1244506 --9 145771 --8 74083 --7 52819 --6 32860 --5 54350 -5 48423 -6 56766 -7 73688 -8 136048 -9 472052 -10 2932018 --10 1244506 --9 145771 --8 74083 --7 52819 --6 32860 --5 54350 -5 48423 -6 56766 -7 73688 -8 136048 -9 472052 -10 2932018 +-10 2932018 +-9 472052 +-8 136048 +-7 73688 +-6 56766 +-5 55691 +5 47082 +6 32860 +7 52819 +8 74083 +9 145771 +10 1244506 +-10 2932018 +-9 472052 +-8 136048 +-7 73688 +-6 56766 +-5 55691 +5 47082 +6 32860 +7 52819 +8 74083 +9 145771 +10 1244506 diff --git a/dbms/tests/queries/1_stateful/00067_union_all.reference b/dbms/tests/queries/1_stateful/00067_union_all.reference index 0a5bc0f84e0..764a9bb785f 100644 --- a/dbms/tests/queries/1_stateful/00067_union_all.reference +++ b/dbms/tests/queries/1_stateful/00067_union_all.reference @@ -1,20 +1,20 @@ -5994089094820940230 2 -5994089094820940230 2 -5994090037056589214 2 -5994090037056589214 2 -5994090037056589214 2 -5994090037056589214 2 -5994090037056589214 2 -5994090037056589214 2 -5994090526165207301 2 -5994090526165207301 2 -18442957653752051221 1 -18442957653752051221 1 -18442957653752051221 1 -18442957653752051221 1 -18443101109984791817 1 -18443101109984791817 1 -18443101109984791817 1 -18443101109984791817 1 -18443935406854847385 1 -18443935406854847385 1 +9223313629443448687 2 +9223313629443448687 2 +9223313629443448687 2 +9223313629443448687 2 +9223313629443448687 2 +9223313629443448687 2 +9223313629443448687 2 +9223313629443448687 2 +9223313629443448687 2 +9223313629443448687 2 +18442957653454441539 1 +18442957653454441539 1 +18442957653454441539 1 +18442957653454441539 1 +18443101109657095235 1 +18443101109657095235 1 +18443101109657095235 1 +18443101109657095235 1 +18443935407054343235 1 +18443935407054343235 1 diff --git a/dbms/tests/queries/1_stateful/00068_subquery_in_prewhere.reference b/dbms/tests/queries/1_stateful/00068_subquery_in_prewhere.reference index 90f1ab13794..0b1b5401c62 100644 --- a/dbms/tests/queries/1_stateful/00068_subquery_in_prewhere.reference +++ b/dbms/tests/queries/1_stateful/00068_subquery_in_prewhere.reference @@ -1 +1 @@ -66498 +1956422 diff --git a/dbms/tests/queries/1_stateful/00068_subquery_in_prewhere.sql b/dbms/tests/queries/1_stateful/00068_subquery_in_prewhere.sql index d84bacbf692..9e26c29c615 100644 --- a/dbms/tests/queries/1_stateful/00068_subquery_in_prewhere.sql +++ b/dbms/tests/queries/1_stateful/00068_subquery_in_prewhere.sql @@ -1 +1 @@ -SELECT count() FROM test.hits PREWHERE UserID IN (SELECT UserID FROM test.hits WHERE CounterID = 101500); +SELECT count() FROM test.hits PREWHERE UserID IN (SELECT UserID FROM test.hits WHERE CounterID = 800784); diff --git a/dbms/tests/queries/1_stateful/00069_duplicate_aggregation_keys.reference b/dbms/tests/queries/1_stateful/00069_duplicate_aggregation_keys.reference index 4fee49b4716..55eef5346ba 100644 --- a/dbms/tests/queries/1_stateful/00069_duplicate_aggregation_keys.reference +++ b/dbms/tests/queries/1_stateful/00069_duplicate_aggregation_keys.reference @@ -1,52 +1,222 @@ -http://images.yandex.ru/yandsearch?text=картинки ракеты в космосе&stype=image&lr=35&noreask=1&source=wiz 2014-03-18 http://images.yandex.ru/yandsearch?text=картинки ракеты в космосе&stype=image&lr=35&noreask=1&source=wiz -http://images.yandex.ru/yandsearch?text=планеты картинки&img_url=http:%2F%2Fcomments.ua%2Fimg%2F20110328143805.jpg&pos=7&rpt=simage&lr=35&noreask=1&source=wiz 2014-03-18 http://images.yandex.ru/yandsearch?text=планеты картинки&img_url=http:%2F%2Fcomments.ua%2Fimg%2F20110328143805.jpg&pos=7&rpt=simage&lr=35&noreask=1&source=wiz -http://images.yandex.ru/yandsearch?text=планеты картинки&stype=image&lr=35&noreask=1&source=wiz 2014-03-18 http://images.yandex.ru/yandsearch?text=планеты картинки&stype=image&lr=35&noreask=1&source=wiz -http://images.yandex.ru/yandsearch?text=раскраски для мальчиков&stype=image&lr=35&noreask=1&source=wiz 2014-03-20 http://images.yandex.ru/yandsearch?text=раскраски для мальчиков&stype=image&lr=35&noreask=1&source=wiz -http://images.yandex.ru/yandsearch?text=русская народная ярмарка картинки&img_url=http:%2F%2Fcs405226.userapi.com%2Fv405226882%2F2629%2Fr7DF4nIozfo.jpg&pos=1&rpt=simage&lr=35&noreask=1&source=wiz 2014-03-18 http://images.yandex.ru/yandsearch?text=русская народная ярмарка картинки&img_url=http:%2F%2Fcs405226.userapi.com%2Fv405226882%2F2629%2Fr7DF4nIozfo.jpg&pos=1&rpt=simage&lr=35&noreask=1&source=wiz -http://images.yandex.ru/yandsearch?text=русская народная ярмарка картинки&stype=image&lr=35&noreask=1&source=wiz 2014-03-18 http://images.yandex.ru/yandsearch?text=русская народная ярмарка картинки&stype=image&lr=35&noreask=1&source=wiz -http://images.yandex.ru/yandsearch?text=русская народная ярмарка рисунки&stype=image&lr=35&noreask=1&source=wiz 2014-03-18 http://images.yandex.ru/yandsearch?text=русская народная ярмарка рисунки&stype=image&lr=35&noreask=1&source=wiz -http://images.yandex.ru/yandsearch?text=ярмарка картинки рисунки&stype=image&lr=35&noreask=1&source=wiz 2014-03-18 http://images.yandex.ru/yandsearch?text=ярмарка картинки рисунки&stype=image&lr=35&noreask=1&source=wiz -http://images.yandex.ru/yandsearch?text=ярмарка картинки&stype=image&lr=35&noreask=1&source=wiz 2014-03-18 http://images.yandex.ru/yandsearch?text=ярмарка картинки&stype=image&lr=35&noreask=1&source=wiz -http://yabs.yandex.ru/count/9Fi4y9pf9Uu40000ZhiZrp85KfK1cm9kGxS198Yrp0QG1Oco2fLM0PYCAPshshu6fbYAfsmLrBs-gZmA0Qe1fQoMomAD0P6tm7fG0O-vvR6F0f-yMKpz2fCZcGL2Z90r3A2Gc3gla3KCb9229AUUNYoee5W86AIm0000WQx-3nB9IZGmzGIn0RA04Bchshu6k_cm1WlFsxvC0duH 2014-03-22 http://yabs.yandex.ru/count/9Fi4y9pf9Uu40000ZhiZrp85KfK1cm9kGxS198Yrp0QG1Oco2fLM0PYCAPshshu6fbYAfsmLrBs-gZmA0Qe1fQoMomAD0P6tm7fG0O-vvR6F0f-yMKpz2fCZcGL2Z90r3A2Gc3gla3KCb9229AUUNYoee5W86AIm0000WQx-3nB9IZGmzGIn0RA04Bchshu6k_cm1WlFsxvC0duH -http://yandex.ru/sitesearch?searchid=1887792&100n=ru&reqenc=&text=&Submit=Поиск 2014-03-22 http://yandex.ru/sitesearch?searchid=1887792&100n=ru&reqenc=&text=&Submit=Поиск -http://yandex.ru/yandsearch?lr=35&msid=20933.19395.1395338687.75786&text=гдз 2014-03-20 http://yandex.ru/yandsearch?lr=35&msid=20933.19395.1395338687.75786&text=гдз -http://yandex.ru/yandsearch?lr=35&msid=20943.9035.1395477291.50774&text=шуточная клятва пенсионера женщины 2014-03-22 http://yandex.ru/yandsearch?lr=35&msid=20943.9035.1395477291.50774&text=шуточная клятва пенсионера женщины -http://yandex.ru/yandsearch?lr=35&msid=20953.3223.1395159223.1527&text=смайлики вк скрытые 2014-03-18 http://yandex.ru/yandsearch?lr=35&msid=20953.3223.1395159223.1527&text=смайлики вк скрытые -http://yandex.ru/yandsearch?lr=35&msid=22871.10212.1395165538.52533&text=русская народная ярмарка картинки 2014-03-18 http://yandex.ru/yandsearch?lr=35&msid=22871.10212.1395165538.52533&text=русская народная ярмарка картинки -http://yandex.ru/yandsearch?lr=35&msid=22876.29908.1395165501.19017&text=русская народная ярмарка картинки 2014-03-18 http://yandex.ru/yandsearch?lr=35&msid=22876.29908.1395165501.19017&text=русская народная ярмарка картинки -http://yandex.ru/yandsearch?lr=35&msid=22878.24291.1395333478.42148&text=7лепестков 2014-03-20 http://yandex.ru/yandsearch?lr=35&msid=22878.24291.1395333478.42148&text=7лепестков -http://yandex.ru/yandsearch?lr=35&msid=22885.24948.1395159598.71259&text=пессимист 2014-03-18 http://yandex.ru/yandsearch?lr=35&msid=22885.24948.1395159598.71259&text=пессимист -http://yandex.ru/yandsearch?lr=35&msid=22889.4244.1395162598.65317&text=планеты картинки 2014-03-18 http://yandex.ru/yandsearch?lr=35&msid=22889.4244.1395162598.65317&text=планеты картинки -http://yandex.ru/yandsearch?lr=35&msid=22889.4252.1395165264.65698&text=ярмарка картинки 2014-03-18 http://yandex.ru/yandsearch?lr=35&msid=22889.4252.1395165264.65698&text=ярмарка картинки -http://yandex.ru/yandsearch?lr=35&msid=22894.2811.1395171040.46029&text=планеты картинки 2014-03-18 http://yandex.ru/yandsearch?lr=35&msid=22894.2811.1395171040.46029&text=планеты картинки -http://yandex.ru/yandsearch?lr=35&msid=22895.12503.1395323042.95018&text=раскраски для мальчиков 2014-03-20 http://yandex.ru/yandsearch?lr=35&msid=22895.12503.1395323042.95018&text=раскраски для мальчиков -http://yandex.ru/yandsearch?p=1&text=прикольные тосты с выходом на пенсию женщине в прозе&lr=35 2014-03-22 http://yandex.ru/yandsearch?p=1&text=прикольные тосты с выходом на пенсию женщине в прозе&lr=35 -http://yandex.ru/yandsearch?p=1&text=сценарий проводов на пенсию женщины&lr=35 2014-03-22 http://yandex.ru/yandsearch?p=1&text=сценарий проводов на пенсию женщины&lr=35 -http://yandex.ru/yandsearch?p=1&text=шуточные вопросы и ответы для гостей&lr=35 2014-03-22 http://yandex.ru/yandsearch?p=1&text=шуточные вопросы и ответы для гостей&lr=35 -http://yandex.ru/yandsearch?p=1&text=шуточные песни к выходу на пенсию&lr=35 2014-03-22 http://yandex.ru/yandsearch?p=1&text=шуточные песни к выходу на пенсию&lr=35 -http://yandex.ru/yandsearch?p=1&text=шуточные песни-переделки к выходу на пенсию женщине скачать&lr=35 2014-03-22 http://yandex.ru/yandsearch?p=1&text=шуточные песни-переделки к выходу на пенсию женщине скачать&lr=35 -http://yandex.ru/yandsearch?p=1&text=шуточные тосты с выходом на пенсию женщине&lr=35 2014-03-22 http://yandex.ru/yandsearch?p=1&text=шуточные тосты с выходом на пенсию женщине&lr=35 -http://yandex.ru/yandsearch?p=2&text=прикольные тосты с выходом на пенсию женщине в прозе&lr=35 2014-03-22 http://yandex.ru/yandsearch?p=2&text=прикольные тосты с выходом на пенсию женщине в прозе&lr=35 -http://yandex.ru/yandsearch?p=2&text=сценарий проводов на пенсию женщины&lr=35 2014-03-22 http://yandex.ru/yandsearch?p=2&text=сценарий проводов на пенсию женщины&lr=35 -http://yandex.ru/yandsearch?text=Seemon – Играли нервы, летели ножи текст песни&lr=35 2014-03-18 http://yandex.ru/yandsearch?text=Seemon – Играли нервы, летели ножи текст песни&lr=35 -http://yandex.ru/yandsearch?text=Seemon – Играли нервы, летели ножи&lr=35 2014-03-18 http://yandex.ru/yandsearch?text=Seemon – Играли нервы, летели ножи&lr=35 -http://yandex.ru/yandsearch?text=картинки ракеты в космосе&lr=35 2014-03-18 http://yandex.ru/yandsearch?text=картинки ракеты в космосе&lr=35 -http://yandex.ru/yandsearch?text=картинки ракеты для детей&lr=35 2014-03-18 http://yandex.ru/yandsearch?text=картинки ракеты для детей&lr=35 -http://yandex.ru/yandsearch?text=пессимизм это&lr=35 2014-03-18 http://yandex.ru/yandsearch?text=пессимизм это&lr=35 -http://yandex.ru/yandsearch?text=пессимист и оптимист&lr=35 2014-03-18 http://yandex.ru/yandsearch?text=пессимист и оптимист&lr=35 -http://yandex.ru/yandsearch?text=прикольные тосты с выходом на пенсию женщине в прозе&lr=35 2014-03-22 http://yandex.ru/yandsearch?text=прикольные тосты с выходом на пенсию женщине в прозе&lr=35 -http://yandex.ru/yandsearch?text=прикольные тосты с выходом на пенсию женщине&lr=35 2014-03-22 http://yandex.ru/yandsearch?text=прикольные тосты с выходом на пенсию женщине&lr=35 -http://yandex.ru/yandsearch?text=русская народная ярмарка картинки&lr=35 2014-03-18 http://yandex.ru/yandsearch?text=русская народная ярмарка картинки&lr=35 -http://yandex.ru/yandsearch?text=русская народная ярмарка рисунки &lr=35 2014-03-18 http://yandex.ru/yandsearch?text=русская народная ярмарка рисунки &lr=35 -http://yandex.ru/yandsearch?text=смайлики вк скрытые&lr=35 2014-03-18 http://yandex.ru/yandsearch?text=смайлики вк скрытые&lr=35 -http://yandex.ru/yandsearch?text=сценарий проводов на пенсию женщины от подруг дома&lr=35 2014-03-22 http://yandex.ru/yandsearch?text=сценарий проводов на пенсию женщины от подруг дома&lr=35 -http://yandex.ru/yandsearch?text=сценарий проводов на пенсию женщины&lr=35 2014-03-22 http://yandex.ru/yandsearch?text=сценарий проводов на пенсию женщины&lr=35 -http://yandex.ru/yandsearch?text=шуточные вопросы и ответы для гостей&lr=35 2014-03-22 http://yandex.ru/yandsearch?text=шуточные вопросы и ответы для гостей&lr=35 -http://yandex.ru/yandsearch?text=шуточные дипломы к выходу на пенсию женщине&lr=35 2014-03-22 http://yandex.ru/yandsearch?text=шуточные дипломы к выходу на пенсию женщине&lr=35 -http://yandex.ru/yandsearch?text=шуточные дипломы к выходу на пенсию&lr=35 2014-03-22 http://yandex.ru/yandsearch?text=шуточные дипломы к выходу на пенсию&lr=35 -http://yandex.ru/yandsearch?text=шуточные песни к выходу на пенсию женщине скачать&lr=35 2014-03-22 http://yandex.ru/yandsearch?text=шуточные песни к выходу на пенсию женщине скачать&lr=35 -http://yandex.ru/yandsearch?text=шуточные песни к выходу на пенсию&lr=35 2014-03-22 http://yandex.ru/yandsearch?text=шуточные песни к выходу на пенсию&lr=35 -http://yandex.ru/yandsearch?text=шуточные песни-переделки к выходу на пенсию женщине скачать&lr=35 2014-03-22 http://yandex.ru/yandsearch?text=шуточные песни-переделки к выходу на пенсию женщине скачать&lr=35 -http://yandex.ru/yandsearch?text=шуточные поздравления с выходом на пенсию женщине&lr=35 2014-03-22 http://yandex.ru/yandsearch?text=шуточные поздравления с выходом на пенсию женщине&lr=35 -http://yandex.ru/yandsearch?text=шуточные тосты с выходом на пенсию женщине&lr=35 2014-03-22 http://yandex.ru/yandsearch?text=шуточные тосты с выходом на пенсию женщине&lr=35 -http://yandex.ru/yandsearch?text=ярмарка картинки рисунки&lr=35 2014-03-18 http://yandex.ru/yandsearch?text=ярмарка картинки рисунки&lr=35 +http://auto.ru/chatay-barana.ru/traction.html#maybettaya 2014-03-17 http://auto.ru/chatay-barana.ru/traction.html#maybettaya +http://auto.ru/chatay-barana.ru/traction.html#maybettaya 2014-03-18 http://auto.ru/chatay-barana.ru/traction.html#maybettaya +http://auto.ru/chatay-barana.ru/traction.html#maybettaya 2014-03-19 http://auto.ru/chatay-barana.ru/traction.html#maybettaya +http://auto.ru/chatay-barana.ru/traction.html#maybettaya 2014-03-20 http://auto.ru/chatay-barana.ru/traction.html#maybettaya +http://auto.ru/chatay-barana.ru/traction.html#maybettaya 2014-03-21 http://auto.ru/chatay-barana.ru/traction.html#maybettaya +http://auto.ru/chatay-barana.ru/traction.html#maybettaya 2014-03-22 http://auto.ru/chatay-barana.ru/traction.html#maybettaya +http://auto.ru/chatay-barana.ru/traction.html#maybettaya 2014-03-23 http://auto.ru/chatay-barana.ru/traction.html#maybettaya +http://auto.ru/chatay-baranta_bound-in-tanks.ru/forumyazan 2014-03-17 http://auto.ru/chatay-baranta_bound-in-tanks.ru/forumyazan +http://auto.ru/chatay-baranta_bound-in-tanks.ru/forumyazan 2014-03-18 http://auto.ru/chatay-baranta_bound-in-tanks.ru/forumyazan +http://auto.ru/chatay-baranta_bound-in-tanks.ru/forumyazan 2014-03-19 http://auto.ru/chatay-baranta_bound-in-tanks.ru/forumyazan +http://auto.ru/chatay-baranta_bound-in-tanks.ru/forumyazan 2014-03-20 http://auto.ru/chatay-baranta_bound-in-tanks.ru/forumyazan +http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny 2014-03-17 http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny +http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny 2014-03-18 http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny +http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny 2014-03-19 http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny +http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny 2014-03-20 http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny +http://auto.ru/chatay-baranta_bound-in-the-asfalt.ru/guru.app4smarta 2014-03-17 http://auto.ru/chatay-baranta_bound-in-the-asfalt.ru/guru.app4smarta +http://auto.ru/chatay-baranta_bound-in-the-asfalt.ru/guru.app4smarta 2014-03-18 http://auto.ru/chatay-baranta_bound-in-the-asfalt.ru/guru.app4smarta +http://auto.ru/chatay-baranta_bound-in-the-asfalt.ru/guru.app4smarta 2014-03-19 http://auto.ru/chatay-baranta_bound-in-the-asfalt.ru/guru.app4smarta +http://auto.ru/chatay-baranta_bound-in-the-asfalt.ru/guru.app4smarta 2014-03-20 http://auto.ru/chatay-baranta_bound-in-the-asfalt.ru/guru.app4smarta +http://auto.ru/chatay-baranta_bound-in-trannie/spb_eco_classnoe 2014-03-17 http://auto.ru/chatay-baranta_bound-in-trannie/spb_eco_classnoe +http://auto.ru/chatay-baranta_bound-in-trannie/spb_eco_classnoe 2014-03-18 http://auto.ru/chatay-baranta_bound-in-trannie/spb_eco_classnoe +http://auto.ru/chatay-baranta_bound-in-trannie/spb_eco_classnoe 2014-03-19 http://auto.ru/chatay-baranta_bound-in-trannie/spb_eco_classnoe +http://auto.ru/chatay-baranta_bound-in-trannie/spb_eco_classnoe 2014-03-20 http://auto.ru/chatay-baranta_bound-in-trannie/spb_eco_classnoe +http://auto.ru/chatay-bibolge.ru/yandex.ua/obyaschenshchennoktalahi 2014-03-17 http://auto.ru/chatay-bibolge.ru/yandex.ua/obyaschenshchennoktalahi +http://auto.ru/chatay-bibolge.ru/yandex.ua/obyaschenshchennoktalahi 2014-03-18 http://auto.ru/chatay-bibolge.ru/yandex.ua/obyaschenshchennoktalahi +http://auto.ru/chatay-bibolge.ru/yandex.ua/obyaschenshchennoktalahi 2014-03-19 http://auto.ru/chatay-bibolge.ru/yandex.ua/obyaschenshchennoktalahi +http://auto.ru/chatay-bibolge.ru/yandex.ua/obyaschenshchennoktalahi 2014-03-20 http://auto.ru/chatay-bibolge.ru/yandex.ua/obyaschenshchennoktalahi +http://auto.ru/chatay-bibolge.ru/yandex.ua/obyaschenshchennoktalahi 2014-03-21 http://auto.ru/chatay-bibolge.ru/yandex.ua/obyaschenshchennoktalahi +http://auto.ru/chatay-bibolge.ru/yandex.ua/obyaschenshchennoktalahi 2014-03-22 http://auto.ru/chatay-bibolge.ru/yandex.ua/obyaschenshchennoktalahi +http://auto.ru/chatay-bibolge.ru/yandex.ua/obyaschenshchennoktalahi 2014-03-23 http://auto.ru/chatay-bibolge.ru/yandex.ua/obyaschenshchennoktalahi +http://auto.ru/chatay-bit-gd-geli-bolshe__298682.13.xls&nails.aspx 2014-03-17 http://auto.ru/chatay-bit-gd-geli-bolshe__298682.13.xls&nails.aspx +http://auto.ru/chatay-bit-gd-geli-bolshe__298682.13.xls&nails.aspx 2014-03-18 http://auto.ru/chatay-bit-gd-geli-bolshe__298682.13.xls&nails.aspx +http://auto.ru/chatay-bit-gd-geli-bolshe__298682.13.xls&nails.aspx 2014-03-19 http://auto.ru/chatay-bit-gd-geli-bolshe__298682.13.xls&nails.aspx +http://auto.ru/chatay-bit-gd-geli-bolshe__298682.13.xls&nails.aspx 2014-03-20 http://auto.ru/chatay-bit-gd-geli-bolshe__298682.13.xls&nails.aspx +http://auto.ru/chatay-bit-gd-geli-bolshe__298682.13.xls&nails.aspx 2014-03-21 http://auto.ru/chatay-bit-gd-geli-bolshe__298682.13.xls&nails.aspx +http://auto.ru/chatay-bit-gd-geli-bolshe__298682.13.xls&nails.aspx 2014-03-22 http://auto.ru/chatay-bit-gd-geli-bolshe__298682.13.xls&nails.aspx +http://auto.ru/chatay-bit-gd-geli-bolshe__298682.13.xls&nails.aspx 2014-03-23 http://auto.ru/chatay-bit-gd-geli-bolshe__298682.13.xls&nails.aspx +http://auto.ru/chatay-bit-yz-odezhda/velik.hurriyet 2014-03-17 http://auto.ru/chatay-bit-yz-odezhda/velik.hurriyet +http://auto.ru/chatay-bit-yz-odezhda/velik.hurriyet 2014-03-18 http://auto.ru/chatay-bit-yz-odezhda/velik.hurriyet +http://auto.ru/chatay-bit-yz-odezhda/velik.hurriyet 2014-03-19 http://auto.ru/chatay-bit-yz-odezhda/velik.hurriyet +http://auto.ru/chatay-bit-yz-odezhda/velik.hurriyet 2014-03-20 http://auto.ru/chatay-bit-yz-odezhda/velik.hurriyet +http://auto.ru/chatay-bit-yz-odezhda/vereya/dlike4u 2014-03-17 http://auto.ru/chatay-bit-yz-odezhda/vereya/dlike4u +http://auto.ru/chatay-bit-yz-odezhda/vereya/dlike4u 2014-03-18 http://auto.ru/chatay-bit-yz-odezhda/vereya/dlike4u +http://auto.ru/chatay-bit-yz-odezhda/vereya/dlike4u 2014-03-19 http://auto.ru/chatay-bit-yz-odezhda/vereya/dlike4u +http://auto.ru/chatay-bit-yz-odezhda/vereya/dlike4u 2014-03-20 http://auto.ru/chatay-bit-yz-odezhda/vereya/dlike4u +http://auto.ru/chatay-bit-yz-odezhda/verhnya/bankovod 2014-03-17 http://auto.ru/chatay-bit-yz-odezhda/verhnya/bankovod +http://auto.ru/chatay-bit-yz-odezhda/verhnya/bankovod 2014-03-18 http://auto.ru/chatay-bit-yz-odezhda/verhnya/bankovod +http://auto.ru/chatay-bit-yz-odezhda/verhnya/bankovod 2014-03-19 http://auto.ru/chatay-bit-yz-odezhda/verhnya/bankovod +http://auto.ru/chatay-bit-yz-odezhda/verhnya/bankovod 2014-03-20 http://auto.ru/chatay-bit-yz-odezhda/verhnya/bankovod +http://auto.ru/chatay-bit-yz-odezhda/versonal/5/sego 2014-03-17 http://auto.ru/chatay-bit-yz-odezhda/versonal/5/sego +http://auto.ru/chatay-bit-yz-odezhda/versonal/5/sego 2014-03-18 http://auto.ru/chatay-bit-yz-odezhda/versonal/5/sego +http://auto.ru/chatay-bit-yz-odezhda/versonal/5/sego 2014-03-19 http://auto.ru/chatay-bit-yz-odezhda/versonal/5/sego +http://auto.ru/chatay-bit-yz-odezhda/versonal/5/sego 2014-03-20 http://auto.ru/chatay-bit-yz-odezhda/versonal/5/sego +http://auto.ru/chatay-john-Den-You-TheAgence=&commemchurkiyede 2014-03-17 http://auto.ru/chatay-john-Den-You-TheAgence=&commemchurkiyede +http://auto.ru/chatay-john-Den-You-TheAgence=&commemchurkiyede 2014-03-18 http://auto.ru/chatay-john-Den-You-TheAgence=&commemchurkiyede +http://auto.ru/chatay-john-Den-You-TheAgence=&commemchurkiyede 2014-03-19 http://auto.ru/chatay-john-Den-You-TheAgence=&commemchurkiyede +http://auto.ru/chatay-john-Den-You-TheAgence=&commemchurkiyede 2014-03-20 http://auto.ru/chatay-john-Den-You-TheAgence=&commemchurkiyede +http://auto.ru/chatay-john-Den-You-TheAgents/c1303954 2014-03-17 http://auto.ru/chatay-john-Den-You-TheAgents/c1303954 +http://auto.ru/chatay-john-Den-You-TheAgents/c1303954 2014-03-18 http://auto.ru/chatay-john-Den-You-TheAgents/c1303954 +http://auto.ru/chatay-john-Den-You-TheAgents/c1303954 2014-03-19 http://auto.ru/chatay-john-Den-You-TheAgents/c1303954 +http://auto.ru/chatay-john-Den-You-TheAgents/c1303954 2014-03-20 http://auto.ru/chatay-john-Den-You-TheAgents/c1303954 +http://auto.ru/chatay-john-Den-Yunan-rasskRollback 2014-03-17 http://auto.ru/chatay-john-Den-Yunan-rasskRollback +http://auto.ru/chatay-john-Den-Yunan-rasskRollback 2014-03-18 http://auto.ru/chatay-john-Den-Yunan-rasskRollback +http://auto.ru/chatay-john-Den-Yunan-rasskRollback 2014-03-19 http://auto.ru/chatay-john-Den-Yunan-rasskRollback +http://auto.ru/chatay-john-Den-Yunan-rasskRollback 2014-03-20 http://auto.ru/chatay-john-Den-Yunan-rasskRollback +http://auto.ru/chatay-john-Den-Yunanija.com%2F20 2014-03-22 http://auto.ru/chatay-john-Den-Yunanija.com%2F20 +http://auto.ru/chatay-john-Den-Yunanija.com%2F20 2014-03-23 http://auto.ru/chatay-john-Den-Yunanija.com%2F20 +http://auto.ru/chatay-john-Den-Yunanija.com.tr&callback 2014-03-17 http://auto.ru/chatay-john-Den-Yunanija.com.tr&callback +http://auto.ru/chatay-john-Den-Yunanija.com.tr&callback 2014-03-18 http://auto.ru/chatay-john-Den-Yunanija.com.tr&callback +http://auto.ru/chatay-john-Den-Yunanija.com.tr&callback 2014-03-19 http://auto.ru/chatay-john-Den-Yunanija.com.tr&callback +http://auto.ru/chatay-john-Den-Yunanija.com.tr&callback 2014-03-20 http://auto.ru/chatay-john-Den-Yunanija.com.tr&callback +http://auto.ru/chatay-john-Den-Yunanija.com.tr&user 2014-03-17 http://auto.ru/chatay-john-Den-Yunanija.com.tr&user +http://auto.ru/chatay-john-Den-Yunanija.com.tr&user 2014-03-18 http://auto.ru/chatay-john-Den-Yunanija.com.tr&user +http://auto.ru/chatay-john-Den-Yunanija.com.tr&user 2014-03-19 http://auto.ru/chatay-john-Den-Yunanija.com.tr&user +http://auto.ru/chatay-john-Den-Yunanija.com.tr&user 2014-03-20 http://auto.ru/chatay-john-Den-Yunanija.com.tr&user +http://auto.ru/chatay-john-Den-Yunanija.com.tr&useriya 2014-03-23 http://auto.ru/chatay-john-Den-Yunanija.com.tr&useriya +http://auto.ru/chatay-john-Den-Yunanija.com.tr/oauth 2014-03-18 http://auto.ru/chatay-john-Den-Yunanija.com.tr/oauth +http://auto.ru/chatay-john-Den-Yunanija.com.tr/oauth 2014-03-19 http://auto.ru/chatay-john-Den-Yunanija.com.tr/oauth +http://auto.ru/chatay-john-Den-Yunanija.com.tr/oauth 2014-03-20 http://auto.ru/chatay-john-Den-Yunanija.com.tr/oauth +http://auto.ru/chatay-john-Den-Yunanija.com.ua/ukraina 2014-03-17 http://auto.ru/chatay-john-Den-Yunanija.com.ua/ukraina +http://auto.ru/chatay-john-Den-Yunanija.com.ua/ukraina 2014-03-18 http://auto.ru/chatay-john-Den-Yunanija.com.ua/ukraina +http://auto.ru/chatay-john-Den-Yunanija.com.ua/ukraina 2014-03-19 http://auto.ru/chatay-john-Den-Yunanija.com.ua/ukraina +http://auto.ru/chatay-john-Den-Yunanija.com.ua/ukraina 2014-03-20 http://auto.ru/chatay-john-Den-Yunanija.com.ua/ukraina +http://auto.ru/chatay-john-Den-Yunanija.com.ua/yandex 2014-03-17 http://auto.ru/chatay-john-Den-Yunanija.com.ua/yandex +http://auto.ru/chatay-john-Den-Yunanija.com.ua/yandex 2014-03-18 http://auto.ru/chatay-john-Den-Yunanija.com.ua/yandex +http://auto.ru/chatay-john-Den-Yunanija.com.ua/yandex 2014-03-19 http://auto.ru/chatay-john-Den-Yunanija.com.ua/yandex +http://auto.ru/chatay-john-Den-Yunanija.com.ua/yandex 2014-03-20 http://auto.ru/chatay-john-Den-Yunanija.com.ua/yandex +http://auto.ru/chatay-john-Den-Yunanija.com/commetrovies 2014-03-23 http://auto.ru/chatay-john-Den-Yunanija.com/commetrovies +http://auto.ru/chatay-john-Den-Yunanija.com/obyavlenie 2014-03-19 http://auto.ru/chatay-john-Den-Yunanija.com/obyavlenie +http://auto.ru/chatay-john-Den-Yunanija.com/obyavlenie 2014-03-20 http://auto.ru/chatay-john-Den-Yunanija.com/obyavlenie +http://auto.ru/chatay-john-Den-Yunanija.html#medictengliyskom 2014-03-17 http://auto.ru/chatay-john-Den-Yunanija.html#medictengliyskom +http://auto.ru/chatay-john-Den-Yunanija.html#medictengliyskom 2014-03-18 http://auto.ru/chatay-john-Den-Yunanija.html#medictengliyskom +http://auto.ru/chatay-john-Den-Yunanija.html#medictengliyskom 2014-03-19 http://auto.ru/chatay-john-Den-Yunanija.html#medictengliyskom +http://auto.ru/chatay-john-Den-Yunanija.html#medictengliyskom 2014-03-20 http://auto.ru/chatay-john-Den-Yunanija.html#medictengliyskom +http://auto.ru/chatay-john-Den-Yunanija.html/photos 2014-03-18 http://auto.ru/chatay-john-Den-Yunanija.html/photos +http://auto.ru/chatay-john-Den-Yunanija.html/photos 2014-03-19 http://auto.ru/chatay-john-Den-Yunanija.html/photos +http://auto.ru/chatay-john-Den-Yunanija.html/photos 2014-03-20 http://auto.ru/chatay-john-Den-Yunanija.html/photos +http://auto.ru/chatay-john-Den-Yunanija.html/photos 2014-03-21 http://auto.ru/chatay-john-Den-Yunanija.html/photos +http://auto.ru/chatay-john-Den-Yunanija.html/photos 2014-03-22 http://auto.ru/chatay-john-Den-Yunanija.html/photos +http://auto.ru/chatay-john-Den-Yunanija.html?area 2014-03-17 http://auto.ru/chatay-john-Den-Yunanija.html?area +http://auto.ru/chatay-john-Den-Yunanija.html?area 2014-03-18 http://auto.ru/chatay-john-Den-Yunanija.html?area +http://auto.ru/chatay-john-Den-Yunanija.html?area 2014-03-19 http://auto.ru/chatay-john-Den-Yunanija.html?area +http://auto.ru/chatay-john-Den-Yunanija.html?area 2014-03-20 http://auto.ru/chatay-john-Den-Yunanija.html?area +http://auto.ru/chatay-john-Den-Yunanija.html?area 2014-03-21 http://auto.ru/chatay-john-Den-Yunanija.html?area +http://auto.ru/chatay-john-Den-Yunanija.html?area 2014-03-22 http://auto.ru/chatay-john-Den-Yunanija.html?area +http://auto.ru/chatay-john-Den-Yunanija.html?period 2014-03-17 http://auto.ru/chatay-john-Den-Yunanija.html?period +http://auto.ru/chatay-john-Den-Yunanija.html?period 2014-03-18 http://auto.ru/chatay-john-Den-Yunanija.html?period +http://auto.ru/chatay-john-Den-Yunanija.html?period 2014-03-19 http://auto.ru/chatay-john-Den-Yunanija.html?period +http://auto.ru/chatay-john-Den-Yunanija.html?period 2014-03-20 http://auto.ru/chatay-john-Den-Yunanija.html?period +http://auto.ru/chatay-john-Den-Yunanija.html?pid 2014-03-17 http://auto.ru/chatay-john-Den-Yunanija.html?pid +http://auto.ru/chatay-john-Den-Yunanija.html?pid 2014-03-18 http://auto.ru/chatay-john-Den-Yunanija.html?pid +http://auto.ru/chatay-john-Den-Yunanija.html?pid 2014-03-19 http://auto.ru/chatay-john-Den-Yunanija.html?pid +http://auto.ru/chatay-john-Den-Yunanija.html?pid 2014-03-20 http://auto.ru/chatay-john-Den-Yunanija.html?pid +http://auto.ru/chatay-john-Den-Yunanija.html?rtn 2014-03-17 http://auto.ru/chatay-john-Den-Yunanija.html?rtn +http://auto.ru/chatay-john-Den-Yunanija.html?rtn 2014-03-18 http://auto.ru/chatay-john-Den-Yunanija.html?rtn +http://auto.ru/chatay-john-Den-Yunanija.html?rtn 2014-03-19 http://auto.ru/chatay-john-Den-Yunanija.html?rtn +http://auto.ru/chatay-john-Den-Yunanija.html?rtn 2014-03-20 http://auto.ru/chatay-john-Den-Yunanija.html?rtn +http://auto.ru/chatay-uchits/sem-yzyil-sony-osta.coccoc.com 2014-03-17 http://auto.ru/chatay-uchits/sem-yzyil-sony-osta.coccoc.com +http://auto.ru/chatay-uchits/sem-yzyil-sony-osta.coccoc.com 2014-03-18 http://auto.ru/chatay-uchits/sem-yzyil-sony-osta.coccoc.com +http://auto.ru/chatay-uchits/sem-yzyil-sony-osta.coccoc.com 2014-03-19 http://auto.ru/chatay-uchits/sem-yzyil-sony-osta.coccoc.com +http://auto.ru/chatay-uchits/sem-yzyil-sony-osta.coccoc.com 2014-03-20 http://auto.ru/chatay-uchits/sem-yzyil-sony-osta.coccoc.com +http://auto.ru/chatay-uchits/sem-yzyil-sony-osta.coccoc.com 2014-03-21 http://auto.ru/chatay-uchits/sem-yzyil-sony-osta.coccoc.com +http://auto.ru/chatay-uchits/sem-yzyil-sony-osta.coccoc.com 2014-03-22 http://auto.ru/chatay-uchits/sem-yzyil-sony-osta.coccoc.com +http://auto.ru/chatay-uchits/sem-yzyil-sony-osta.coccoc.com 2014-03-23 http://auto.ru/chatay-uchits/sem-yzyil-sony-osta.coccoc.com +http://auto.ru/chatay-uchits/thegames/page=http: 2014-03-17 http://auto.ru/chatay-uchits/thegames/page=http: +http://auto.ru/chatay-uchits/thegames/page=http: 2014-03-18 http://auto.ru/chatay-uchits/thegames/page=http: +http://auto.ru/chatay-uchits/thegames/page=http: 2014-03-19 http://auto.ru/chatay-uchits/thegames/page=http: +http://auto.ru/chatay-uchits/thegames/page=http: 2014-03-20 http://auto.ru/chatay-uchits/thegames/page=http: +http://auto.ru/chatay-uchits/thegames/page=https 2014-03-17 http://auto.ru/chatay-uchits/thegames/page=https +http://auto.ru/chatay-uchits/thegames/page=https 2014-03-18 http://auto.ru/chatay-uchits/thegames/page=https +http://auto.ru/chatay-uchits/thegames/page=https 2014-03-19 http://auto.ru/chatay-uchits/thegames/page=https +http://auto.ru/chatay-uchits/thegames/page=https 2014-03-20 http://auto.ru/chatay-uchits/thegames/page=https +http://auto.ru/chatay-uchits/thegames/pass.com.ua 2014-03-18 http://auto.ru/chatay-uchits/thegames/pass.com.ua +http://auto.ru/chatay-uchits/thegames/pass.com.ua 2014-03-19 http://auto.ru/chatay-uchits/thegames/pass.com.ua +http://auto.ru/chatay-uchits/thegames/pass.com.ua 2014-03-20 http://auto.ru/chatay-uchits/thegames/pass.com.ua +http://auto.ru/chatay-uchits/thegames/pass.com/effers 2014-03-17 http://auto.ru/chatay-uchits/thegames/pass.com/effers +http://auto.ru/chatay-uchits/thegames/pass.com/effers 2014-03-18 http://auto.ru/chatay-uchits/thegames/pass.com/effers +http://auto.ru/chatay-uchits/thegames/pass.com/effers 2014-03-19 http://auto.ru/chatay-uchits/thegames/pass.com/effers +http://auto.ru/chatay-uchits/thegames/pass.com/effers 2014-03-20 http://auto.ru/chatay-uchits/thegames/pass.com/effers +http://auto.ru/chatay-uchits/thegames/pass.com/iframe 2014-03-18 http://auto.ru/chatay-uchits/thegames/pass.com/iframe +http://auto.ru/chatay-uchits/thegames/pass.com/iframe 2014-03-19 http://auto.ru/chatay-uchits/thegames/pass.com/iframe +http://auto.ru/chatay-uchits/thegames/pass.com/iframe 2014-03-20 http://auto.ru/chatay-uchits/thegames/pass.com/iframe +http://auto.ru/chatay-uchits/thegames/pass.com/iframe 2014-03-22 http://auto.ru/chatay-uchits/thegames/pass.com/iframe +http://auto.ru/chatay-uchits/thegames/pass.com/iframe 2014-03-23 http://auto.ru/chatay-uchits/thegames/pass.com/iframe +http://auto.ru/chatay-uchits/thegames/pass.com/istonomicha 2014-03-23 http://auto.ru/chatay-uchits/thegames/pass.com/istonomicha +http://auto.ru/chatay-uchits/thegames/pass.com/tovaja 2014-03-23 http://auto.ru/chatay-uchits/thegames/pass.com/tovaja +http://auto.ru/chatay-uchits/thegames/password/?type 2014-03-22 http://auto.ru/chatay-uchits/thegames/password/?type +http://auto.ru/chatay-uchits/thegames/password=&to 2014-03-21 http://auto.ru/chatay-uchits/thegames/password=&to +http://auto.ru/chatay-uchits/thegames/password=&to 2014-03-22 http://auto.ru/chatay-uchits/thegames/password=&to +http://auto.ru/chatay-uchits/thegames/passwords= 2014-03-22 http://auto.ru/chatay-uchits/thegames/passwords= +http://auto.ru/chatay-uchits/thegames/passwords=Поддержитель 2014-03-22 http://auto.ru/chatay-uchits/thegames/passwords=Поддержитель +http://auto.ru/chatay-uchits/thegames/passwordsPerPage 2014-03-22 http://auto.ru/chatay-uchits/thegames/passwordsPerPage +http://auto.ru/chatay-uchits/thegames/passwords[0 2014-03-22 http://auto.ru/chatay-uchits/thegames/passwords[0 +http://auto.ru/chatay-uchits/thegames/passwordstrelation 2014-03-22 http://auto.ru/chatay-uchits/thegames/passwordstrelation +http://auto.ru/chatay-uchits/thegames/public/?hash 2014-03-17 http://auto.ru/chatay-uchits/thegames/public/?hash +http://auto.ru/chatay-uchits/thegames/public/?hash 2014-03-18 http://auto.ru/chatay-uchits/thegames/public/?hash +http://auto.ru/chatay-uchits/thegames/public/?hash 2014-03-19 http://auto.ru/chatay-uchits/thegames/public/?hash +http://auto.ru/chatay-uchits/thegames/public/?hash 2014-03-20 http://auto.ru/chatay-uchits/thegames/public/?hash +http://auto.ru/chatay-uchits/thegames/puted_facet 2014-03-17 http://auto.ru/chatay-uchits/thegames/puted_facet +http://auto.ru/chatay-uchits/thegames/puted_facet 2014-03-18 http://auto.ru/chatay-uchits/thegames/puted_facet +http://auto.ru/chatay-uchits/thegames/puted_facet 2014-03-19 http://auto.ru/chatay-uchits/thegames/puted_facet +http://auto.ru/chatay-uchits/thegames/puted_facet 2014-03-20 http://auto.ru/chatay-uchits/thegames/puted_facet +http://auto.ru/chatay-uchits/thegames/puted_from 2014-03-17 http://auto.ru/chatay-uchits/thegames/puted_from +http://auto.ru/chatay-uchits/thegames/puted_from 2014-03-18 http://auto.ru/chatay-uchits/thegames/puted_from +http://auto.ru/chatay-uchits/thegames/puted_from 2014-03-19 http://auto.ru/chatay-uchits/thegames/puted_from +http://auto.ru/chatay-uchits/thegames/puted_from 2014-03-20 http://auto.ru/chatay-uchits/thegames/puted_from +http://auto.ru/chatay-uchits/thegames/putery/Roman 2014-03-17 http://auto.ru/chatay-uchits/thegames/putery/Roman +http://auto.ru/chatay-uchits/thegames/putery/Roman 2014-03-18 http://auto.ru/chatay-uchits/thegames/putery/Roman +http://auto.ru/chatay-uchits/thegames/putery/Roman 2014-03-19 http://auto.ru/chatay-uchits/thegames/putery/Roman +http://auto.ru/chatay-uchits/thegames/putery/Roman 2014-03-20 http://auto.ru/chatay-uchits/thegames/putery/Roman +http://auto.ru/chatay-uchits/thegames/putina/artira 2014-03-17 http://auto.ru/chatay-uchits/thegames/putina/artira +http://auto.ru/chatay-uchits/thegames/putina/artira 2014-03-18 http://auto.ru/chatay-uchits/thegames/putina/artira +http://auto.ru/chatay-uchits/thegames/putina/artira 2014-03-19 http://auto.ru/chatay-uchits/thegames/putina/artira +http://auto.ru/chatay-uchits/thegames/putina/artira 2014-03-20 http://auto.ru/chatay-uchits/thegames/putina/artira +http://auto.ru/chatay-uchits/thegames/putina_2008 2014-03-17 http://auto.ru/chatay-uchits/thegames/putina_2008 +http://auto.ru/chatay-uchits/thegames/putina_2008 2014-03-18 http://auto.ru/chatay-uchits/thegames/putina_2008 +http://auto.ru/chatay-uchits/thegames/putina_2008 2014-03-19 http://auto.ru/chatay-uchits/thegames/putina_2008 +http://auto.ru/chatay-uchits/thegames/putina_2008 2014-03-20 http://auto.ru/chatay-uchits/thegames/putina_2008 +http://auto.ru/chatay-uchits/thegames/putinatik_3 2014-03-17 http://auto.ru/chatay-uchits/thegames/putinatik_3 +http://auto.ru/chatay-uchits/thegames/putinatik_3 2014-03-18 http://auto.ru/chatay-uchits/thegames/putinatik_3 +http://auto.ru/chatay-uchits/thegames/putinatik_3 2014-03-19 http://auto.ru/chatay-uchits/thegames/putinatik_3 +http://auto.ru/chatay-uchits/thegames/putinatik_3 2014-03-20 http://auto.ru/chatay-uchits/thegames/putinatik_3 +http://auto.ru/chatay-uchits/thegames/putinato-po 2014-03-17 http://auto.ru/chatay-uchits/thegames/putinato-po +http://auto.ru/chatay-uchits/thegames/putinato-po 2014-03-18 http://auto.ru/chatay-uchits/thegames/putinato-po +http://auto.ru/chatay-uchits/thegames/putinato-po 2014-03-19 http://auto.ru/chatay-uchits/thegames/putinato-po +http://auto.ru/chatay-uchits/thegames/putinato-po 2014-03-20 http://auto.ru/chatay-uchits/thegames/putinato-po +http://auto.ru/chatayru.tsn.ua/artisemeyen 2014-03-17 http://auto.ru/chatayru.tsn.ua/artisemeyen +http://auto.ru/chatayru.tsn.ua/artisemeyen 2014-03-18 http://auto.ru/chatayru.tsn.ua/artisemeyen +http://auto.ru/chatayru.tsn.ua/artisemeyen 2014-03-19 http://auto.ru/chatayru.tsn.ua/artisemeyen +http://auto.ru/chatayru.tsn.ua/artisemeyen 2014-03-20 http://auto.ru/chatayru.tsn.ua/artisemeyen +http://auto.ru/chatayru.tsn.ua/artisemeyen 2014-03-21 http://auto.ru/chatayru.tsn.ua/artisemeyen +http://auto.ru/chatayru.tsn.ua/artisemeyen 2014-03-22 http://auto.ru/chatayru.tsn.ua/artisemeyen +http://auto.ru/chatayru.tsn.ua/artisemeyen 2014-03-23 http://auto.ru/chatayru.tsn.ua/artisemeyen +http://auto.ru/click.shortcut=true&groups[]=300&racelog&id=1872347416/view/48/r=a.me/istant-poshiny/9513701.ru/news.biz/sony 2014-03-21 http://auto.ru/click.shortcut=true&groups[]=300&racelog&id=1872347416/view/48/r=a.me/istant-poshiny/9513701.ru/news.biz/sony +http://auto.ru/click.shortcut=true&groups[]=300&racelog&id=1872347416/view/48/r=a.me/istant-poshiny/9513701.ru/news.biz/sony 2014-03-22 http://auto.ru/click.shortcut=true&groups[]=300&racelog&id=1872347416/view/48/r=a.me/istant-poshiny/9513701.ru/news.biz/sony +http://faber/frame/frm_index.ru 2014-03-17 http://faber/frame/frm_index.ru +http://faber/frame/frm_index.ru 2014-03-18 http://faber/frame/frm_index.ru +http://faber/frame/frm_index.ru 2014-03-19 http://faber/frame/frm_index.ru +http://faber/frame/frm_index.ru 2014-03-20 http://faber/frame/frm_index.ru +http://faber/frame/frm_index.ru 2014-03-21 http://faber/frame/frm_index.ru +http://faber/frame/frm_index.ru 2014-03-22 http://faber/frame/frm_index.ru +http://faber/frame/frm_index.ru 2014-03-23 http://faber/frame/frm_index.ru +http://minsk/odessages.yandex.ru/vorozhitelnichaet-risunk-v-kiev/школад&startv.com.tr 2014-03-17 http://minsk/odessages.yandex.ru/vorozhitelnichaet-risunk-v-kiev/школад&startv.com.tr +http://minsk/odessages.yandex.ru/vorozhitelnichaet-risunk-v-kiev/школад&startv.com.tr 2014-03-18 http://minsk/odessages.yandex.ru/vorozhitelnichaet-risunk-v-kiev/школад&startv.com.tr +http://minsk/odessages.yandex.ru/vorozhitelnichaet-risunk-v-kiev/школад&startv.com.tr 2014-03-19 http://minsk/odessages.yandex.ru/vorozhitelnichaet-risunk-v-kiev/школад&startv.com.tr +http://minsk/odessages.yandex.ru/vorozhitelnichaet-risunk-v-kiev/школад&startv.com.tr 2014-03-20 http://minsk/odessages.yandex.ru/vorozhitelnichaet-risunk-v-kiev/школад&startv.com.tr +http://minsk/odessages.yandex.ru/vorozhitelniki.ru/?mode=on&order=02.03.1307%26lr%3D1%26sid%3D12499454a73bce1f196d88644194 2014-03-17 http://minsk/odessages.yandex.ru/vorozhitelniki.ru/?mode=on&order=02.03.1307%26lr%3D1%26sid%3D12499454a73bce1f196d88644194 +http://minsk/odessages.yandex.ru/vorozhitelniki.ru/?mode=on&order=02.03.1307%26lr%3D1%26sid%3D12499454a73bce1f196d88644194 2014-03-18 http://minsk/odessages.yandex.ru/vorozhitelniki.ru/?mode=on&order=02.03.1307%26lr%3D1%26sid%3D12499454a73bce1f196d88644194 +http://minsk/odessages.yandex.ru/vorozhitelniki.ru/?mode=on&order=02.03.1307%26lr%3D1%26sid%3D12499454a73bce1f196d88644194 2014-03-19 http://minsk/odessages.yandex.ru/vorozhitelniki.ru/?mode=on&order=02.03.1307%26lr%3D1%26sid%3D12499454a73bce1f196d88644194 +http://minsk/odessages.yandex.ru/vorozhitelniki.ru/?mode=on&order=02.03.1307%26lr%3D1%26sid%3D12499454a73bce1f196d88644194 2014-03-20 http://minsk/odessages.yandex.ru/vorozhitelniki.ru/?mode=on&order=02.03.1307%26lr%3D1%26sid%3D12499454a73bce1f196d88644194 +http://minsk/odessages.yandex.ru/vorozhitelniy-bolsha.info=ww-1425-wh-113467157_254222883_0&gearbox][2]=0&sort=mobilein 2014-03-19 http://minsk/odessages.yandex.ru/vorozhitelniy-bolsha.info=ww-1425-wh-113467157_254222883_0&gearbox][2]=0&sort=mobilein +http://minsk/odessages.yandex.ru/vorozhitelniy-bolsha.info=ww-1425-wh-113467157_254222883_0&gearbox][2]=0&sort=mobilein 2014-03-20 http://minsk/odessages.yandex.ru/vorozhitelniy-bolsha.info=ww-1425-wh-113467157_254222883_0&gearbox][2]=0&sort=mobilein +http://minsk/odessages.yandex.ru/vorozhitelniy-bolsha.info=ww-1425-wh-113467157_254222883_0&gearbox][2]=0&sort=mobilein 2014-03-22 http://minsk/odessages.yandex.ru/vorozhitelniy-bolsha.info=ww-1425-wh-113467157_254222883_0&gearbox][2]=0&sort=mobilein +http://minsk/odessages.yandex.ru/vorozhitelniy-bolsha.info=ww-1425-wh-113467157_254222883_0&gearbox][2]=0&sort=mobilein 2014-03-23 http://minsk/odessages.yandex.ru/vorozhitelniy-bolsha.info=ww-1425-wh-113467157_254222883_0&gearbox][2]=0&sort=mobilein +http://minsk/odessages.yandex.ru/vorozhitelno_otdyh/tube.com/Magazin%2Fttopic.ru/ulichnopoisk.ru/novosti/352662-25-teknoloji 2014-03-17 http://minsk/odessages.yandex.ru/vorozhitelno_otdyh/tube.com/Magazin%2Fttopic.ru/ulichnopoisk.ru/novosti/352662-25-teknoloji +http://minsk/odessages.yandex.ru/vorozhitelno_otdyh/tube.com/Magazin%2Fttopic.ru/ulichnopoisk.ru/novosti/352662-25-teknoloji 2014-03-18 http://minsk/odessages.yandex.ru/vorozhitelno_otdyh/tube.com/Magazin%2Fttopic.ru/ulichnopoisk.ru/novosti/352662-25-teknoloji +http://minsk/odessages.yandex.ru/vorozhitelno_otdyh/tube.com/Magazin%2Fttopic.ru/ulichnopoisk.ru/novosti/352662-25-teknoloji 2014-03-19 http://minsk/odessages.yandex.ru/vorozhitelno_otdyh/tube.com/Magazin%2Fttopic.ru/ulichnopoisk.ru/novosti/352662-25-teknoloji +http://minsk/odessages.yandex.ru/vorozhitelno_otdyh/tube.com/Magazin%2Fttopic.ru/ulichnopoisk.ru/novosti/352662-25-teknoloji 2014-03-20 http://minsk/odessages.yandex.ru/vorozhitelno_otdyh/tube.com/Magazin%2Fttopic.ru/ulichnopoisk.ru/novosti/352662-25-teknoloji +http://minsk/odessages.yandex.ru/vorozhitelnosti/Armanada-yeni-boyiny_i_motot-birthday=1#body_key=50aa1f0bbce1fc00b6a54 2014-03-17 http://minsk/odessages.yandex.ru/vorozhitelnosti/Armanada-yeni-boyiny_i_motot-birthday=1#body_key=50aa1f0bbce1fc00b6a54 +http://minsk/odessages.yandex.ru/vorozhitelnosti/Armanada-yeni-boyiny_i_motot-birthday=1#body_key=50aa1f0bbce1fc00b6a54 2014-03-18 http://minsk/odessages.yandex.ru/vorozhitelnosti/Armanada-yeni-boyiny_i_motot-birthday=1#body_key=50aa1f0bbce1fc00b6a54 +http://minsk/odessages.yandex.ru/vorozhitelnosti/Armanada-yeni-boyiny_i_motot-birthday=1#body_key=50aa1f0bbce1fc00b6a54 2014-03-19 http://minsk/odessages.yandex.ru/vorozhitelnosti/Armanada-yeni-boyiny_i_motot-birthday=1#body_key=50aa1f0bbce1fc00b6a54 +http://minsk/odessages.yandex.ru/vorozhitelnosti/Armanada-yeni-boyiny_i_motot-birthday=1#body_key=50aa1f0bbce1fc00b6a54 2014-03-20 http://minsk/odessages.yandex.ru/vorozhitelnosti/Armanada-yeni-boyiny_i_motot-birthday=1#body_key=50aa1f0bbce1fc00b6a54 diff --git a/dbms/tests/queries/1_stateful/00069_duplicate_aggregation_keys.sql b/dbms/tests/queries/1_stateful/00069_duplicate_aggregation_keys.sql index ec92d4a8ace..1f09b656712 100644 --- a/dbms/tests/queries/1_stateful/00069_duplicate_aggregation_keys.sql +++ b/dbms/tests/queries/1_stateful/00069_duplicate_aggregation_keys.sql @@ -1 +1 @@ -SELECT URL, EventDate, max(URL) FROM test.hits WHERE CounterID = 731962 AND UserID = 2651474201385397001 GROUP BY URL, EventDate, EventDate ORDER BY URL, EventDate; +SELECT URL, EventDate, max(URL) FROM test.hits WHERE CounterID = 1704509 AND UserID = 4322253409885123546 GROUP BY URL, EventDate, EventDate ORDER BY URL, EventDate; diff --git a/dbms/tests/queries/1_stateful/00071_merge_tree_optimize_aio.reference b/dbms/tests/queries/1_stateful/00071_merge_tree_optimize_aio.reference index d366a28c867..1cdb7c64b9b 100644 --- a/dbms/tests/queries/1_stateful/00071_merge_tree_optimize_aio.reference +++ b/dbms/tests/queries/1_stateful/00071_merge_tree_optimize_aio.reference @@ -1,100 +1,100 @@ -2014-03-18 00:00:00 2014-03-18 152905 1149766421370435682 http://podrobnosti.ua/kaleidoscope/2014/03/17/965156.html http://podrobnosti.ua/society/2014/02/19/959834.html -2014-03-18 00:00:00 2014-03-18 303268 5927163351345602343 http://5-shagov.ru/grippe/ http://5-shagov.ru/parents/hospitalization.php -2014-03-18 00:00:00 2014-03-18 722884 8841128461379701592 http://youtube.com/watch?v=JeOfJi4ZPXw http://yandex.ru/video/search?text=Girlicious - Stupid Shit танцы&filmId=JrZrradeUXI -2014-03-18 00:00:00 2014-03-18 722889 712457281393857753 http://images.yandex.ua/yandsearch?text=тату для мужчин&img_url=http:%2F%2Fi1031.photobucket.com%2Falbums%2Fy379%2Fviki_loves_me%2Fpic_355.jpg&pos=5&rpt=simage&lr=10363&noreask=1&source=wiz http://yandex.ua/yandsearch?text=тату для мужщин&lr=10363 -2014-03-18 00:00:00 2014-03-18 731962 6127057901301120896 http://yandex.ru/yandsearch?text=как правильно посадить лилию в грунт&lr=213 http://yandex.ru/yandsearch?lr=213&oprnd=5603136039&text=как правильно посадить лилию в горшок -2014-03-18 00:00:00 2014-03-18 4308403 260966421390077298 http://lenta.ru/rubrics/ussr/ -2014-03-18 00:00:00 2014-03-18 10095472 3034723561394017307 http://kolesa.kz/a/show/15015981 http://auto.yandex.ru/volkswagen/passat/6391673?state=USED -2014-03-18 00:00:00 2014-03-18 11229106 799814451394901442 http://prntscr.com/31iiin -2014-03-18 00:00:00 2014-03-18 21263557 1158837751395057041 http://hugesex.tv/ar/newest.html http://hugesex.tv/ar/خولات.html -2014-03-18 00:00:00 2014-03-18 23785906 47041921395093581 http://mcums.com/videos/licking-and-fucking-shaved-pussy-of-brunette-gf/ http://mcums.com/tags/pussy-licking/ -2014-03-18 00:00:01 2014-03-18 132528 7556707501378763475 http://eva.ru/jsf/forum/frame-content-post-message.jsp?topicId=3236928&boardId=131&messageId=84758330 http://eva.ru/jsf/forum/frame-content-tree-topic-messages.jsp?topicId=3236928&reload=&showAll=false&reloadTo= -2014-03-18 00:00:01 2014-03-18 722889 1211651541357579377 http://images.yandex.ru/yandsearch?source=wiz&fp=0&text=хайрюнниса гюль&noreask=1&pos=18&lr=101084&rpt=simage&uinfo=ww-980-wh-598-fw-938-fh-448-pd-2&img_url=http:%2F%2Fimgnews.hurriyet.com.tr%2FLiveImages\\photonews\\Today\'s News in Pictures - Feb. 13, 2009\\5.jpg http://yandex.ru/yandsearch?lr=101084&text=хайрюнниса гюль -2014-03-18 00:00:01 2014-03-18 731962 1996577731349080058 http://yandex.by/yandsearch?text=каскад&clid=9582&lr=157 http://yandex.by/yandsearch?rdrnd=764250&text=01100011&clid=9582&lr=157&redircnt=1395086250.1 -2014-03-18 00:00:01 2014-03-18 922978 2402137561319480416 http://segodnya.ua/allnews/p3.html http://segodnya.ua/allnews/p2.html -2014-03-18 00:00:01 2014-03-18 1143050 2007287821344978095 https://mail.yandex.ru/for/webprofiters.ru/neo2/#folder/2090000160054569829 https://mail.yandex.ru/for/webprofiters.ru/neo2/#inbox/thread/2090000004677772251 -2014-03-18 00:00:01 2014-03-18 5503465 3099908391394899634 http://lamoda.ru/c/563/bags-sumki-chehli/?genders=women&sitelink=topmenu http://lamoda.ru/?ef_id=UyYS-wAABWroAQsl:20140317195941:s -2014-03-18 00:00:01 2014-03-18 9927757 1886498191394574261 http://yandex.com.tr/ -2014-03-18 00:00:01 2014-03-18 10193245 5187318751332874613 https://parimatch.by/ https://parimatch.by/?login=1 -2014-03-18 00:00:01 2014-03-18 10849243 712457281393857753 http://images.yandex.ua/yandsearch?text=тату для мужчин&img_url=http:%2F%2Fi1031.photobucket.com%2Falbums%2Fy379%2Fviki_loves_me%2Fpic_355.jpg&pos=5&rpt=simage&lr=10363&noreask=1&source=wiz http://yandex.ua/yandsearch?text=тату для мужщин&lr=10363 -2014-03-18 00:00:01 2014-03-18 12725416 6312826621394013737 goal://megogo.net/playtime http://megogo.net/ru/view/44131-zakrytaya-shkola-sezon-2-seriya-24.html -2014-03-18 00:00:01 2014-03-18 16436437 9135489181394970660 http://naitimp3.com/search/?query=Сектор Газа Роковой год 1999 http://go.mail.ru/search?q=сектор газа в роковой год можете мне не верить 1999 слушать -2014-03-18 00:00:01 2014-03-18 23414332 303872061330707283 http://blognews.am/arm/press/141257/ https://facebook.com/ -2014-03-18 00:00:01 2014-03-18 23544181 303872061330707283 http://blognews.am/arm/press/141257/ https://facebook.com/ -2014-03-18 00:00:01 2014-03-18 24129763 2090186401395086374 http://flvto.com/es/ -2014-03-18 00:00:02 2014-03-18 731962 2415724631390139276 http://yandex.ru/yandsearch?text=смотреть аниме очень приятно, бог&lr=213 http://yandex.ru/yandsearch?lr=213&text=смотреть аниме удар крови -2014-03-18 00:00:02 2014-03-18 1143050 2007287821344978095 https://mail.yandex.ru/for/webprofiters.ru/neo2/#folder/2090000160054569829/thread/2090000001841781953 -2014-03-18 00:00:02 2014-03-18 1143050 2007287821344978095 https://mail.yandex.ru/for/webprofiters.ru/neo2/#folder/2090000160054569829/thread/2090000001841781953 https://mail.yandex.ru/for/webprofiters.ru/neo2/#folder/2090000160054569829 -2014-03-18 00:00:02 2014-03-18 10041976 4249766291378661688 http://magazin-gobelenov.ru/catalog/elizabet/ http://magazin-gobelenov.ru/catalog/angelina/ -2014-03-18 00:00:02 2014-03-18 10740559 5234980771391796639 http://ludivteme.com/user/contacts/ http://ludivteme.com/user/post/?uId=2617253&postId=2713054&ref=fav -2014-03-18 00:00:02 2014-03-18 11492179 690778681385387852 http://ru.tsn.ua/politika -2014-03-18 00:00:02 2014-03-18 12539611 1949138091341424951 http://pup-sik.ru/main/8-tanki-onlajn.html -2014-03-18 00:00:02 2014-03-18 13814323 48915321355515668 http://lentaporno.com/minet/page/2/ -2014-03-18 00:00:02 2014-03-18 15044245 4249766291378661688 http://magazin-gobelenov.ru/catalog/elizabet/ http://magazin-gobelenov.ru/catalog/angelina/ -2014-03-18 00:00:02 2014-03-18 16137184 1595224551373963471 http://seria-online.ru/100579-v-lesah-i-na-gorah-1-sezon-20-serija.html#pInfoData http://seria-online.ru/100578-v-lesah-i-na-gorah-1-sezon-19-serija.html -2014-03-18 00:00:02 2014-03-18 19957570 355266201395086276 http://photos.wowgirls.com/aa73789d/MTY1MjY6NDQ6MzI/ http://milkmanbook.com/ -2014-03-18 00:00:02 2014-03-18 23194813 2630182581386173797 http://posta.com.tr/magazin/GaleriHaber/Cicekciyi-gorunce---.htm?ArticleID=220836&PageIndex=3 http://posta.com.tr/magazin/GaleriHaber/Cicekciyi-gorunce---.htm?ArticleID=220836&PageIndex=2 -2014-03-18 00:00:02 2014-03-18 23414332 1949138091341424951 http://pup-sik.ru/main/8-tanki-onlajn.html -2014-03-18 00:00:02 2014-03-18 23427556 1167178261394699416 goal://debilizator.tv/Online http://debilizator.tv/tnt/ -2014-03-18 00:00:02 2014-03-18 24322408 531864411394811824 http://sefan.mobi/yo.php?id=6&place=main http://sefan.ru/ -2014-03-18 00:00:03 2014-03-18 115931 62014561357385318 http://aukro.ua/videoregistratory-111986?a_enum[695][1]=1&change_view=Найти >&listing_interval=7&listing_sel=2&order=qd&offerTypeBuyNow=1&ap=1&aid=17390505 -2014-03-18 00:00:03 2014-03-18 731962 4598420951330592755 http://yandex.ru/yandsearch?lr=43&msid=22881.14658.1395086392.12925&oprnd=9814174567&text=авито http://yandex.ru/ -2014-03-18 00:00:03 2014-03-18 942065 728339451390597173 http://utkonos.ru/cat/catalogue/41/page/3?property[]=480:230849 http://utkonos.ru/cat/catalogue/41/page/3?property[]=56:409&property[]=480:230849 -2014-03-18 00:00:03 2014-03-18 1143050 2007287821344978095 https://mail.yandex.ru/for/webprofiters.ru/neo2/#folder/2090000160054569829/thread/2090000001841781953 -2014-03-18 00:00:03 2014-03-18 6599752 1618423011295641161 http://pornoload.com/video/gestkoe http://pornoload.com/video/orgii -2014-03-18 00:00:03 2014-03-18 23609092 303872061330707283 http://blognews.am/arm/press/141257/ https://facebook.com/ -2014-03-18 00:00:03 2014-03-18 24142063 1833987961394132913 http://searcher.takataka.coccoc.com/searcher/frame/default?abid=fdbb8847fde149930234e02fc55dc6f0 http://tinngoisao.vn/tin-tuc/lo-dien-thi-sinh-cua-hoa-hau-phu-nhan-tai-my-2014 -2014-03-18 00:00:04 2014-03-18 62180 773991171388847630 http://rutube.ru/video/9049a252a077b229303b4cbe3fd08cd1/ http://rutube.ru/popup_http://rutube.ru/video/9049a252a077b229303b4cbe3fd08cd1/ -2014-03-18 00:00:04 2014-03-18 62180 773991171388847630 http://rutube.ru/video/9049a252a077b229303b4cbe3fd08cd1/ http://rutube.ru/popup_http://rutube.ru/video/9049a252a077b229303b4cbe3fd08cd1/ -2014-03-18 00:00:04 2014-03-18 722545 913874051395084276 http://yandex.ru/ -2014-03-18 00:00:04 2014-03-18 722889 824625261393690464 http://images.yandex.ua/yandsearch?tld=ua&p=5&text=ржачные картинки&fp=5&pos=167&uinfo=ww-1905-wh-936-fw-1680-fh-598-pd-1&rpt=simage&img_url=http:%2F%2Fcs7002.userapi.com%2Fv7002416%2F30%2FyjhueCmnUIc.jpg http://yandex.ua/ -2014-03-19 00:00:00 2014-03-19 722545 695702261356209942 http://yandex.by/ -2014-03-19 00:00:00 2014-03-19 722545 7181768941378219775 http://yandex.ru/ -2014-03-19 00:00:00 2014-03-19 722545 9484883841392744913 http://yandex.ru/ -2014-03-19 00:00:00 2014-03-19 731962 7190331241393769781 http://yandex.ua/yandsearch?text=.Народная Солянка + Поиски чернобыльского Шахматиста прохождение.контейнер калмыка&clid=2070746&lr=143 -2014-03-19 00:00:00 2014-03-19 2237260 4222828211330284262 http://avito.ru/kamensk-shahtinskiy?p=2 http://avito.ru/kamensk-shahtinskiy -2014-03-19 00:00:00 2014-03-19 12901042 2297304011387025366 http://radario.ru/vk/app?appKey=685c02962351452f9dc5497cad23fa9d&api_url=http://api.vk.com/api.php&api_id=3786618&api_settings=0&viewer_id=1528921&viewer_type=0&sid=4527ab66148ec89be4b6f51b9b8236e8caec812dcf6aded81d0908ed794b8ec513824a549e7bbad3bd2ef&secret=d54dbc6532&access_token=e619ee54a2d5687ac4147c0d6b35adbd36827847a05983c5e7f407288f02281b0708e1be5dc4f1da03c5c&user_id=0&group_id=0&is_app_user=0&auth_key=0bc4451e889647c1d615673e5fe39422&language=0&parent_language=0&ad_info=ElsdCQBZQ11tBwVNRARQBHR/FAsmMQxVUUZGNgBQbwYfQyQrWQA=&is_secure=0&ads_app_id=3786618_47e214d0ab09e5d822&api_result={"response":[{"uid":1528921,"first_name":"Doctor","last_name":"Banan","sex":2,"nickname":"","bdate":"18.2.1990","photo_big":"http:\\%2F\\%2Fcs425527.vk.me\\%2Fv425527921\\%2F879f\\%2F0Ka3g6CM2a0.jpg"}]}&referrer=unknown&lc_name=ae2044e2&hash= http://vk.com/msessions -2014-03-19 00:00:00 2014-03-19 15003622 61132721390316282 http://acunn.com/survivor#anket http://acunn.com/survivor -2014-03-19 00:00:01 2014-03-19 109993 1578933551394054044 http://molotok.ru/myaccount/smanager.php?page=auctions&type=not_sold&p=87 http://molotok.ru/myaccount/smanager.php?page=auctions&type=not_sold -2014-03-19 00:00:01 2014-03-19 722545 2436598271392110542 http://yandex.ru/ -2014-03-19 00:00:01 2014-03-19 11514781 2111148871395066201 http://intellect-video.com/5345/BBC--Horizon--Prizrak-v-moikh-genakh-online/ http://intellect-video.com/natural-sciences-biology/ -2014-03-19 00:00:01 2014-03-19 12040240 2040458821375390840 http://gazetaby.com/cont/list.php?sn_arx=1&sn_cat=32 -2014-03-19 00:00:01 2014-03-19 15003622 61132721390316282 http://acunn.com/survivor -2014-03-19 00:00:01 2014-03-19 21270109 781274971395130639 http://fotostrana.ru/user/71384319/ http://e.mail.ru/cgi-bin/link?check=1&refresh=1&cnf=3506c8&url=http:%2F%2Ffotostrana.ru%2Fuser%2Fautologin%2F%3Fu%3D71384319%26h%3Ddb59c4c43dfb7f3%26eRf%3D1301%26t%3D1301%26v%3D2%26time%3D1395130362%26to_url%3D%2Fuser%2F74646240%2F%26utm_campaign%3Demail_notify%26utm_content%3Dlink%26utm_medium%3Demail_1301%26utm_source%3Devent_0&msgid=13951308140000000339;0,1&x-email=anderson_66@mail.ru&js=1&redir=1 -2014-03-19 00:00:01 2014-03-19 21279787 7255079541391233211 http://diary.ru/~Tahgira/p196293215.htm?oam#more1 http://mylostharem.diary.ru/?favorite&from=40 -2014-03-19 00:00:01 2014-03-19 21879736 2581102761372914766 http://cocok.mobi/video/listing/3/3 http://cocok.mobi/video/listing/3/2 -2014-03-19 00:00:01 2014-03-19 22063525 2634772471395187200 http://porniac.de/free-porn-teen-german-deutsch-creampie-inpussy http://google.de/url?sa=t&rct=j&q=&esrc=s&source=web&cd=3&ved=0CEAQFjAC&url=http:%2F%2Fwww.porniac.de%2Ffree-porn-teen-german-deutsch-creampie-inpussy&ei=utwoU9OBOcig4gTt0IH4Bg&usg=AFQjCNG__41cA_2JKqf2AOYlxK8Vg5-hHA&bvm=bv.62922401,d.bGE -2014-03-19 00:00:01 2014-03-19 22273222 7971532421394017800 http://video.nur.kz/serial/14-velikolepnyy-vek/sezon-4-seria-19&ref=search_redirect&autoplay=1 -2014-03-19 00:00:02 2014-03-19 73874 5271033891286095023 http://livetv.sx/eventinfo/218436_chelsea_galatasaray/ http://livetv.sx/allupcomingsports/1/ -2014-03-19 00:00:02 2014-03-19 187695 6517393831394014299 http://malls.ru/upload/resize_cache/iblock/ea6/800_800_1bb65d3589fb69dd4a17961e4a75772f4/ea6b77961089460feaf0260d7d28345f.jpg http://malls.ru/rus/rent/demand/id45961.shtml -2014-03-19 00:00:02 2014-03-19 9927988 1968640351383680744 http://yandex.com.tr/yandsearch?lr=11503&text=anne yeşil örgü modelleri http://yandex.com.tr/ -2014-03-19 00:00:02 2014-03-19 23075158 1856338591385061414 http://1.newtop1.ru/?tid=999364201 http://proligtb.com/news.php?tz=2873580 -2014-03-19 00:00:02 2014-03-19 23711410 683272311395171829 https://betcruise.com/signin/expire/ https://betcruise.com/signup/success/ -2014-03-19 00:00:03 2014-03-19 48221 939058481380477704 http://love.mail.ru/search.phtml?ia=F&lf=M&af=26&at=35&wp=1&wv=0&wvc=0&ni=1&wr=0&sz=b&s_c=1012_25547957_25552720_0&geo=0&s_tg=&geo=0&t=o http://love.mail.ru/search.phtml?t=&sz=b&ia=F&lf=M&af=26&at=35&s_c=1012_25547957_25552720_0&target=&wp=1 -2014-03-19 00:00:03 2014-03-19 106966 3556373091312196176 http://rosbalt.ru/tests/gibdd -2014-03-19 00:00:03 2014-03-19 722545 149923051366137669 http://yandex.ru/ -2014-03-19 00:00:03 2014-03-19 731962 7407418091393711524 http://hghltd.yandex.net/yandbtm?fmode=inject&url=http:%2F%2Fweb-ulitka.ru%2Fshowthread.php%3Ft%3D955&tld=ru&lang=ru&text=Advanced System Protector ключ&l10n=ru&mime=html&sign=cc8ef02d6794cf571f8e88f8c4e5db2b&keyno=0 http://yandex.ru/yandsearch?clid=1955452&lr=11116&text=Advanced System Protector ключ -2014-03-19 00:00:03 2014-03-19 7970125 1852607591356602753 http://casio.prommag.ru/watches/protrek/protrek.html http://casio.prommag.ru/watches/protrek/protrek.html -2014-03-19 00:00:03 2014-03-19 10935790 9734083581388238977 http://domexpo.ru/company_info.htm?id=520 -2014-03-19 00:00:03 2014-03-19 22273222 7971532421394017800 goal://video.nur.kz/UPPOD_PLAY http://video.nur.kz/serial/14-velikolepnyy-vek/sezon-4-seria-19&ref=search_redirect&autoplay=1 -2014-03-19 00:00:03 2014-03-19 22273222 7971532421394017800 http://video.nur.kz/serial/14-velikolepnyy-vek/sezon-4-seria-19&ref=search_redirect&autoplay=1#UPPOD_PLAY_HIT http://video.nur.kz/serial/14-velikolepnyy-vek/sezon-4-seria-19&ref=search_redirect&autoplay=1 -2014-03-19 00:00:03 2014-03-19 23723584 7524871931395144801 http://mamba.ru/my/messages.phtml http://mamba.ru/ -2014-03-19 00:00:04 2014-03-19 48221 1173254791395172208 http://love.mail.ru/search.phtml?ia=M&lf=F&af=18&at=80&wp=1&wv=0&wvc=0&ni=1&wr=0&gid=1395172789&t=o&sz=b&s_c=3159_4052_0_0&geo=0&s_tg=&target=Sex&offset=0&nchanged=1395092224&noid=1243669604 http://love.mail.ru/search.phtml?ia=M&lf=F&af=18&at=80&wp=1&wv=0&wvc=0&ni=1&wr=0&gid=1395172383&sz=b&s_c=3159_4052_0_0&geo=0&s_tg=&target=Sex&geo=0&t=o -2014-03-19 00:00:04 2014-03-19 48221 7524871931395144801 http://mamba.ru/my/messages.phtml http://mamba.ru/ -2014-03-19 00:00:04 2014-03-19 79376 781274971395130639 http://my.mail.ru/?from=email http://my.mail.ru/?from=email -2014-03-19 00:00:04 2014-03-19 731962 1265434081376817556 http://yandex.ru/yandsearch?text=dj paroff слушать онлайн бесплатно&lr=2 http://yandex.ru/yandsearch?lr=2&text=depeche mode -2014-03-19 00:00:04 2014-03-19 2199583 1471157941382120170 http://litres.ru/?td -2014-03-19 00:00:04 2014-03-19 2237260 5917401071394958225 http://avito.ru/sankt-peterburg/avtomobili_s_probegom/ford_transit_1992_270304651 http://avito.ru/sankt-peterburg/avtomobili_s_probegom/ford/transit?pmax=150000&pmin=0 -2014-03-19 00:00:04 2014-03-19 2344120 8729055551317074933 http://proelectro.ru/cabinet/notices/list http://proelectro.ru/cabinet/notices/view/id/46749 -2014-03-19 00:00:04 2014-03-19 6969847 8729055551317074933 http://proelectro.ru/cabinet/notices/list http://proelectro.ru/cabinet/notices/view/id/46749 -2014-03-19 00:00:04 2014-03-19 7234936 2683079681355752489 http://korrespondent.net/ http://korrespondent.net/ukraine/politics/3320139-yarosh-staranyiamy-putyna-myr-stoyt-na-porohe-tretei-myrovoi-voiny -2014-03-19 00:00:04 2014-03-19 7604263 948490331394925267 http://small-games.info/?go=game&c=12&i=3968 https://google.com.ua/ -2014-03-19 00:00:04 2014-03-19 9422815 22123591394214537 http://audiopoisk.com/?q=зара http://audiopoisk.com/?q=Соундтреки из кино филыма можно звать мама -2014-03-19 00:00:04 2014-03-19 12725416 963351201395165694 goal://megogo.net/playtime http://megogo.net/ru/view/14391-mamy.html -2014-03-19 00:00:04 2014-03-19 12725416 1837943761391284585 http://megogo.net/ru/view/152541-dumay-kak-zhenshchina-seriya-1.html -2014-03-19 00:00:04 2014-03-19 12725416 6312826621394013737 goal://megogo.net/pause http://megogo.net/ru/view/44021-zakrytaya-shkola-sezon-2-seriya-13.html -2014-03-19 00:00:04 2014-03-19 12787931 2355954411395172413 http://sumo.ua/ https://google.com.ua/ -2014-03-19 00:00:04 2014-03-19 12908773 1558109441368022235 http://adultmanga.ru/sadistic_boy/vol1/1?mature=1 http://adultmanga.ru/sadistic_boy/vol1/1 -2014-03-19 00:00:04 2014-03-19 13375903 2246101281391512061 http://dota2.starladder.tv/tournament/6360 -2014-03-19 00:00:04 2014-03-19 19762435 5917401071394958225 http://avito.ru/sankt-peterburg/avtomobili_s_probegom/ford_transit_1992_270304651 http://avito.ru/sankt-peterburg/avtomobili_s_probegom/ford/transit?pmax=150000&pmin=0 -2014-03-19 00:00:04 2014-03-19 19765189 2088033481395179889 http://amkspor.com/2014/03/18/derin-futbolda-buyuk-kavga-274576/ http://sozcu.com.tr/ +2014-03-18 00:00:00 2014-03-18 170282 6048134779943662800 http://fanati-avtomobili_s_probegom/livemaste-twittemp/150714%2FWay%2F8056/partshowthreads&projects/14183_0&geo=r24R24&order +2014-03-18 00:00:00 2014-03-18 170282 6048134779943662800 http://fanati-avtomobili_s_probegom/livemaste-twittemp/150714%2FWay%2F8056/partshowthreads&projects/14183_0&geo=r24R24&order http://tanks.ru/samara.ru/records=Будь смотреть +2014-03-18 00:00:00 2014-03-18 257987 4339541024790670892 http://cian.ru/confirm.php&api_url=http://image&lr=108&text=270.EwIsyD&id=288410f87ef6591&text=сколько +2014-03-18 00:00:00 2014-03-18 631207 2192343902602423790 http://otvetim-merka.az/automobili http://pogoda.yandsearch;web +2014-03-18 00:00:00 2014-03-18 732797 955298336868677420 http://public/?hash=8jo504d974c28b276ab608de59f4a32b0645-fw-1415f87c2ea1f81ebd88c04d02cfdbb85226&wp +2014-03-18 00:00:00 2014-03-18 792887 2047284387776565057 http://korer.ru/categories.ru/?vkb http://vk.com/what=&priceTo-1512357®ion=0&clid=188_900b24f4b0b339a&keyno=0&l10n=ru&mc=4.41939e8a3fd9ca06Q&data=UlNrNmk5WktYejR0eWJFYk1LdmtxbEd4RYbyawIy +2014-03-18 00:00:00 2014-03-18 2526748 1217731179612043583 http://games.aspx?user +2014-03-18 00:00:00 2014-03-18 3498488 2010285617164728895 http://turkeyhotelines/21/view/508419939.139556%26bn%3D300%2F18 http://tankionliness harbox +2014-03-18 00:00:00 2014-03-18 3965364 1171271584863537056 http://category_id=100017 +2014-03-18 00:00:00 2014-03-18 5881900 4512998514182757914 http://e.mail.yandsearcheski?p=2&uinfo=ww-1583-wh-618-21857440/a13492/153/drugsearch?text=попорно http://avito.ru/prod.ru/photos?album_id=18&name":"Марина","first_name":"27 +2014-03-18 00:00:00 2014-03-18 16368233 1285157443497624768 http://afishi/landisher +2014-03-18 00:00:00 2014-03-18 22446879 1712232580571384293 https://auto.ru/work_etti&is_mobi/video.biz/201380020035364&secret=506d9e3dfbd268e6b6630e58 http://ication.html#/battle-ru1.html#edition=devloc&q=mechanged +2014-03-18 00:00:00 2014-03-18 30296134 1777042705640872159 http://image&lr=444496679e3189.4 http://1movie/boys?page +2014-03-18 00:00:00 2014-03-18 30712540 1901613100139799404 http://tv.yandex.ru/?_sys-hasa techenie/offel_65 +2014-03-18 00:00:00 2014-03-18 30868219 12124273907997428803 http://yandex.ru/clck/jsredir.com.tr&used http://hoodiezhda/privatelinks,pre,sites/qiwi.kz/?d-377-obuv/?search;web;;%2Fweb +2014-03-18 00:00:00 2014-03-18 31275924 1176335773382681385 http://blogspot.com.tr/widget.aspx&refereren-unutulmartf-onlajn/3-1-0-1395478765795401659483222da2bc7c98d391a48s120a52dff34074ee19e01feeesminecrafta-skor.com:80&text=одноклассники тачки для ротасон смоленьком язык. Подключ&stype=1&searchplus.ru/yandex.ru/24000525959251&FromGeo=2_6865883-krymafoking-butylove.mail.yandex.ua/news%2Ffoto-galeri.hurriyet.com/iframe/graf_35_et._209720465&z=13&lr=53&uinfo=ww-1263-wh-587-pd-1.25&run[1]=&extras[15]=0&extras[11]%3D%3DfrSMw%3D52%26curresponden-color_id_matrities[]=2014032113&clid=19554148.html?prx=557&doc=50000_v_posledovanie_is_11&From=navigaci-hazron +2014-03-18 00:00:00 2014-03-18 31996448 98670839878845626 http://ottogf-two/page=1&sid=359&krl[1]=&year[ +2014-03-18 00:00:01 2014-03-18 214340 716222823614200680 http://restipa-elenko-visibirsk/kombies:mobili_s_probegom/m/gazeta.ru http://vk.com.ua/bb/viewforum/phpBB3/view_video-audioptolovye-igrydlja-konsor.net/ru/view_intext=куплю мерседейства?&page3.censor.net/mygami-vesna_2006017a2cd9f891486e93d307c247d03a4ce655727057206b993v0-LJxpWyJNVbaoiiUtIAsxqV6ESUwsk7yk7EJ6g4BvDLYhn0AGi17xVPy7MuhF36pPSSV95VnRNR2doaDAydG1hUVg4S2dYYVE4aHBsNkJYbUFodHRwOi8vd3d31f3b8bc17242e99fd4327f2bf3c35294278413225589563/9980.aspx?api_settiden-katin.votpuskarbox][2]=&engine_volum.html/760/galeri/saglik-oyunuoyna.cc/pickundu/recipes/dzherridetay/8055.1&fr=vbm&fr2=query=haircasindanted/mitsubishi/profile/4421487][FROM=7&NIGHT_-ysddom/volgography/list=PLKMQIXapRUKow&data=UlNrNmk5WktYejR0eWJFYk1LdmtxcmJkUVJoZl93RkVVNVNSRERMUTdEMDBENd-bota.ru/search?lr=108&color_id=0&is_section +2014-03-18 00:00:01 2014-03-18 219339 433572290661205817 http://dosug.ru/jobs/a_the_shuby1wcm9sZ2UmdXRtX3NvcnQ http://uzmantv.com.tr/yazabavnyie-miikag8IQ_p3s0v3h7L11S4Ja5VuNf1 +2014-03-18 00:00:01 2014-03-18 598875 1496705010753950302 http://public_search +2014-03-18 00:00:01 2014-03-18 697178 4200893070175653811 http://hurpass.com.tr&user_pts=&states/?r=main.com.ua/yandex.ru/testle.direct=http:%2F%2Fnews.yandex.ua/plyusy-kushniy_novari +2014-03-18 00:00:01 2014-03-18 1179593 5926418179564641464 http:%2F%2Fbig%2F350696309c32ff47f4ef75588911 http://yandex.ru/photo=1&pmax=2005260250.Pompanie.aspx +2014-03-18 00:00:01 2014-03-18 1704509 1609925195107931879 http://auto.ru/chatay-barana.ru/traction.html#maybettaya +2014-03-18 00:00:01 2014-03-18 1704509 9003784859839045966 http://auto.ru/chatay-barana.ru/traction.html#maybettaya +2014-03-18 00:00:01 2014-03-18 1704509 9003784859839045966 http://auto.ru/chatay-barana.ru/traction.html#maybettaya +2014-03-18 00:00:01 2014-03-18 11285298 3048464416329576572 http://yandex.php?item-2014/91248&st=135&api_id=189421609&confirm +2014-03-18 00:00:01 2014-03-18 13129405 1660475275256148984 http://yandsearch&fp=1&pmax=17005/33-kilo-ukrepka.ru/?/trana.fm/adalettext http://avito.ru/?actions?album_id=0&year +2014-03-18 00:00:01 2014-03-18 20391337 3803178643246309835 http://jizz.html?ia=M&lf=F&af=40&qdsی.html?ia=M&lf=F&af=45&wp=1&text=эксплуатация анкерных +2014-03-18 00:00:01 2014-03-18 22275661 657286527437920382 http://state][1320]=&extras[18]%3D0%26xpid +2014-03-18 00:00:01 2014-03-18 23696567 6063978816750177114 http://forum.svadba-male/5972736/arams/109820e2f7a23be79&lang http://slovary_ctxt=Рахукол +2014-03-18 00:00:01 2014-03-18 26476181 7473263210959306111 http://kabi-duygunila-spb.ru/188/?from=sform.ru/riba4c15557 http://line.com/scription_id=&state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUd +2014-03-18 00:00:01 2014-03-18 26738469 1272764555417836964 http://wwwww.hurriyet.com%2Fmy-s-grekom.news%2F2369%26bn%3D7400a4c392f9cb3fc4d498ff45fd8c3d4/5317&i=1&wv=0&wvc=0&ni=1&wv http://situation/53986/fils.aspx?nickie_obruce +2014-03-18 00:00:01 2014-03-18 28966358 1060646201794570810 http://yandsearch?text=обсуждения http://kelebekgaleri.net/besplayer&join.ua/rezerval=1&bodystyle.com/8-marter97.org/moskva/realty_nedvizhimost_vznosa&submit=Найти&mark_id%3D2%26cof%3D25673878242980726/2/3/teenty;2.html&lang=ru&mc=3.9698270266743599250072115/2368/1/2602491&CMD=-RR=9,0,0.0015&acceOfSearch[year=2014/9122-vykly_i_mashnee-1NiR2hvdy8zMDkyLmh0bWw%3D%26extras[14]=0&country_id2-54-568-fw-842349&group_alian and Ma"|Grantipasta.com.ua/zhivotnyi-budet_pl%2Fkatalog/Healty.yandex.ru/tyument=ms-android.medley1/articles/u/72836eef\\%2Fcs60812&field_localiskille-sampionaty/?year[1]=&photostrahan/avtomatology/otzone/22/sochiy_modelid=13952333.72611&lr=1436.4967961725-32-64550/2/15/shop/ration/search/?q=audi/1562661084&text=гей для девочку&uuid=&startMessage_max=360&uuid=&state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgowt9XvoT-twMUKrgP_GFS4BsMRxVNufN1RVFKT2JFaU5wTy1QSmJwSUZhMGFCUzRCUzRGZvU2ZwU01kN-vOE9WOGt6djg2ekx3QkhHNW5maDhUSl80Njk4Nw +2014-03-18 00:00:01 2014-03-18 32402449 1651220797830840768 http://avito.ru/yandsearchResultant.ru.html&__utmcsr=googulochnoyi_motorhipo.com/hyundame-10-online.com/wp-contact&submission_time.ru/cat/mated=&geo=0&results.php?category/classnogorset/karaoke.yandex.ru/neo2/#inbox/?back_list?folder:247008/11498855460c84d11b&uuinfo/neo2/#inbox/?back_url=http:%2F%2Fdota2.ru/news.fishki-komnatalog/434-e69745426&nohead.php?page.all.com http://turbobit.ru/accountry=RUB;prom.ru/page=0&price_usd[1]=&extras[27]=0&snowide=0&l10n=ru&mc=4.96731-yatci-10.html?period +2014-03-18 00:00:02 2014-03-18 185050 2722170180705134315 http://spb-n.ru/aquare%2Fc_1240b02c5b89f072c702a4a10 http:\\%2Fc5WutigJbjk.jpg","photos?album_id=40311&group +2014-03-18 00:00:02 2014-03-18 598875 1216034325372652899 http://public_search +2014-03-18 00:00:02 2014-03-18 732797 920932492738614960 http://ameriya-sovets-horoskop.ru +2014-03-18 00:00:02 2014-03-18 14850592 1510556435918013377 http://mail.ru/yandex.ua/obyavlenie-observer=www-1285-wh-7773579&lr=148749966 +2014-03-18 00:00:02 2014-03-18 15338418 28483488602843017 http://comics.ruvr.ru/yandex.ru/society/ihelp.yandex.ru/yandex.ru/clck/jsredirectoryy-podrjadnye-retseptembed/657507 http://brezentry=ua&text=&etext=списьма той армянской одежда своими лініями +2014-03-18 00:00:02 2014-03-18 15789492 1874632264275552253 http://avito.ru/search?text=акт моловна http://mirtesentry[2 +2014-03-18 00:00:02 2014-03-18 17420663 159564314819780023 http://fast-golove.mail.yandex.php?PAGEN http://cars.auto.ru/yandex.ru/ukrcrewell?rated&num=49193151704494c3b5c3fcdb3e8090 +2014-03-18 00:00:02 2014-03-18 17854283 306862282026091586 http://yandex.ru/news.spor.mynet.com/crocko.ru http://google.az/auto.yandsearch +2014-03-18 00:00:02 2014-03-18 20241833 1540288275562371672 http://news/2014/03/04 http://earthod]=0&extras[27 +2014-03-18 00:00:02 2014-03-18 22918081 2182763963112132984 http://2gis.php?page-1141159/12-chemode=incev-2013/veli_169818198&max=2000349dec506508452205&List=4&msid=0&with +2014-03-18 00:00:02 2014-03-18 29429523 1404950772984356705 https://mail=1613/album/45517-gorunce&type=3#photo/torre_when=113 http://yandex.kz;yandex +2014-03-18 00:00:02 2014-03-18 33334423 559086571024955768 http://yandex.ru/ok_ads_mg_aa%2Fs57.radikavkaz/aziz-yapret-edition_timeout=14403 http://mynet.ru/search +2014-03-18 00:00:03 2014-03-18 59183 1496705010753950302 https://grozhniy/avtomobility.com/iframe/frm_index http://tvraina-pomostinenie-voyna.com +2014-03-18 00:00:03 2014-03-18 63469 3560037355667177305 http://doc/1437831&is_mobile=0&_p=30&offset=0&doma_dachalsya_v http://auth.mail.ru/chevronal/article&ie +2014-03-18 00:00:03 2014-03-18 170282 6048134779943662800 http://fanati-avtomobile=0&redir?from=wiz&fp=2 http://tanks.ru/search&z=video&text=билайн&uuid=&state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgowt9Xsh9AmfW +2014-03-18 00:00:03 2014-03-18 732797 9761870251124411459 http://kinogo.net/used/sale/5960777-disk:%2F%2Fwwww.buhgalter_app_id=1218.html#/battle/ffffer?wtf=&pt=&f[2816dbbc-penzensk.irr.net/experiod=dab01fcc011b533a427/#message/2480000&butto.ru/collects/igry_nam_stars/vaz_lada http://sham/928863d2dIZUZKeNw3PTbYTncrrCC9Kw6BLzxiErj2jBN7zgQNxu-DnuJ7UazZPenNwNTM2 +2014-03-19 00:00:00 2014-03-19 125776 7252775134812284033 http://dekor-kartgoal://chat&user_pass.com/tribute https://mvideo/HlsFrameID=1410495430 +2014-03-19 00:00:00 2014-03-19 675104 1029600628558229256 http://rusplt.ru/yandex.php?id=10000000000000005002408063e4dac54565-pd-1&img_url=http://myhdporno-ukraintpeters/elab.com.tr/?aff +2014-03-19 00:00:00 2014-03-19 732797 122602645439848448 http://public/?hash=8vIx7SAgLkks2Y38Mm1JprUJ-jizauyc/magazinam.whotras[25]=&engine_key=&wheelys-bangry_start=2674,55.95558%26bt%3D1397/notebook?q=стровый-рынке крованновая.jpg&c=53272&message +2014-03-19 00:00:00 2014-03-19 8680230 205695669535653109 http://bigcinema.ru/salatasarisoedynenykh-mart-2000038370 http://sharini-gosports.ru/forklifoval/5/sort=&obj=248®ion_owners_xxxxo-11301030114-zeki-evrolet/classniki.ru/pacancident.directadvert_rossiya-pomochka_zombid=&state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgowt9XvqxGyo_rnZJl2CUvGj2O5Sexy_s_roditsinav-sonics/185314291491-bolume][min]=0&q[price_value_max":"http://megogo.net/15000&pmin=&f_unity/weekly/?page=69&i=29752d3b1ab1770277922359117_18_xe.html?pid=7597&ocid=8817603.30340&p=2&id=17506520627e96db293a1fb97c992c3637_2968359881616.199454&lr=2&msid=810306&text=чехол&searchResultant.ru/model=lando.ua/galeri.hurriyet/notificatalog-va-trika-rus-name=8d81d66658162172-ister.ru/Rating.kz%2Fweb%2Fitem%2Ftitle,pos,p6,source,web&cd=37980936849485062903316862,bs.1,d.bGQ&cad=rjt +2014-03-19 00:00:00 2014-03-19 10998110 5592452394226169805 http://debili_s_probegom/search[mark +2014-03-19 00:00:00 2014-03-19 12864910 2543118835429830843 http://uyelik.hurriyet.com/vaz_lada +2014-03-19 00:00:00 2014-03-19 14739804 792283415880675681 http://saechku_virtualitelik.hurriyet http://pochildren.dnevniki.ru/touchshie-rodolzhenskor.com +2014-03-19 00:00:00 2014-03-19 17337667 67799227256418375 http://search?win=11&pos=22&img_url=http:%2F%2Fcs411276 +2014-03-19 00:00:00 2014-03-19 18811220 1135784929265179133 http://searchere=all&filter_pages.yandex.ru/video/search +2014-03-19 00:00:00 2014-03-19 19630020 842981240308972984 http://superjob.24ru.com.tr&callback_url=http http://yandex.ru;yandsearch +2014-03-19 00:00:00 2014-03-19 23447120 184573673733080802 http://yandex.ru/mult-sum=2014&is_mobili_s_probegom/cgi-bin/msglist http://goodok-skirova/avtomobile-news/13528471346 +2014-03-19 00:00:01 2014-03-19 46086 9280632542283254851 http://images.yandex.ru/music/20118.9_l,_D11023 http://yandex.ru/rossiya/stranskaia-moz-borba-zamok.ru/anasayfa +2014-03-19 00:00:01 2014-03-19 87423 2762756140212156801 http://sravnoe.ua/download/warth-chrome/shop.bigparazzet.com/offiliate=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgowt9XujWdVUNG4aJsDjDYnl9rlTrQZNLyr_3-mschiny.html?t=13954871581428394_Stras[29 +2014-03-19 00:00:01 2014-03-19 732797 2553037403182603460 http://public/?hash=8jo50417888/album/2013/patiana http://vk.com/publ/48/4.shtml#/battle/flash=#games.org/ +2014-03-19 00:00:01 2014-03-19 800784 1523891866394685648 http://brandal-vneshom http://kelebekgaleri.hurriyet +2014-03-19 00:00:01 2014-03-19 5869009 212340088576779871 http://tura-kozharovsk.drom.ru/news/sport_by%3D0%26city=istan/dogova/tennie_public/?logoda.yandex.ru/modules-5 http://horo.mail.yandsearch.php?id=3FBp1pE1UXI&wheel][1]=1&from=yandsearch;web;;%2Fweb%2Fitaljanslated +2014-03-19 00:00:01 2014-03-19 9153177 9298406401149324355 http://imageformal&statistichestenyuk-svaty_peppetitbit.net/rossobstoyalquestian http://krasi-4724.shtml&lang=ru&mc=2.481714c1415358/13/groups[]=17687&lr=213&text=ответы +2014-03-19 00:00:01 2014-03-19 12541302 16644846719988223043 https://hurpass.ru http://tv.php?set +2014-03-19 00:00:01 2014-03-19 12761932 943807938538976460 http://videovol-9-sezon http://go.mail.ru&js=1&ref +2014-03-19 00:00:01 2014-03-19 12864910 2543118835429830843 http://uyelik.hurriyet.com/vaz_lada http://censov.tr/fotogaleri/yaslandsearch;web;;%2Fweb%2Fitem%2Ftabs +2014-03-19 00:00:01 2014-03-19 17337667 67799227256418375 http://search?clid=1784%26nid http://ses-shop/Minsk.mvideo/search;web;;%2Fweb%2Fitem%2Ftitle,pos,p2,source=web&text=women.aspx?DocID=2213eUh0eWxEUGVwU2tXMUNDTElLVFqRzJlN0NGbDh5aGNpX0dQUlQ4ZG5XeH_2dPqoEH0_QW5LX21YdGtXTktPc0ttVnRfZGJ1U1ZIbXNMc0pVZDUUNSnR2SjlRZjdpdfNmNrazBvS2Jw3fQXpZVmtiLU5lcFE1Yi9zY2hlc3RpLm5ldCBivaLoad&FromGeoRadius +2014-03-19 00:00:01 2014-03-19 20208635 743795618912745809 http://free.ru/api.php?k=532a851f53d40c70f00000IiYSg3u2_-KVgcy5xdWljayUyQ2luby_dachat&user_page https://davi-6-nisafak.com/id188d9dc3cd121f2&keyno=0&searchText +2014-03-19 00:00:01 2014-03-19 21852496 1928938514100078282 http://araylis-unknowl/Proceduras-dobryila_Ettiere/6098092 http://ks.imhonetskaya-zhizn-v-hd +2014-03-19 00:00:01 2014-03-19 23540618 1884353571447239988 https://ayakta-yolo.jpg?act=ellikte-prografik-saati-kliches http://yandex.ru/yandsearch?text=деятель +2014-03-19 00:00:01 2014-03-19 30296134 24204270600999397 http://image&lr=213&img http://1movie/by_days=60&search?text=в ногтях +2014-03-19 00:00:01 2014-03-19 33010362 8723854704339558313 http://wotlauncher.html?page=region=2&msid=2296 +2014-03-19 00:00:02 2014-03-19 208747 818798661186687796 https://magiclandiya/transmissing-this.com/cadilly http://yandsearch[driverendedsearch?text=дешевле +2014-03-19 00:00:02 2014-03-19 716829 7931013822657338421 http://korablitz.ru/L_1OFFERS_CRD http://vk.com/l.php?topic=502 +2014-03-19 00:00:02 2014-03-19 1704509 1229512756943344113 http://auto.ru/chatay-barana.ru/traction.html#maybettaya +2014-03-19 00:00:02 2014-03-19 3807842 1138827739467216018 http://wot/html?l=radikal.com.tr/oauth http://yandex.ru;yandex.ru/chatka-sweetStep +2014-03-19 00:00:02 2014-03-19 9545777 4851729539838772597 http://loveplace:C}{ifSearch?source=psearch[engine_pid330e99bdb7c5f6b86ae67 +2014-03-19 00:00:02 2014-03-19 10581377 1315726515182016464 http://sporta/Futbol/avtomobile=0&availables/categories<{catalog/veskie.ru%2Fimg%2F%2Fkelebekgaleri/abdullaId=1jm9Jwqvs3M&wheelog&category=article/2013/8495834006.jца перцы&lr=19305614702068-kto-geograf.com/battle-ru7.html http://newdata=UlNrNmk5WktYejR0eWJFYk1LdmtxdmFiNGhLWV2eDFqZEdWtqdlln7YmMEKuXK_NOvBft4vqUPZEezyj51d813f&keyno=0&l10n=ru&mc=5.1254456842193-predpriya/gruzoviki +2014-03-19 00:00:02 2014-03-19 10726710 427396681451334333 http://fotos/photo +2014-03-19 00:00:02 2014-03-19 10769545 509274620842776000 http:%2F%2Fimg10.jpg?naments&sign=4cea22afbb59.html#/battle/ffffset":0}&filter http://mamba.ru/volgogradskaya_oblasti-m.ru/yandex.ua;yandsearch?text=черны 2011 +2014-03-19 00:00:02 2014-03-19 15469035 2163028668691208859 http://e.mail.ru/penth-170-smiley_gardno=0&l10n=ru&mobile=0&axleId=vestny-java%3D%3DfnSYg%3D%3D22%2F6D31102 +2014-03-19 00:00:02 2014-03-19 16303334 2655082216407003525 http://mac-os-widgetHoverlay=loginza.ru/21/dzhi/kompanie/krashivai.ru/neo2/#message/209000b7/?ref=https://home_page=http://fanatoster/sayfa%2FDLXitVHB6BlL95nuoNYfWKxd1PPwzlGVjkwcXljT1ZtdldC10LrRg9C70LDRgtC40LP3A http://youjizz.html?text=269.html?ia=M&lf=M&af=38&secret +2014-03-19 00:00:02 2014-03-19 16303334 2655082216407003525 http://photos?album_92_30092&img_url=http://forumlar-ligan +2014-03-19 00:00:02 2014-03-19 21566939 7712464540816566838 https://m.sprashivai.ru%2Fu768887.30.13952139%2F20090&text=наталья http://hepsiburada_Kristika.yandsearch?clid=40313&keyno=0 +2014-03-19 00:00:02 2014-03-19 26231141 1838624557351541445 http://gazetecture/2090000000000000034050932 +2014-03-19 00:00:03 2014-03-19 72208 280043544899887027 http://nevestivate/niki-24.jpg","photo https://bike/mult-games-majka-1_Iw45J31bOFAjp8wJpcel5R2NlU0tya00 +2014-03-19 00:00:03 2014-03-19 425290 4707526996599558574 http://hugesex.html http://apps.oyunskoronezhskaya +2014-03-19 00:00:03 2014-03-19 598875 3176817625720166582 http://public_search +2014-03-19 00:00:03 2014-03-19 633130 9039184642399513643 http://publ/tv-canli-yayinda-yavrupa-istory/search?lr=35&text=01&carfilternet/ukrasnodar http://vk.com/katalog/album +2014-03-19 00:00:03 2014-03-19 716829 1055052575843419080 http://korablitz.ru/L_1OFFERS_CRD +2014-03-19 00:00:03 2014-03-19 716829 3432594507250246056 http://korablitz.ru/L_1OFFERS_CRD http://auto.yandex.ru/catalog01_6000-fw-938-fh-448-pd-2 +2014-03-19 00:00:03 2014-03-19 732797 4764906385288064092 http://public/?hash=8vIx7SAgLkksPiNPkhcmFtbXlfeWFpX0Vien8xIAEsQVgWCVtUeF5_d34CemhbU0deYhg1O3RuXFdfR0pxCDBIFic4aAMMVQ1AA1MuTEhEGHVfV1dnY3wPFWVyYWxpX3NvY2hrb3Yvc1xUXk5IdwozXkV5fAZR http://vk.com/what=&price_do=7000&q=отдам даромашних услово.рф/Тестории по +2014-03-19 00:00:03 2014-03-19 792887 536346843347204662 http://korer.ru/categories.ru/?vkb http://vk.com/what=&probegom?pmax=250004/3094e1556755256181?wmodelid=18822&text=ged-games.ru.html?kkv1=2014/5-yeniry-starvettext=дицензионные плака&uuid=&state=AiuY0DBWFJ4ePaEse6rgeAjgs2pI3DW99KUdgowt9XvqxGyo_rnZJpNjfFDg3rin7oLRs831ba031 +2014-03-19 00:00:03 2014-03-19 1960469 29706847274882738 http://otvetimleri.slavl.ru http://diskovye-kuren +2014-03-19 00:00:03 2014-03-19 5262869 1739472721812280620 http:%2F%2Ffan%26rleurl%3Dhttp://porta/Futbol https://google.ru/click=&charter_enum[12]=508&u +2014-03-19 00:00:03 2014-03-19 9942562 6362666921123835505 http://youtube.ru/personnirovaya_knigame/frm_index.php?categor/86433.html?pid=209471-pd-1&p=8 http://go.mailbox.ru/admebel.ru/journal-refera66.ru/denij diff --git a/dbms/tests/queries/1_stateful/00072_compare_date_and_string_index.reference b/dbms/tests/queries/1_stateful/00072_compare_date_and_string_index.reference index 97db25771ee..acdeef673f0 100644 --- a/dbms/tests/queries/1_stateful/00072_compare_date_and_string_index.reference +++ b/dbms/tests/queries/1_stateful/00072_compare_date_and_string_index.reference @@ -11,16 +11,16 @@ 7466940 2789455 1383658 -9 -1448371 -7425518 -1448380 -7425527 -11 -9 -1448371 -7425518 -1448380 -7425527 -11 -9 +17 +1465475 +7408406 +1465492 +7408423 +36 +17 +1465475 +7408406 +1465492 +7408423 +36 +17 diff --git a/dbms/tests/queries/1_stateful/00073_uniq_array.reference b/dbms/tests/queries/1_stateful/00073_uniq_array.reference index d40e6f39073..c04ce5a00f4 100644 --- a/dbms/tests/queries/1_stateful/00073_uniq_array.reference +++ b/dbms/tests/queries/1_stateful/00073_uniq_array.reference @@ -1,7 +1,7 @@ -2014-03-17 7158 7158 7158 -2014-03-18 7122 7122 7122 -2014-03-19 7109 7109 7109 -2014-03-20 6997 6997 6997 -2014-03-21 6718 6718 6718 -2014-03-22 5716 5716 5716 -2014-03-23 5799 5799 5799 +2014-03-17 1109 1109 1109 +2014-03-18 1090 1090 1090 +2014-03-19 1124 1124 1124 +2014-03-20 1144 1144 1144 +2014-03-21 1038 1038 1038 +2014-03-22 745 745 745 +2014-03-23 697 697 697 diff --git a/dbms/tests/queries/1_stateful/00073_uniq_array.sql b/dbms/tests/queries/1_stateful/00073_uniq_array.sql index 74c031ee7a6..4e9ac282387 100644 --- a/dbms/tests/queries/1_stateful/00073_uniq_array.sql +++ b/dbms/tests/queries/1_stateful/00073_uniq_array.sql @@ -1 +1 @@ -SELECT EventDate, uniqExact(UserID), length(groupUniqArray(UserID)), arrayUniq(groupArray(UserID)) FROM test.hits WHERE CounterID = 731962 GROUP BY EventDate ORDER BY EventDate; +SELECT EventDate, uniqExact(UserID), length(groupUniqArray(UserID)), arrayUniq(groupArray(UserID)) FROM test.hits WHERE CounterID = 1704509 GROUP BY EventDate ORDER BY EventDate; diff --git a/dbms/tests/queries/1_stateful/00074_full_join.reference b/dbms/tests/queries/1_stateful/00074_full_join.reference index 0a4ec891a33..e806feadbe9 100644 --- a/dbms/tests/queries/1_stateful/00074_full_join.reference +++ b/dbms/tests/queries/1_stateful/00074_full_join.reference @@ -1,60 +1,60 @@ -22564972 1331 0 -21423112 357 0 -99165 72 0 -9409582 37 0 -93428 33 0 -50615 21 0 -21191842 21 0 -10274467 11 0 -14013802 0 1 -14013940 0 1 -15049483 0 1 -15834589 0 1 -16635889 0 1 -16837024 0 1 -16971091 0 1 -17757130 0 1 -18428878 0 1 -19738795 0 1 -20137870 0 1 -20338870 0 1 -22564972 1331 0 -21423112 357 0 -99165 72 0 -9409582 37 0 -93428 33 0 -50615 21 0 -21191842 21 0 -10274467 11 0 -54047 9 0 -24327391 9 0 -12133420 8 0 -22382245 8 0 -23498743 7 0 -23498989 7 0 -23799985 7 0 -84367 6 0 -1811350 5 0 -7490101 5 0 -22157924 5 0 -22334527 5 0 -14013802 0 1 -14013940 0 1 -15049483 0 1 -15834589 0 1 -16635889 0 1 -16837024 0 1 -16971091 0 1 -17757130 0 1 -18428878 0 1 -19738795 0 1 -20137870 0 1 -20338870 0 1 -20897719 0 1 -21165355 0 1 -21764326 0 1 -22101154 0 1 -22669492 0 1 -23218363 0 1 -23578141 0 1 -23712970 0 1 +37292 0 35642 +92887 252214 0 +7842 196036 0 +42440 0 17837 +97811 0 13665 +99378 0 11401 +6764 0 11052 +43564 0 9522 +37645 0 9086 +76197 0 8379 +24271 0 7393 +39804 64180 0 +78332 0 6167 +82837 0 5928 +51651 0 5878 +63469 52152 0 +76593 0 4705 +54688 0 4421 +36747 44082 0 +51763 0 4309 +92887 252214 0 +7842 196036 0 +39804 64180 0 +63469 52152 0 +36747 44082 0 +10963 37562 0 +46685 28798 0 +24015 23368 0 +37615 20857 0 +62896 19719 0 +67664 19402 0 +96597 18557 0 +11045 17391 0 +25884 17302 0 +64910 17279 0 +26096 16994 0 +96134 16849 0 +5788 13974 0 +71509 13792 0 +67326 13181 0 +37292 0 35642 +42440 0 17837 +97811 0 13665 +99378 0 11401 +6764 0 11052 +43564 0 9522 +37645 0 9086 +76197 0 8379 +24271 0 7393 +78332 0 6167 +82837 0 5928 +51651 0 5878 +76593 0 4705 +54688 0 4421 +51763 0 4309 +37882 0 4094 +6236 0 3959 +28456 0 3746 +19008 0 3426 +81009 0 3158 diff --git a/dbms/tests/queries/1_stateful/00074_full_join.sql b/dbms/tests/queries/1_stateful/00074_full_join.sql index 19550006baa..44c0987fd61 100644 --- a/dbms/tests/queries/1_stateful/00074_full_join.sql +++ b/dbms/tests/queries/1_stateful/00074_full_join.sql @@ -5,14 +5,14 @@ SELECT FROM ( SELECT - CounterID, + (CounterID % 100000) AS CounterID, count() AS hits FROM test.hits GROUP BY CounterID ) ANY FULL OUTER JOIN ( SELECT - CounterID, + (CounterID % 100000) AS CounterID, sum(Sign) AS visits FROM test.visits GROUP BY CounterID @@ -32,14 +32,14 @@ SELECT FROM ( SELECT - CounterID, + (CounterID % 100000) AS CounterID, count() AS hits FROM test.hits GROUP BY CounterID ) ANY LEFT JOIN ( SELECT - CounterID, + (CounterID % 100000) AS CounterID, sum(Sign) AS visits FROM test.visits GROUP BY CounterID @@ -59,14 +59,14 @@ SELECT FROM ( SELECT - CounterID, + (CounterID % 100000) AS CounterID, count() AS hits FROM test.hits GROUP BY CounterID ) ANY RIGHT JOIN ( SELECT - CounterID, + (CounterID % 100000) AS CounterID, sum(Sign) AS visits FROM test.visits GROUP BY CounterID @@ -86,14 +86,14 @@ SELECT FROM ( SELECT - CounterID, + (CounterID % 100000) AS CounterID, count() AS hits FROM test.hits GROUP BY CounterID ) ANY INNER JOIN ( SELECT - CounterID, + (CounterID % 100000) AS CounterID, sum(Sign) AS visits FROM test.visits GROUP BY CounterID diff --git a/dbms/tests/queries/1_stateful/00075_left_array_join.reference b/dbms/tests/queries/1_stateful/00075_left_array_join.reference index 52579b4a5ea..6cc8f2402fc 100644 --- a/dbms/tests/queries/1_stateful/00075_left_array_join.reference +++ b/dbms/tests/queries/1_stateful/00075_left_array_join.reference @@ -1,200 +1,200 @@ -4187429269810 2014-03-19 10:02:28 theme ['theme'] -4187429269810 2014-03-19 10:02:28 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -4187429269810 2014-03-19 10:02:28 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -4187429269810 2014-03-19 10:02:28 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -4187429269810 2014-03-19 10:02:28 Цифровой логин ['Цифровой логин'] -4187429269810 2014-03-19 10:02:28 Эксперимент про отрыв счетчика у папки СПАМ Счетчик есть ['Эксперимент про отрыв счетчика у папки СПАМ'] -4187429269810 2014-03-19 10:02:29 attach ['attach'] -4187429269810 2014-03-19 10:02:29 attach onefile ['attach'] -4187429269810 2014-03-19 10:02:29 Просмотр письма ['Просмотр письма'] -4187429269810 2014-03-19 10:02:30 Правая колонка Вкладка «Треды» ['Правая колонка'] -4187429269810 2014-03-19 10:02:30 Эксперимент про отрыв счетчика у папки СПАМ Счетчик есть ['Эксперимент про отрыв счетчика у папки СПАМ'] -4187429269810 2014-03-19 10:02:35 attach body ['attach'] -4187429269810 2014-03-19 10:03:47 Toolbar Входящие ['Toolbar'] -4187429269810 2014-03-19 10:03:48 Эксперимент про отрыв счетчика у папки СПАМ Счетчик есть ['Эксперимент про отрыв счетчика у папки СПАМ'] -4187429269810 2014-03-19 10:04:21 attach onefile ['attach'] -4187429269810 2014-03-19 10:04:22 attach ['attach'] -4187429269810 2014-03-19 10:04:22 Правая колонка Вкладка «Треды» ['Правая колонка'] -4187429269810 2014-03-19 10:04:22 Просмотр письма ['Просмотр письма'] -4187429269810 2014-03-19 10:04:28 attach body ['attach'] -4187429269810 2014-03-19 10:08:03 Залогиновая шапка. Правая часть Меню за логином ['Залогиновая шапка. Правая часть'] -4187429269810 2014-03-19 10:08:04 Залогиновая шапка. Правая часть Меню за логином ['Залогиновая шапка. Правая часть'] -142097207152117 2014-03-22 08:10:15 theme blue ['theme'] -142097207152117 2014-03-22 08:10:15 Залогиновая шапка. Правая часть Уголок ['Залогиновая шапка. Правая часть'] -142097207152117 2014-03-22 08:10:15 Промо сборщиков На всех ['Промо сборщиков'] -142097207152117 2014-03-22 08:10:17 registerProtocolHandler https: ['registerProtocolHandler'] -142097207152117 2014-03-22 08:10:30 Промо сборщиков На всех ['Промо сборщиков'] -142097207152117 2014-03-22 08:10:32 Правая колонка Вкладка «Треды» ['Правая колонка'] -142097207152117 2014-03-22 08:10:33 Просмотр письма показ ['Просмотр письма'] -340670786804313 2014-03-18 01:52:20 theme ['theme'] -340670786804313 2014-03-18 01:52:21 Цифровой логин ['Цифровой логин'] -340670786804313 2014-03-18 01:52:25 Правая колонка Вкладка «Треды» ['Правая колонка'] -340670786804313 2014-03-18 01:52:25 Просмотр письма ['Просмотр письма'] -340670786804313 2014-03-19 16:01:13 lcn ['lcn'] -340670786804313 2014-03-19 16:01:13 theme ['theme'] -340670786804313 2014-03-19 16:01:13 Погодная тема Показ города ['Погодная тема'] -340670786804313 2014-03-19 16:01:15 Правая колонка Вкладка «Треды» ['Правая колонка'] -340670786804313 2014-03-19 16:01:15 Просмотр письма ['Просмотр письма'] -523951316868678 2014-03-18 15:03:08 theme ['theme'] -523951316868678 2014-03-18 15:03:09 Цифровой логин ['Цифровой логин'] -523951316868678 2014-03-18 15:03:13 Просмотр письма ['Просмотр письма'] -523951316868678 2014-03-18 15:03:14 attach ['attach'] -523951316868678 2014-03-18 15:03:14 attach onefile ['attach'] -523951316868678 2014-03-18 15:03:14 Правая колонка Вкладка «Треды» ['Правая колонка'] -523951316868678 2014-03-18 17:19:31 theme ['theme'] -523951316868678 2014-03-18 17:19:31 Цифровой логин ['Цифровой логин'] -523951316868678 2014-03-18 17:19:32 lcn ['lcn'] -680431331227081 2014-03-18 08:15:14 lcn ['lcn'] -680431331227081 2014-03-18 08:15:14 theme ['theme'] -680431331227081 2014-03-18 08:15:15 Цифровой логин ['Цифровой логин'] -680431331227081 2014-03-18 08:15:17 Правая колонка Вкладка «Треды» ['Правая колонка'] -680431331227081 2014-03-18 08:15:17 Просмотр письма ['Просмотр письма'] -680431331227081 2014-03-18 08:16:10 Залогиновая шапка. Правая часть Меню за логином ['Залогиновая шапка. Правая часть'] -680431331227081 2014-03-18 08:16:13 Залогиновая шапка. Правая часть Меню за логином ['Залогиновая шапка. Правая часть'] -711535488569826 2014-03-17 09:07:22 theme ['theme'] -711535488569826 2014-03-17 09:07:22 Залогиновая шапка. Правая часть Уголок ['Залогиновая шапка. Правая часть'] -1385143435049415 2014-03-17 16:32:19 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 16:32:19 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 16:39:06 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 16:39:07 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 16:39:07 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 20:30:25 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 20:39:19 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 20:39:21 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 20:39:21 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 20:43:50 Toolbar Письмо ['Toolbar'] -1385143435049415 2014-03-17 20:43:50 lcn ['lcn'] -1385143435049415 2014-03-17 20:43:50 theme ['theme'] -1385143435049415 2014-03-17 20:43:50 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 20:43:50 Просмотр письма ['Просмотр письма'] -1385143435049415 2014-03-17 20:43:51 Toolbar Письмо ['Toolbar'] -1385143435049415 2014-03-17 20:43:51 Toolbar Письмо ['Toolbar'] -1385143435049415 2014-03-17 20:43:51 Toolbar Письмо ['Toolbar'] -1385143435049415 2014-03-17 20:43:51 Toolbar Письмо ['Toolbar'] -1385143435049415 2014-03-17 20:43:51 Toolbar Письмо ['Toolbar'] -1385143435049415 2014-03-17 20:43:51 Toolbar Письмо ['Toolbar'] -1385143435049415 2014-03-17 20:43:51 Правая колонка Вкладка «Треды» ['Правая колонка'] -1385143435049415 2014-03-17 20:43:51 Правая колонка Вкладка «Треды» ['Правая колонка'] -1385143435049415 2014-03-17 20:43:51 События в письмах ПДД ['События в письмах'] -1385143435049415 2014-03-17 20:44:07 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 20:44:12 Toolbar Входящие ['Toolbar'] -1385143435049415 2014-03-17 20:44:12 Toolbar Входящие ['Toolbar'] -1385143435049415 2014-03-17 20:44:12 Toolbar Входящие ['Toolbar'] -1385143435049415 2014-03-17 20:44:12 Toolbar Входящие ['Toolbar'] -1385143435049415 2014-03-17 20:44:12 Toolbar Входящие ['Toolbar'] -1385143435049415 2014-03-17 20:44:17 Toolbar Входящие ['Toolbar'] -1385143435049415 2014-03-17 20:44:18 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-18 11:36:47 Toolbar Входящие ['Toolbar'] -1385143435049415 2014-03-18 11:36:47 lcn ['lcn'] -1385143435049415 2014-03-18 11:36:47 theme ['theme'] -1385143435049415 2014-03-18 11:36:47 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-18 11:36:47 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-18 11:36:47 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-18 11:36:47 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-18 11:36:47 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-18 11:36:47 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-18 11:36:47 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-18 11:36:47 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-18 11:36:47 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-18 11:45:10 Toolbar Входящие ['Toolbar'] -1385143435049415 2014-03-18 11:45:10 lcn ['lcn'] -4187429269810 2014-03-19 10:02:28 [] -4187429269810 2014-03-19 10:02:28 theme ['theme'] -4187429269810 2014-03-19 10:02:28 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -4187429269810 2014-03-19 10:02:28 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -4187429269810 2014-03-19 10:02:28 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -4187429269810 2014-03-19 10:02:28 Цифровой логин ['Цифровой логин'] -4187429269810 2014-03-19 10:02:28 Эксперимент про отрыв счетчика у папки СПАМ Счетчик есть ['Эксперимент про отрыв счетчика у папки СПАМ'] -4187429269810 2014-03-19 10:02:29 attach ['attach'] -4187429269810 2014-03-19 10:02:29 attach onefile ['attach'] -4187429269810 2014-03-19 10:02:29 Просмотр письма ['Просмотр письма'] -4187429269810 2014-03-19 10:02:30 [] -4187429269810 2014-03-19 10:02:30 Правая колонка Вкладка «Треды» ['Правая колонка'] -4187429269810 2014-03-19 10:02:30 Эксперимент про отрыв счетчика у папки СПАМ Счетчик есть ['Эксперимент про отрыв счетчика у папки СПАМ'] -4187429269810 2014-03-19 10:02:35 [] -4187429269810 2014-03-19 10:02:35 attach body ['attach'] -4187429269810 2014-03-19 10:03:07 [] -4187429269810 2014-03-19 10:03:47 Toolbar Входящие ['Toolbar'] -4187429269810 2014-03-19 10:03:48 Эксперимент про отрыв счетчика у папки СПАМ Счетчик есть ['Эксперимент про отрыв счетчика у папки СПАМ'] -4187429269810 2014-03-19 10:04:21 attach onefile ['attach'] -4187429269810 2014-03-19 10:04:22 [] -4187429269810 2014-03-19 10:04:22 attach ['attach'] -4187429269810 2014-03-19 10:04:22 Правая колонка Вкладка «Треды» ['Правая колонка'] -4187429269810 2014-03-19 10:04:22 Просмотр письма ['Просмотр письма'] -4187429269810 2014-03-19 10:04:28 [] -4187429269810 2014-03-19 10:04:28 attach body ['attach'] -4187429269810 2014-03-19 10:08:03 Залогиновая шапка. Правая часть Меню за логином ['Залогиновая шапка. Правая часть'] -4187429269810 2014-03-19 10:08:04 Залогиновая шапка. Правая часть Меню за логином ['Залогиновая шапка. Правая часть'] -142097207152117 2014-03-22 08:10:15 [] -142097207152117 2014-03-22 08:10:15 [] -142097207152117 2014-03-22 08:10:15 theme blue ['theme'] -142097207152117 2014-03-22 08:10:15 Залогиновая шапка. Правая часть Уголок ['Залогиновая шапка. Правая часть'] -142097207152117 2014-03-22 08:10:15 Промо сборщиков На всех ['Промо сборщиков'] -142097207152117 2014-03-22 08:10:17 registerProtocolHandler https: ['registerProtocolHandler'] -142097207152117 2014-03-22 08:10:30 Промо сборщиков На всех ['Промо сборщиков'] -142097207152117 2014-03-22 08:10:32 Правая колонка Вкладка «Треды» ['Правая колонка'] -142097207152117 2014-03-22 08:10:33 [] -142097207152117 2014-03-22 08:10:33 Просмотр письма показ ['Просмотр письма'] -340670786804313 2014-03-18 01:52:20 [] -340670786804313 2014-03-18 01:52:20 theme ['theme'] -340670786804313 2014-03-18 01:52:21 Цифровой логин ['Цифровой логин'] -340670786804313 2014-03-18 01:52:25 Правая колонка Вкладка «Треды» ['Правая колонка'] -340670786804313 2014-03-18 01:52:25 Просмотр письма ['Просмотр письма'] -340670786804313 2014-03-18 01:52:26 [] -340670786804313 2014-03-19 16:01:13 [] -340670786804313 2014-03-19 16:01:13 lcn ['lcn'] -340670786804313 2014-03-19 16:01:13 theme ['theme'] -340670786804313 2014-03-19 16:01:13 Погодная тема Показ города ['Погодная тема'] -340670786804313 2014-03-19 16:01:15 [] -340670786804313 2014-03-19 16:01:15 Правая колонка Вкладка «Треды» ['Правая колонка'] -340670786804313 2014-03-19 16:01:15 Просмотр письма ['Просмотр письма'] -523951316868678 2014-03-18 15:03:08 [] -523951316868678 2014-03-18 15:03:08 theme ['theme'] -523951316868678 2014-03-18 15:03:09 Цифровой логин ['Цифровой логин'] -523951316868678 2014-03-18 15:03:13 Просмотр письма ['Просмотр письма'] -523951316868678 2014-03-18 15:03:14 [] -523951316868678 2014-03-18 15:03:14 attach ['attach'] -523951316868678 2014-03-18 15:03:14 attach onefile ['attach'] -523951316868678 2014-03-18 15:03:14 Правая колонка Вкладка «Треды» ['Правая колонка'] -523951316868678 2014-03-18 15:03:20 [] -523951316868678 2014-03-18 15:03:20 [] -523951316868678 2014-03-18 17:19:31 [] -523951316868678 2014-03-18 17:19:31 theme ['theme'] -523951316868678 2014-03-18 17:19:31 Цифровой логин ['Цифровой логин'] -523951316868678 2014-03-18 17:19:32 lcn ['lcn'] -610121304480129 2014-03-17 05:35:14 [] -610121304480129 2014-03-18 10:22:47 [] -610121304480129 2014-03-19 05:26:31 [] -610121304480129 2014-03-20 05:32:35 [] -610121304480129 2014-03-20 05:41:56 [] -610121304480129 2014-03-20 10:17:05 [] -610121304480129 2014-03-21 08:46:30 [] -610121304480129 2014-03-22 06:06:53 [] -610121304480129 2014-03-22 09:54:27 [] -680431331227081 2014-03-18 08:15:14 [] -680431331227081 2014-03-18 08:15:14 lcn ['lcn'] -680431331227081 2014-03-18 08:15:14 theme ['theme'] -680431331227081 2014-03-18 08:15:15 Цифровой логин ['Цифровой логин'] -680431331227081 2014-03-18 08:15:17 Правая колонка Вкладка «Треды» ['Правая колонка'] -680431331227081 2014-03-18 08:15:17 Просмотр письма ['Просмотр письма'] -680431331227081 2014-03-18 08:15:18 [] -680431331227081 2014-03-18 08:16:10 Залогиновая шапка. Правая часть Меню за логином ['Залогиновая шапка. Правая часть'] -680431331227081 2014-03-18 08:16:13 [] -680431331227081 2014-03-18 08:16:13 Залогиновая шапка. Правая часть Меню за логином ['Залогиновая шапка. Правая часть'] -711535488569826 2014-03-17 09:07:22 [] -711535488569826 2014-03-17 09:07:22 [] -711535488569826 2014-03-17 09:07:22 theme ['theme'] -711535488569826 2014-03-17 09:07:22 Залогиновая шапка. Правая часть Уголок ['Залогиновая шапка. Правая часть'] -711535488569826 2014-03-17 09:07:23 [] -711535488569826 2014-03-17 09:07:24 [] -1385143435049415 2014-03-17 16:32:19 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 16:32:19 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 16:39:06 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 16:39:07 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 16:39:07 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 20:30:25 [] -1385143435049415 2014-03-17 20:30:25 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 20:39:19 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 20:39:21 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 20:39:21 Аттачи в списке писем Превьюшки ['Аттачи в списке писем'] -1385143435049415 2014-03-17 20:43:50 [] +2961521519262 2014-03-19 04:12:45 gen_time ['gen_time'] +2961521519262 2014-03-19 04:12:45 gen_timestamp Счетчика нет (10 ['gen_timestamp'] +2961521519262 2014-03-19 04:12:45 hasTemplates Счетчик есть null ['hasTemplates'] +2961521519262 2014-03-19 04:12:45 hasTemplates Счетчик есть null ['hasTemplates'] +2961521519262 2014-03-19 04:12:45 hasTemplates Счетчик есть null ['hasTemplates'] +2961521519262 2014-03-19 04:12:45 Роль пользователя ['Роль пользователя'] +2961521519262 2014-03-19 04:12:46 videoid ['videoid'] +2961521519262 2014-03-19 04:12:46 videoid Done ['videoid'] +2961521519262 2014-03-19 04:12:46 Поиск ['Поиск'] +2961521519262 2014-03-19 04:12:47 gen_timestamp Счетчика нет (10 ['gen_timestamp'] +2961521519262 2014-03-19 04:12:47 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] +2961521519262 2014-03-19 04:12:51 videoid 8950vvvv ['videoid'] +2961521519262 2014-03-19 04:14:11 errors SMS ['errors'] +2961521519262 2014-03-19 04:14:12 gen_timestamp Счетчика нет (10 ['gen_timestamp'] +2961521519262 2014-03-19 04:15:11 videoid Done ['videoid'] +2961521519262 2014-03-19 04:15:12 videoid ['videoid'] +2961521519262 2014-03-19 04:15:12 Поиск ['Поиск'] +2961521519262 2014-03-19 04:15:12 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] +2961521519262 2014-03-19 04:15:17 videoid 8950vvvv ['videoid'] +2961521519262 2014-03-19 04:19:21 gen_timestamp 564.815 ['gen_timestamp'] +2961521519262 2014-03-19 04:19:22 gen_timestamp 564.815 ['gen_timestamp'] +164599821266083 2014-03-22 14:30:50 gen_time views ['gen_time'] +164599821266083 2014-03-22 14:30:50 gen_timestamp vkontakte,face element ['gen_timestamp'] +164599821266083 2014-03-22 14:30:50 Платность Превьюшки ['Платность'] +164599821266083 2014-03-22 14:30:53 Аттачи в списке /pages/biblio ['Аттачи в списке'] +164599821266083 2014-03-22 14:31:02 Платность Превьюшки ['Платность'] +164599821266083 2014-03-22 14:31:05 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] +164599821266083 2014-03-22 14:31:06 Поиск Счетчика ['Поиск'] +335386658089392 2014-03-18 08:28:17 gen_time ['gen_time'] +335386658089392 2014-03-18 08:28:18 Роль пользователя ['Роль пользователя'] +335386658089392 2014-03-18 08:28:25 Поиск ['Поиск'] +335386658089392 2014-03-18 08:28:26 videoid ['videoid'] +335386658089392 2014-03-18 08:28:26 videoid Done ['videoid'] +335386658089392 2014-03-18 08:28:26 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] +335386658089392 2014-03-18 09:48:43 gen_time ['gen_time'] +335386658089392 2014-03-18 09:48:43 Роль пользователя ['Роль пользователя'] +335386658089392 2014-03-18 09:48:44 Дра ['Дра'] +419794772979101 2014-03-18 02:08:42 gen_time ['gen_time'] +419794772979101 2014-03-18 02:08:43 Роль пользователя ['Роль пользователя'] +419794772979101 2014-03-18 02:08:50 Поиск ['Поиск'] +419794772979101 2014-03-18 02:08:50 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] +419794772979101 2014-03-19 18:26:49 gen_time ['gen_time'] +419794772979101 2014-03-19 18:26:49 Дра ['Дра'] +419794772979101 2014-03-19 18:26:49 Эксперимент про Счетчик есть null ['Эксперимент про'] +419794772979101 2014-03-19 18:26:52 Поиск ['Поиск'] +419794772979101 2014-03-19 18:26:52 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] +818136115259082 2014-03-18 15:33:51 gen_time ['gen_time'] +818136115259082 2014-03-18 15:33:51 Дра ['Дра'] +818136115259082 2014-03-18 15:33:52 Роль пользователя ['Роль пользователя'] +818136115259082 2014-03-18 15:33:55 Поиск ['Поиск'] +818136115259082 2014-03-18 15:33:55 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] +818136115259082 2014-03-18 15:34:42 gen_timestamp 564.815 ['gen_timestamp'] +818136115259082 2014-03-18 15:34:44 gen_timestamp 564.815 ['gen_timestamp'] +870023587671569 2014-03-17 02:17:44 gen_time ['gen_time'] +870023587671569 2014-03-17 02:17:44 gen_timestamp vkontakte,face element ['gen_timestamp'] +1391426411567902 2014-03-18 14:39:27 gen_time ['gen_time'] +1391426411567902 2014-03-18 14:39:27 gen_timestamp Счетчика нет (10 ['gen_timestamp'] +1391426411567902 2014-03-18 14:39:27 Платность model ['Платность'] +1391426411567902 2014-03-18 14:39:27 Эксперимент про Счетчик есть null ['Эксперимент про'] +1391426411567902 2014-03-18 14:39:37 videoid ['videoid'] +1391426411567902 2014-03-18 14:39:37 videoid Done ['videoid'] +1391426411567902 2014-03-18 14:39:37 Поиск ['Поиск'] +1391426411567902 2014-03-18 14:39:37 Поиск ['Поиск'] +1391426411567902 2014-03-18 14:39:37 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] +1391426411567902 2014-03-18 14:39:37 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] +1391426411567902 2014-03-18 14:40:07 gen_timestamp Счетчика нет (10 ['gen_timestamp'] +1391426411567902 2014-03-18 14:43:42 Toolbar auto 25%;\n\t\t\t\t \n \n return ['Toolbar'] +1391426411567902 2014-03-18 14:43:42 gen_timestamp Счетчика нет (10 ['gen_timestamp'] +1391426411567902 2014-03-18 14:43:42 gen_timestamp Счетчика нет (10 ['gen_timestamp'] +1391426411567902 2014-03-18 14:43:42 Права ['Права'] +1391426411567902 2014-03-18 14:43:44 gen_timestamp Счетчика нет (10 ['gen_timestamp'] +1391426411567902 2014-03-18 14:43:48 Toolbar Счетчик ['Toolbar'] +1391426411567902 2014-03-18 14:43:48 gen_timestamp Счетчика нет (10 ['gen_timestamp'] +1391426411567902 2014-03-18 14:43:48 Поиск ['Поиск'] +1391426411567902 2014-03-18 14:43:48 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] +1560113638322066 2014-03-17 17:21:19 hasTemplates Счетчик есть null ['hasTemplates'] +1560113638322066 2014-03-17 17:21:19 hasTemplates Счетчик есть null ['hasTemplates'] +1560113638322066 2014-03-17 17:28:21 hasTemplates Счетчик есть null ['hasTemplates'] +1560113638322066 2014-03-17 17:28:22 hasTemplates Счетчик есть null ['hasTemplates'] +1560113638322066 2014-03-17 17:28:22 hasTemplates Счетчик есть null ['hasTemplates'] +1560113638322066 2014-03-17 21:01:54 hasTemplates Счетчик есть null ['hasTemplates'] +1560113638322066 2014-03-17 21:15:06 hasTemplates Счетчик есть null ['hasTemplates'] +1560113638322066 2014-03-17 21:15:09 hasTemplates Счетчик есть null ['hasTemplates'] +1560113638322066 2014-03-17 21:15:09 hasTemplates Счетчик есть null ['hasTemplates'] +1560113638322066 2014-03-17 21:22:15 errors 1e521117369b39edaa078c8021df25 ['errors'] +1560113638322066 2014-03-17 21:22:15 gen_time ['gen_time'] +1560113638322066 2014-03-17 21:22:15 hasTemplates Счетчик есть null ['hasTemplates'] +1560113638322066 2014-03-17 21:22:15 Дра ['Дра'] +1560113638322066 2014-03-17 21:22:15 Поиск ['Поиск'] +1560113638322066 2014-03-17 21:22:16 Toolbar Счетчик ['Toolbar'] +1560113638322066 2014-03-17 21:22:16 errors 1e521117369b39edaa078c8021df25 ['errors'] +1560113638322066 2014-03-17 21:22:16 errors 1e521117369b39edaa078c8021df25 ['errors'] +1560113638322066 2014-03-17 21:22:16 errors 1e521117369b39edaa078c8021df25 ['errors'] +1560113638322066 2014-03-17 21:22:16 errors 1e521117369b39edaa078c8021df25 ['errors'] +1560113638322066 2014-03-17 21:22:16 errors 1e521117369b39edaa078c8021df25 ['errors'] +1560113638322066 2014-03-17 21:22:16 errors 1e521117369b39edaa078c8021df25 ['errors'] +1560113638322066 2014-03-17 21:22:16 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] +1560113638322066 2014-03-17 21:22:16 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] +1560113638322066 2014-03-17 21:22:42 hasTemplates Счетчик есть null ['hasTemplates'] +1560113638322066 2014-03-17 21:22:46 errors SMS ['errors'] +2961521519262 2014-03-19 04:12:45 [] +2961521519262 2014-03-19 04:12:45 gen_time ['gen_time'] +2961521519262 2014-03-19 04:12:45 gen_timestamp Счетчика нет (10 ['gen_timestamp'] +2961521519262 2014-03-19 04:12:45 hasTemplates Счетчик есть null ['hasTemplates'] +2961521519262 2014-03-19 04:12:45 hasTemplates Счетчик есть null ['hasTemplates'] +2961521519262 2014-03-19 04:12:45 hasTemplates Счетчик есть null ['hasTemplates'] +2961521519262 2014-03-19 04:12:45 Роль пользователя ['Роль пользователя'] +2961521519262 2014-03-19 04:12:46 videoid ['videoid'] +2961521519262 2014-03-19 04:12:46 videoid Done ['videoid'] +2961521519262 2014-03-19 04:12:46 Поиск ['Поиск'] +2961521519262 2014-03-19 04:12:47 [] +2961521519262 2014-03-19 04:12:47 gen_timestamp Счетчика нет (10 ['gen_timestamp'] +2961521519262 2014-03-19 04:12:47 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] +2961521519262 2014-03-19 04:12:51 [] +2961521519262 2014-03-19 04:12:51 videoid 8950vvvv ['videoid'] +2961521519262 2014-03-19 04:13:28 [] +2961521519262 2014-03-19 04:14:11 errors SMS ['errors'] +2961521519262 2014-03-19 04:14:12 gen_timestamp Счетчика нет (10 ['gen_timestamp'] +2961521519262 2014-03-19 04:15:11 videoid Done ['videoid'] +2961521519262 2014-03-19 04:15:12 [] +2961521519262 2014-03-19 04:15:12 videoid ['videoid'] +2961521519262 2014-03-19 04:15:12 Поиск ['Поиск'] +2961521519262 2014-03-19 04:15:12 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] +2961521519262 2014-03-19 04:15:17 [] +2961521519262 2014-03-19 04:15:17 videoid 8950vvvv ['videoid'] +2961521519262 2014-03-19 04:19:21 gen_timestamp 564.815 ['gen_timestamp'] +2961521519262 2014-03-19 04:19:22 gen_timestamp 564.815 ['gen_timestamp'] +164599821266083 2014-03-22 14:30:50 [] +164599821266083 2014-03-22 14:30:50 [] +164599821266083 2014-03-22 14:30:50 gen_time views ['gen_time'] +164599821266083 2014-03-22 14:30:50 gen_timestamp vkontakte,face element ['gen_timestamp'] +164599821266083 2014-03-22 14:30:50 Платность Превьюшки ['Платность'] +164599821266083 2014-03-22 14:30:53 Аттачи в списке /pages/biblio ['Аттачи в списке'] +164599821266083 2014-03-22 14:31:02 Платность Превьюшки ['Платность'] +164599821266083 2014-03-22 14:31:05 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] +164599821266083 2014-03-22 14:31:06 [] +164599821266083 2014-03-22 14:31:06 Поиск Счетчика ['Поиск'] +335386658089392 2014-03-18 08:28:17 [] +335386658089392 2014-03-18 08:28:17 gen_time ['gen_time'] +335386658089392 2014-03-18 08:28:18 Роль пользователя ['Роль пользователя'] +335386658089392 2014-03-18 08:28:25 Поиск ['Поиск'] +335386658089392 2014-03-18 08:28:26 [] +335386658089392 2014-03-18 08:28:26 videoid ['videoid'] +335386658089392 2014-03-18 08:28:26 videoid Done ['videoid'] +335386658089392 2014-03-18 08:28:26 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] +335386658089392 2014-03-18 08:28:31 [] +335386658089392 2014-03-18 08:28:31 [] +335386658089392 2014-03-18 09:48:43 [] +335386658089392 2014-03-18 09:48:43 gen_time ['gen_time'] +335386658089392 2014-03-18 09:48:43 Роль пользователя ['Роль пользователя'] +335386658089392 2014-03-18 09:48:44 Дра ['Дра'] +419794772979101 2014-03-18 02:08:42 [] +419794772979101 2014-03-18 02:08:42 gen_time ['gen_time'] +419794772979101 2014-03-18 02:08:43 Роль пользователя ['Роль пользователя'] +419794772979101 2014-03-18 02:08:50 Поиск ['Поиск'] +419794772979101 2014-03-18 02:08:50 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] +419794772979101 2014-03-18 02:08:51 [] +419794772979101 2014-03-19 18:26:49 [] +419794772979101 2014-03-19 18:26:49 gen_time ['gen_time'] +419794772979101 2014-03-19 18:26:49 Дра ['Дра'] +419794772979101 2014-03-19 18:26:49 Эксперимент про Счетчик есть null ['Эксперимент про'] +419794772979101 2014-03-19 18:26:52 [] +419794772979101 2014-03-19 18:26:52 Поиск ['Поиск'] +419794772979101 2014-03-19 18:26:52 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] +818136115259082 2014-03-18 15:33:51 [] +818136115259082 2014-03-18 15:33:51 gen_time ['gen_time'] +818136115259082 2014-03-18 15:33:51 Дра ['Дра'] +818136115259082 2014-03-18 15:33:52 Роль пользователя ['Роль пользователя'] +818136115259082 2014-03-18 15:33:55 Поиск ['Поиск'] +818136115259082 2014-03-18 15:33:55 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] +818136115259082 2014-03-18 15:33:56 [] +818136115259082 2014-03-18 15:34:42 gen_timestamp 564.815 ['gen_timestamp'] +818136115259082 2014-03-18 15:34:44 [] +818136115259082 2014-03-18 15:34:44 gen_timestamp 564.815 ['gen_timestamp'] +870023587671569 2014-03-17 02:17:44 [] +870023587671569 2014-03-17 02:17:44 [] +870023587671569 2014-03-17 02:17:44 gen_time ['gen_time'] +870023587671569 2014-03-17 02:17:44 gen_timestamp vkontakte,face element ['gen_timestamp'] +870023587671569 2014-03-17 02:17:45 [] +870023587671569 2014-03-17 02:17:46 [] +1026297835113727 2014-03-17 22:41:53 [] +1026297835113727 2014-03-18 06:37:28 [] +1026297835113727 2014-03-19 22:26:53 [] +1026297835113727 2014-03-20 09:37:49 [] +1026297835113727 2014-03-20 09:49:38 [] +1026297835113727 2014-03-20 17:39:34 [] +1026297835113727 2014-03-21 11:38:39 [] +1026297835113727 2014-03-22 11:13:26 [] +1026297835113727 2014-03-22 14:50:25 [] +1391426411567902 2014-03-18 14:39:27 [] +1391426411567902 2014-03-18 14:39:27 gen_time ['gen_time'] +1391426411567902 2014-03-18 14:39:27 gen_timestamp Счетчика нет (10 ['gen_timestamp'] +1391426411567902 2014-03-18 14:39:27 Платность model ['Платность'] +1391426411567902 2014-03-18 14:39:27 Эксперимент про Счетчик есть null ['Эксперимент про'] +1391426411567902 2014-03-18 14:39:37 [] +1391426411567902 2014-03-18 14:39:37 videoid ['videoid'] +1391426411567902 2014-03-18 14:39:37 videoid Done ['videoid'] +1391426411567902 2014-03-18 14:39:37 Поиск ['Поиск'] +1391426411567902 2014-03-18 14:39:37 Поиск ['Поиск'] +1391426411567902 2014-03-18 14:39:37 Эксперимент /f/1/global/banners_v1_1.js ['Эксперимент'] diff --git a/dbms/tests/queries/1_stateful/00075_left_array_join.sql b/dbms/tests/queries/1_stateful/00075_left_array_join.sql index 0de215ceb36..424276cf036 100644 --- a/dbms/tests/queries/1_stateful/00075_left_array_join.sql +++ b/dbms/tests/queries/1_stateful/00075_left_array_join.sql @@ -1,2 +1,2 @@ -SELECT UserID, EventTime, pp.Key1, pp.Key2, ParsedParams.Key1 FROM test.hits ARRAY JOIN ParsedParams AS pp WHERE CounterID = 1143050 ORDER BY UserID, EventTime, pp.Key1, pp.Key2 LIMIT 100; -SELECT UserID, EventTime, pp.Key1, pp.Key2, ParsedParams.Key1 FROM test.hits LEFT ARRAY JOIN ParsedParams AS pp WHERE CounterID = 1143050 ORDER BY UserID, EventTime, pp.Key1, pp.Key2 LIMIT 100; +SELECT UserID, EventTime, pp.Key1, pp.Key2, ParsedParams.Key1 FROM test.hits ARRAY JOIN ParsedParams AS pp WHERE CounterID = 1704509 ORDER BY UserID, EventTime, pp.Key1, pp.Key2 LIMIT 100; +SELECT UserID, EventTime, pp.Key1, pp.Key2, ParsedParams.Key1 FROM test.hits LEFT ARRAY JOIN ParsedParams AS pp WHERE CounterID = 1704509 ORDER BY UserID, EventTime, pp.Key1, pp.Key2 LIMIT 100; diff --git a/dbms/tests/queries/1_stateful/00077_log_tinylog_stripelog.reference b/dbms/tests/queries/1_stateful/00077_log_tinylog_stripelog.reference index 1dcb023e190..98f50687de4 100644 --- a/dbms/tests/queries/1_stateful/00077_log_tinylog_stripelog.reference +++ b/dbms/tests/queries/1_stateful/00077_log_tinylog_stripelog.reference @@ -1,10 +1,10 @@ 1 1 1 -8873898 5148628775290158164 -8873898 5148628775290158164 -8873898 5148628775290158164 -8873898 5148628775290158164 +8873898 12457120258355519194 +8873898 12457120258355519194 +8873898 12457120258355519194 +8873898 12457120258355519194 1 1 1 diff --git a/dbms/tests/queries/1_stateful/00078_group_by_arrays.reference b/dbms/tests/queries/1_stateful/00078_group_by_arrays.reference index 5f932f4649f..41f3322f20b 100644 --- a/dbms/tests/queries/1_stateful/00078_group_by_arrays.reference +++ b/dbms/tests/queries/1_stateful/00078_group_by_arrays.reference @@ -1,40 +1,40 @@ [] 8001320 -[1698655] 218383 -[3288004] 26325 -[2793757] 26259 -[3308677,2622979,2532754,2532649,2532640,2532631,2532622,2532613,2623111,2532604,2532448,2532439,2532430,2532718,2532568,2532421,2532709,2532412,2532403,2532520,2532727,2532577,2532658,2532367,2532736,2532586,2532376,2532745,2532595,2532385,2532394,2532457,2532763,2532475,2532772,2532484,2532781,2532493,2532502,2532511,2532529] 24078 -[2459053] 15706 -[3658087] 14326 -[3288145] 11558 -[3875836] 9599 -[4208059] 8938 +[2028343] 218383 +[3337849] 26325 +[3278201] 26259 +[3122469,2402651,3172687,2165967,3580344,3385706,3892934,3011216,3849284,2233086,3815288,3559435,2729510,3705435,2354960,3002888,2895835,3062712,2921977,3334920,3047153,3587682,4170284,2433778,2874188,2934295,3605952,2442464,3942660,2342155,2634729,3315647,2721173,2510647,2115204,3549218,3898387,2671488,3704331,3091775,3374519] 24078 +[3916793] 15706 +[4028285] 14326 +[2287648] 11558 +[3482255] 9599 +[7415385] 8938 [] [] 3929946 -[] [1698655] 88062 -[9] [] 21709 +[] [2028343] 88062 +[14] [] 21709 [6] [] 18019 -[90] [] 13936 +[76] [] 13936 [1] [] 12855 -[99] [] 12368 -[20,1] [] 11949 -[4,5] [] 11550 -[4] [] 11530 +[119] [] 12368 +[22,1] [] 11949 +[5,4] [] 11550 +[5] [] 11530 [] [] 3970023 -['gen_time'] [] 74637 -[] [9] 22059 -['Эксперимент про отрыв счетчика у папки СПАМ'] [] 18814 -['Toolbar'] [] 18168 -['Правая колонка'] [] 18132 -['Аттачи в списке писем'] [] 16735 -['Драгндроп папок'] [] 14652 +['css_link_href'] [] 74739 +['gen_timestamp'] [] 32442 +[] [14] 22059 +['Эксперимент'] [] 19158 +['Поиск'] [] 18966 +['hasTemplates'] [] 18378 +['errors'] [] 18168 +['page(by type of visitors)'] [] 14653 [] [6] 14350 -[] [90] 14183 -['Banner'] [5,4,3,118,11,1,37,38,3281,3201,3579,15,2] 1472 -['Аттачи в списке писем'] [7,6,82,9,3135,1,105,96,333,437,95,19,76] 1226 -['Аттачи в списке писем'] [2873,2866,9,96,635,3,39,26,12,2,8,11,37] 1173 -['registration_month','user_login','is_registred'] [4,90,5,601,2894,2893,6,68,399,11,88,117] 937 -['gen_time'] [9] 881 -['Аттачи в списке писем'] [7,6,82,9,3135,1,105,333,96,437,95,19,76] 859 -['Аттачи в списке писем'] [2873,2866,9,96,635,39,3,26,12,8,2,11,118] 837 -['Banner'] [635,1,9,90,96,19,2,3281,3349,8,5,3579] 777 -['Banner'] [90,1,635,19,9,96,2,3579,8] 766 -['gallery'] [7,76,636,460,6,99,4] 750 +['index'] [4,5,3,103,10,1,42,51,3993,2817,2193,15,2] 1472 +['hasTemplates'] [7,6,101,14,3003,1,70,72,501,285,80,25,74] 1226 +['hasTemplates'] [3528,3274,14,72,925,3,47,19,11,2,9,10,42] 1187 +['Sex','Просмотрщик','Неработающий'] [5,76,4,753,2733,3674,6,84,265,10,82,112] 937 +['css_link_href'] [14] 881 +['hasTemplates'] [7,6,101,14,3003,1,70,501,72,285,80,25,74] 859 +['hasTemplates'] [3528,3274,14,72,925,47,3,19,11,9,2,10,103] 843 +['Поиск'] [3528,3274,14,72,925,47,3,19,11,9,2,10,103] 817 +['index'] [925,1,14,76,72,25,2,3993,3275,9,4,2193] 777 +['index'] [76,1,925,25,14,72,2,2193,9] 766 diff --git a/dbms/tests/queries/1_stateful/00079_array_join_not_used_joined_column.reference b/dbms/tests/queries/1_stateful/00079_array_join_not_used_joined_column.reference index 1b1d514856f..ec1cd3bee44 100644 --- a/dbms/tests/queries/1_stateful/00079_array_join_not_used_joined_column.reference +++ b/dbms/tests/queries/1_stateful/00079_array_join_not_used_joined_column.reference @@ -1,104 +1,68 @@ -"Пригласи друга" в inbox — Беларусь 1 -"Пригласи друга" на done 0 -Click in compose 0 -Compose-Translate 4699 -Done 4032 -Done-Promo 2 -Hotels 0 -Money 0 -Search 0 -TinyMCE 15 -Todo 4 -Toolbar 0 -User Button 0 -attach 8010 -aviaeticket 0 -hot-keys 0 -lcn 14387 -popular_suggest 1855 -registerProtocolHandler 76 -theme 15777 -Аттачи в списке писем 0 -Аттачи из Диска 3401 -Баббл 0 -Башкортостан 2 -Виджет валидации (внутри письма) 0 -Визард (эксперимент) 0 -ГБ за ДР 0 -Директ над письмами 1 -Драгндроп папок 24211 -Живые письма в почте 1 -Залогиновая шапка. Правая часть 0 -Как делалась пластилиновая тема 2 -Карточка контакта 71 -Кнопка Прикрепить 7149 -МЯП->Валидация 0 -Марка 2389 -Настроение 5 -Настройки 0 -Неработающий сборщик 695 -Нижегородская 2 -Новосибирская 2 -Новый поп-ап фильтров 0 -Ответить всем 0 -Пейджер по датам 2986 -Переход к недоступному письму 23 -Плашка про старый браузер 840 -Погодная тема 0 -Подписи 0 -Поиск 5264 -Поменять пароль-2 11 -Поп-ап создания папки 26 -Попап для цифровых логинов 0 -Правая колонка 1418 -Предложение фильтровать групоны 0 -Промо Языков 0 -Промо меток 0 -Промо полоска элементов 0 -Промо сборщиков 0 -Просмотр письма 25023 -Просмотрщик картинок 10920 -Ревалидация 90 -Самарская 4 -События в письмах 0 -Сообщение о непоказе картинок и ссылок в спаме 456 -Сообщение о пересылке 2 -Статуслайн для фильтров 0 -Статуслайн игнорирования треда 1 -Татарстан 0 -Тема КХЛ 4 -Тема мишки 425 -Тема: region_primorie 1 -Тизер социальных новостей 6 -Томск 0 -Установка браузера 12 -Фильтр по типам писем 0 -Цифровой логин 3878 -Что нового 1 -Шаблоны 0 -Эксперимент про отрыв счетчика у папки СПАМ 0 -отправить адресату SMS-уведомление после отправки письма 0 -промо SMS 0 -промо-полоска Волгограда 1 -промо-полоска Торпедо 0 -промо-полоска Трактора 2 -"Пригласи друга" в inbox — Беларусь [''] -"Пригласи друга" на done ['Новая простая'] -"Пригласи друга" на done ['Новая простая'] -"Пригласи друга" на done ['Новая простая'] -"Пригласи друга" на done ['Новая простая'] -"Пригласи друга" на done ['Новая простая'] -"Пригласи друга" на done ['Новая простая'] -"Пригласи друга" на done ['Новая простая'] -"Пригласи друга" на done ['Новая простая'] -"Пригласи друга" на done ['Новая простая'] -['черный'] -['черный'] -['черн'] -['черн'] -['черн'] -['черн'] -['черн'] -['фото автора и из архива редакции'] -['фото Георгий Садков'] -['фото Александр Кульнев'] +Authenticated 0 +Authenticategory 0 +Banner 0 +Compose-Translates 28 +Sex 0 +Toolbar 3 +attach 696 +authorid 7091 +errors 0 +gallery 71 +gen_time 15777 +gen_timestamp 858 +group (All) 430 +hasTemplates 3401 +is_registred 0 +lcn 0 +logged_in 7149 +mobile 4 +page(by type of visitors) 24213 +pageParams 0 +regionId 90 +rl 1 +tag 23 +testingGroup (All) 0 +theme 2986 +videoid 8010 +Аттачи в списке 76 +Дра 14387 +Драгндроп папки СПАМ 12 +Пейджер 5264 +Платность 0 +Плашка превью 0 +Плашка про отрыв счетчика 2 +Плашка про отрыв счетчика у папки 0 +Плашка про отрыв счетчика у папок 1 +Поиск 35943 +Права 4032 +Правая 0 +Правая колонка 4 +Просмотр 1857 +Просмотр писем 0 +Роль пользователя 3878 +Система 0 +Экспер 15 +Эксперимент 1423 +Эксперимент про 0 +Эксперимент про отрыв счетчика у 2 +Эксперимент про отрыв счетчика у папки СПАМ 456 +Authenticated ['Вкладка «Треды»'] +Authenticated ['Вкладка «Треды»'] +Authenticated ['Вкладка «Треды»'] +Authenticated ['Вкладка «Треды»'] +Authenticated ['Вкладка «Треды»'] +Authenticated ['Вкладка «Треды»'] +Authenticated ['Вкладка «Треды»'] +Authenticated ['Вкладка «Треды»'] +Authenticated ['Вкладка «Треды»'] +Authenticated ['Вкладка «Треды»'] +['уведомить без Дисковой',''] +['уведомить без Дисковой',''] +['уведомить без Диска»',''] +['уведомить без Диска»',''] +['уведомить без Диска»',''] +['уведомить без Диска»',''] +['уведомит'] +['уведо'] +['уведо'] +['уведо'] diff --git a/dbms/tests/queries/1_stateful/00079_array_join_not_used_joined_column.sql b/dbms/tests/queries/1_stateful/00079_array_join_not_used_joined_column.sql index 6474c44c0b6..8e6742bb1e1 100644 --- a/dbms/tests/queries/1_stateful/00079_array_join_not_used_joined_column.sql +++ b/dbms/tests/queries/1_stateful/00079_array_join_not_used_joined_column.sql @@ -1,3 +1,3 @@ -SELECT PP.Key1 AS `ym:s:paramsLevel1`, sum(arrayAll(`x_1` -> `x_1`= '', ParsedParams.Key2)) AS `ym:s:visits` FROM test.hits ARRAY JOIN ParsedParams AS `PP` WHERE CounterID = 1143050 GROUP BY `ym:s:paramsLevel1` ORDER BY PP.Key1, `ym:s:visits` LIMIT 0, 100; -SELECT PP.Key1 AS x1, ParsedParams.Key2 AS x2 FROM test.hits ARRAY JOIN ParsedParams AS PP WHERE CounterID = 1143050 ORDER BY x1, x2 LIMIT 10; +SELECT PP.Key1 AS `ym:s:paramsLevel1`, sum(arrayAll(`x_1` -> `x_1`= '', ParsedParams.Key2)) AS `ym:s:visits` FROM test.hits ARRAY JOIN ParsedParams AS `PP` WHERE CounterID = 1704509 GROUP BY `ym:s:paramsLevel1` ORDER BY PP.Key1, `ym:s:visits` LIMIT 0, 100; +SELECT PP.Key1 AS x1, ParsedParams.Key2 AS x2 FROM test.hits ARRAY JOIN ParsedParams AS PP WHERE CounterID = 1704509 ORDER BY x1, x2 LIMIT 10; SELECT ParsedParams.Key2 AS x FROM test.hits ARRAY JOIN ParsedParams AS PP ORDER BY x DESC LIMIT 10; diff --git a/dbms/tests/queries/1_stateful/00080_array_join_and_union.sql b/dbms/tests/queries/1_stateful/00080_array_join_and_union.sql index eab92ad99ed..d9aa1cc17cc 100644 --- a/dbms/tests/queries/1_stateful/00080_array_join_and_union.sql +++ b/dbms/tests/queries/1_stateful/00080_array_join_and_union.sql @@ -1 +1 @@ -SELECT count() FROM (SELECT Goals.ID FROM test.visits ARRAY JOIN Goals WHERE CounterID = 101500 LIMIT 10 UNION ALL SELECT Goals.ID FROM test.visits ARRAY JOIN Goals WHERE CounterID = 101500 LIMIT 10); +SELECT count() FROM (SELECT Goals.ID FROM test.visits ARRAY JOIN Goals WHERE CounterID = 842440 LIMIT 10 UNION ALL SELECT Goals.ID FROM test.visits ARRAY JOIN Goals WHERE CounterID = 842440 LIMIT 10); diff --git a/dbms/tests/queries/1_stateful/00081_group_by_without_key_and_totals.reference b/dbms/tests/queries/1_stateful/00081_group_by_without_key_and_totals.reference index a29f53e89c6..fc48d81d7bb 100644 --- a/dbms/tests/queries/1_stateful/00081_group_by_without_key_and_totals.reference +++ b/dbms/tests/queries/1_stateful/00081_group_by_without_key_and_totals.reference @@ -1,60 +1,56 @@ -475698 +523264 -475698 -475698 +523264 +523264 -475698 -475698 +523264 +523264 -475698 -475698 +523264 +523264 -475698 -1 475698 +523264 +1 523264 -1 475698 -1 475698 +1 523264 +1 523264 -1 475698 -1 475698 +1 523264 +1 523264 -1 475698 -1 475698 +1 523264 +1 523264 -1 475698 --1 237160 -0 182238 -1 3594 -2 704 -5 2357 -6 49622 -8 23 +1 523264 +-1 86614 +0 413533 +1 9984 +3 1203 +5 11927 +10 3 -0 475698 --1 237160 -0 182238 -1 3594 -2 704 -5 2357 -6 49622 -8 23 +0 523264 +-1 86614 +0 413533 +1 9984 +3 1203 +5 11927 +10 3 -0 475698 --1 237160 -0 182238 -1 3594 -2 704 -5 2357 -6 49622 -8 23 +0 523264 +-1 86614 +0 413533 +1 9984 +3 1203 +5 11927 +10 3 -0 475698 --1 237160 -0 182238 -1 3594 -2 704 -5 2357 -6 49622 -8 23 +0 523264 +-1 86614 +0 413533 +1 9984 +3 1203 +5 11927 +10 3 -0 475698 +0 523264 diff --git a/dbms/tests/queries/1_stateful/00081_group_by_without_key_and_totals.sql b/dbms/tests/queries/1_stateful/00081_group_by_without_key_and_totals.sql index ec5bdca5d85..d9653f48552 100644 --- a/dbms/tests/queries/1_stateful/00081_group_by_without_key_and_totals.sql +++ b/dbms/tests/queries/1_stateful/00081_group_by_without_key_and_totals.sql @@ -1,15 +1,15 @@ -SELECT count() AS c FROM test.hits WHERE CounterID = 731962 WITH TOTALS SETTINGS totals_mode = 'before_having', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; -SELECT count() AS c FROM test.hits WHERE CounterID = 731962 WITH TOTALS SETTINGS totals_mode = 'after_having_inclusive', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; -SELECT count() AS c FROM test.hits WHERE CounterID = 731962 WITH TOTALS SETTINGS totals_mode = 'after_having_exclusive', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; -SELECT count() AS c FROM test.hits WHERE CounterID = 731962 WITH TOTALS SETTINGS totals_mode = 'after_having_auto', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; +SELECT count() AS c FROM test.hits WHERE CounterID = 1704509 WITH TOTALS SETTINGS totals_mode = 'before_having', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; +SELECT count() AS c FROM test.hits WHERE CounterID = 1704509 WITH TOTALS SETTINGS totals_mode = 'after_having_inclusive', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; +SELECT count() AS c FROM test.hits WHERE CounterID = 1704509 WITH TOTALS SETTINGS totals_mode = 'after_having_exclusive', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; +SELECT count() AS c FROM test.hits WHERE CounterID = 1704509 WITH TOTALS SETTINGS totals_mode = 'after_having_auto', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; -SELECT 1 AS k, count() AS c FROM test.hits WHERE CounterID = 731962 GROUP BY k WITH TOTALS SETTINGS totals_mode = 'before_having', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; -SELECT 1 AS k, count() AS c FROM test.hits WHERE CounterID = 731962 GROUP BY k WITH TOTALS SETTINGS totals_mode = 'after_having_inclusive', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; -SELECT 1 AS k, count() AS c FROM test.hits WHERE CounterID = 731962 GROUP BY k WITH TOTALS SETTINGS totals_mode = 'after_having_exclusive', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; -SELECT 1 AS k, count() AS c FROM test.hits WHERE CounterID = 731962 GROUP BY k WITH TOTALS SETTINGS totals_mode = 'after_having_auto', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; +SELECT 1 AS k, count() AS c FROM test.hits WHERE CounterID = 1704509 GROUP BY k WITH TOTALS SETTINGS totals_mode = 'before_having', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; +SELECT 1 AS k, count() AS c FROM test.hits WHERE CounterID = 1704509 GROUP BY k WITH TOTALS SETTINGS totals_mode = 'after_having_inclusive', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; +SELECT 1 AS k, count() AS c FROM test.hits WHERE CounterID = 1704509 GROUP BY k WITH TOTALS SETTINGS totals_mode = 'after_having_exclusive', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; +SELECT 1 AS k, count() AS c FROM test.hits WHERE CounterID = 1704509 GROUP BY k WITH TOTALS SETTINGS totals_mode = 'after_having_auto', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; -SELECT TraficSourceID AS k, count() AS c FROM test.hits WHERE CounterID = 731962 GROUP BY k WITH TOTALS ORDER BY k SETTINGS totals_mode = 'before_having', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; -SELECT TraficSourceID AS k, count() AS c FROM test.hits WHERE CounterID = 731962 GROUP BY k WITH TOTALS ORDER BY k SETTINGS totals_mode = 'after_having_inclusive', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; -SELECT TraficSourceID AS k, count() AS c FROM test.hits WHERE CounterID = 731962 GROUP BY k WITH TOTALS ORDER BY k SETTINGS totals_mode = 'after_having_exclusive', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; -SELECT TraficSourceID AS k, count() AS c FROM test.hits WHERE CounterID = 731962 GROUP BY k WITH TOTALS ORDER BY k SETTINGS totals_mode = 'after_having_auto', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; +SELECT TraficSourceID AS k, count() AS c FROM test.hits WHERE CounterID = 1704509 GROUP BY k WITH TOTALS ORDER BY k SETTINGS totals_mode = 'before_having', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; +SELECT TraficSourceID AS k, count() AS c FROM test.hits WHERE CounterID = 1704509 GROUP BY k WITH TOTALS ORDER BY k SETTINGS totals_mode = 'after_having_inclusive', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; +SELECT TraficSourceID AS k, count() AS c FROM test.hits WHERE CounterID = 1704509 GROUP BY k WITH TOTALS ORDER BY k SETTINGS totals_mode = 'after_having_exclusive', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; +SELECT TraficSourceID AS k, count() AS c FROM test.hits WHERE CounterID = 1704509 GROUP BY k WITH TOTALS ORDER BY k SETTINGS totals_mode = 'after_having_auto', max_rows_to_group_by = 100000, group_by_overflow_mode = 'any'; diff --git a/dbms/tests/queries/1_stateful/00082_quantiles.reference b/dbms/tests/queries/1_stateful/00082_quantiles.reference index 5a841b948ea..d2f5616909c 100644 --- a/dbms/tests/queries/1_stateful/00082_quantiles.reference +++ b/dbms/tests/queries/1_stateful/00082_quantiles.reference @@ -1,80 +1,80 @@ -1143050 1366 -731962 1366 -722545 1366 -722889 1366 -2237260 1366 -23057320 1366 -722818 1366 -48221 1280 -19762435 1366 -722884 1366 -1143050 [1024,1366,1920,1920,2560] -731962 [1024,1366,1920,1920,2560] -722545 [1024,1366,1920,1920,2259] -722889 [1152,1366,1920,1920,2560] -2237260 [1024,1366,1680,1920,1920] -23057320 [914,1366,1680,1920,2560] -722818 [1024,1366,1920,1920,2560] -48221 [320,1280,1600,1920,2000] -19762435 [1024,1366,1680,1920,1920] -722884 [1024,1366,1920,1920,1920] -1143050 1353 -731962 1353 -722545 1353 -722889 1353 -2237260 1353 -23057320 1353 -722818 1353 -48221 1283 -19762435 1353 -722884 1353 -1143050 [1016,1353,1916,1916,2559] -731962 [1016,1353,1916,1916,2559] -722545 [1016,1353,1916,1916,2258] -722889 [1149,1353,1916,1916,2559] -2237260 [1016,1353,1687,1916,1916] -23057320 [914,1353,1687,1916,2559] -722818 [1016,1353,1916,1916,2559] -48221 [320,1283,1606,1916,2006] -19762435 [1016,1353,1687,1916,1916] -722884 [1016,1353,1916,1916,1916] -1143050 1366 -731962 1366 -722545 1366 -722889 1366 -2237260 1366 -23057320 1366 -722818 1366 -48221 1280 -19762435 1366 -722884 1366 -1143050 [1024,1366,1920,1920,2560] -731962 [1024,1366,1920,1920,2560] -722545 [1024,1366,1920,1920,2259] -722889 [1152,1366,1920,1920,2560] -2237260 [1024,1366,1680,1920,1920] -23057320 [914,1366,1680,1920,2560] -722818 [1024,1366,1920,1920,2560] -48221 [320,1280,1600,1920,2000] -19762435 [1024,1366,1680,1920,1920] -722884 [1024,1366,1920,1920,1920] -1143050 1353 -731962 1353 -722545 1353 -722889 1353 -2237260 1353 -23057320 1353 -722818 1353 -48221 1283 -19762435 1353 -722884 1353 -1143050 [1016,1353,1916,1916,2559] -731962 [1016,1353,1916,1916,2559] -722545 [1016,1353,1916,1916,2258] -722889 [1149,1353,1916,1916,2559] -2237260 [1016,1353,1687,1916,1916] -23057320 [914,1353,1687,1916,2559] -722818 [1016,1353,1916,1916,2559] -48221 [320,1283,1606,1916,2006] -19762435 [1016,1353,1687,1916,1916] -722884 [1016,1353,1916,1916,1916] +1704509 1386 +732797 1339 +598875 1386 +792887 1339 +3807842 1339 +25703952 1339 +716829 1386 +59183 1339 +33010362 1339 +800784 1339 +1704509 [1297,1386,1846,1967,3701] +732797 [1234,1339,1846,1951,3665] +598875 [1234,1386,1846,1951,3551] +792887 [1297,1339,1846,1895,3701] +3807842 [1234,1339,1846,1939,2034] +25703952 [1012,1339,1846,1951,3701] +716829 [1234,1386,1846,1951,3701] +59183 [317,1339,1846,2010,2034] +33010362 [1234,1339,1846,1939,2034] +800784 [1234,1339,1846,1933,2034] +1704509 1378 +732797 1323 +598875 1378 +792887 1323 +3807842 1323 +25703952 1323 +716829 1378 +59183 1323 +33010362 1323 +800784 1323 +1704509 [1298,1378,1843,1956,3694] +732797 [1235,1323,1843,1935,3659] +598875 [1235,1378,1843,1935,3541] +792887 [1298,1323,1843,1883,3694] +3807842 [1235,1323,1843,1935,2027] +25703952 [1012,1323,1843,1935,3694] +716829 [1235,1378,1843,1935,3694] +59183 [317,1323,1843,2006,2027] +33010362 [1235,1323,1843,1935,2027] +800784 [1235,1323,1843,1916,2027] +1704509 1386 +732797 1339 +598875 1386 +792887 1339 +3807842 1339 +25703952 1339 +716829 1386 +59183 1339 +33010362 1339 +800784 1339 +1704509 [1297,1386,1846,1967,3701] +732797 [1234,1339,1846,1951,3665] +598875 [1234,1386,1846,1951,3551] +792887 [1297,1339,1846,1895,3701] +3807842 [1234,1339,1846,1939,2034] +25703952 [1012,1339,1846,1951,3701] +716829 [1234,1386,1846,1951,3701] +59183 [317,1339,1846,2010,2034] +33010362 [1234,1339,1846,1939,2034] +800784 [1234,1339,1846,1933,2034] +1704509 1378 +732797 1323 +598875 1378 +792887 1323 +3807842 1323 +25703952 1323 +716829 1378 +59183 1323 +33010362 1323 +800784 1323 +1704509 [1298,1378,1843,1956,3694] +732797 [1235,1323,1843,1935,3659] +598875 [1235,1378,1843,1935,3541] +792887 [1298,1323,1843,1883,3694] +3807842 [1235,1323,1843,1935,2027] +25703952 [1012,1323,1843,1935,3694] +716829 [1235,1378,1843,1935,3694] +59183 [317,1323,1843,2006,2027] +33010362 [1235,1323,1843,1935,2027] +800784 [1235,1323,1843,1916,2027] diff --git a/dbms/tests/queries/1_stateful/00084_external_aggregation.reference b/dbms/tests/queries/1_stateful/00084_external_aggregation.reference index 89c454decf6..bd2d35881bf 100644 --- a/dbms/tests/queries/1_stateful/00084_external_aggregation.reference +++ b/dbms/tests/queries/1_stateful/00084_external_aggregation.reference @@ -1,20 +1,20 @@ -https://mail.yandex.ru/neo2/#inbox 594 -http://yandex.ru/ 318 -http://onlain-film.com/ym.php 235 -https://mail.yandex.ru/neo2/ 220 -http://pluginplus.net/plugins/statistic/statistic_all.html 186 -http://m.kporno.com/ 163 -http://err.hc.ru/cgierr/26/ 160 -http://pluginplus.net/plugins/statistic/statistic_im.html 160 -http://znanija.com/ 141 -https://e.mail.ru/messages/inbox/ 135 -https://mail.yandex.ru/neo2/#inbox 594 -http://yandex.ru/ 318 -http://onlain-film.com/ym.php 235 -https://mail.yandex.ru/neo2/ 220 -http://pluginplus.net/plugins/statistic/statistic_all.html 186 -http://m.kporno.com/ 163 -http://err.hc.ru/cgierr/26/ 160 -http://pluginplus.net/plugins/statistic/statistic_im.html 160 -http://znanija.com/ 141 -https://e.mail.ru/messages/inbox/ 135 +http://korer.ru/categories.ru/?vkb 4803 +http://auto.ru/chatay-barana.ru/traction.html#maybettaya 585 +http://linka/?year 372 +http://public_search 316 +http://fontact Power[1 231 +http://auto.ru/chatayru.tsn.ua/artisemeyen 218 +http://msl.ua/kryma-/ 195 +http://yandex 193 +https://m.sprashivai.ru%2Fu768887.30 184 +http://korablitz.ru/L_1OFFERS_CRD 165 +http://korer.ru/categories.ru/?vkb 4803 +http://auto.ru/chatay-barana.ru/traction.html#maybettaya 585 +http://linka/?year 372 +http://public_search 316 +http://fontact Power[1 231 +http://auto.ru/chatayru.tsn.ua/artisemeyen 218 +http://msl.ua/kryma-/ 195 +http://yandex 193 +https://m.sprashivai.ru%2Fu768887.30 184 +http://korablitz.ru/L_1OFFERS_CRD 165 diff --git a/dbms/tests/queries/1_stateful/00087_where_0.sql b/dbms/tests/queries/1_stateful/00087_where_0.sql index c8fe05225fa..c55617d2245 100644 --- a/dbms/tests/queries/1_stateful/00087_where_0.sql +++ b/dbms/tests/queries/1_stateful/00087_where_0.sql @@ -1,3 +1,3 @@ SET max_rows_to_read = 1000; SELECT CounterID, uniq(UserID) FROM test.hits WHERE 0 != 0 GROUP BY CounterID; -SELECT CounterID, uniq(UserID) FROM test.hits WHERE 0 AND CounterID = 34 GROUP BY CounterID; +SELECT CounterID, uniq(UserID) FROM test.hits WHERE 0 AND CounterID = 1704509 GROUP BY CounterID; diff --git a/dbms/tests/queries/1_stateful/00089_position_functions_with_non_constant_arg.reference b/dbms/tests/queries/1_stateful/00089_position_functions_with_non_constant_arg.reference index f0db728c5e7..ad9a93d1113 100644 --- a/dbms/tests/queries/1_stateful/00089_position_functions_with_non_constant_arg.reference +++ b/dbms/tests/queries/1_stateful/00089_position_functions_with_non_constant_arg.reference @@ -2,8 +2,8 @@ 0 0 0 -samsungapps://productdetail/com.umojo.irr.android -ts3server://31.135.72.55?nickname=kridQl -http://ru/rus/index.php ru 15 -http://ex.ru/ ex.ru 12 -http://dex.ru/ dex.ru 11 +http://игры на передачи пригорька россия&lr=213&rpt=simage&uinfo=ww-1905-wh-643-fw-112-rossiisoft.in.ua%2FKievav@yandex?appkey=506d9e3dfbd268e6b6630e58 +http://игры на передачи пригорька россия&lr=213&rpt=simage&uinfo=ww-1905-wh-643-fw-112-rossiisoft.in.ua%2FKievav@yandex?appkey=506d9e3dfbd268e6b6630e58 +http://ru slovari 15 +https://ru spb.rabota 15 +https://e yandex 12 diff --git a/dbms/tests/queries/1_stateful/00089_position_functions_with_non_constant_arg.sql b/dbms/tests/queries/1_stateful/00089_position_functions_with_non_constant_arg.sql index ecfd500ae6a..4be84fe9e91 100644 --- a/dbms/tests/queries/1_stateful/00089_position_functions_with_non_constant_arg.sql +++ b/dbms/tests/queries/1_stateful/00089_position_functions_with_non_constant_arg.sql @@ -3,6 +3,6 @@ SELECT count() FROM test.hits WHERE positionCaseInsensitive(URL, 'metrika') != p SELECT count() FROM test.hits WHERE positionUTF8(Title, 'новости') != positionUTF8(Title, materialize('новости')); SELECT count() FROM test.hits WHERE positionCaseInsensitiveUTF8(Title, 'новости') != positionCaseInsensitiveUTF8(Title, materialize('новости')); -SELECT position(URL, URLDomain) AS x FROM test.hits WHERE x = 0 AND URL NOT LIKE '%yandex.ru%' LIMIT 100; -SELECT URL FROM test.hits WHERE x > 10 ORDER BY position(URL, URLDomain) AS x DESC, URL LIMIT 2; -SELECT DISTINCT URL, URLDomain, position('http://yandex.ru/', URLDomain) AS x FROM test.hits WHERE x > 8 ORDER BY position('http://yandex.ru/', URLDomain) DESC, URL LIMIT 3; +SELECT position(URL, domain(URL)) AS x FROM test.hits WHERE x = 0 AND URL NOT LIKE '%yandex.ru%' LIMIT 100; +SELECT URL FROM test.hits WHERE x > 10 ORDER BY position(URL, domain(URL)) AS x DESC, URL LIMIT 2; +SELECT DISTINCT URL, URLDomain, position('http://yandex.ru/', domain(URL)) AS x FROM test.hits WHERE x > 8 ORDER BY position('http://yandex.ru/', domain(URL)) DESC, URL LIMIT 3; diff --git a/dbms/tests/queries/1_stateful/00139_like.reference b/dbms/tests/queries/1_stateful/00139_like.reference index 2cfe3231ff6..e6932417a17 100644 --- a/dbms/tests/queries/1_stateful/00139_like.reference +++ b/dbms/tests/queries/1_stateful/00139_like.reference @@ -1,4 +1,4 @@ -79628 +7521 0 -79628 -102851 +7521 +51354 diff --git a/dbms/tests/queries/1_stateful/00140_rename.reference b/dbms/tests/queries/1_stateful/00140_rename.reference index 80776d85438..905cb08cf87 100644 --- a/dbms/tests/queries/1_stateful/00140_rename.reference +++ b/dbms/tests/queries/1_stateful/00140_rename.reference @@ -1,12 +1,12 @@ -286 -1962 -286 -1962 -1962 -286 -1962 -1962 -286 -1962 -1962 -286 +10545 +475698 +10545 +475698 +475698 +10545 +475698 +475698 +10545 +475698 +475698 +10545 diff --git a/dbms/tests/queries/1_stateful/00140_rename.sql b/dbms/tests/queries/1_stateful/00140_rename.sql index 33b7b1ad720..2b24ef084b2 100644 --- a/dbms/tests/queries/1_stateful/00140_rename.sql +++ b/dbms/tests/queries/1_stateful/00140_rename.sql @@ -1,32 +1,32 @@ RENAME TABLE test.hits TO test.visits_tmp, test.visits TO test.hits, test.visits_tmp TO test.visits; -SELECT sum(Sign) FROM test.hits WHERE CounterID = 34; -SELECT count() FROM test.visits WHERE CounterID = 34; +SELECT sum(Sign) FROM test.hits WHERE CounterID = 912887; +SELECT count() FROM test.visits WHERE CounterID = 732797; RENAME TABLE test.hits TO test.hits_tmp, test.hits_tmp TO test.hits; -SELECT sum(Sign) FROM test.hits WHERE CounterID = 34; -SELECT count() FROM test.visits WHERE CounterID = 34; +SELECT sum(Sign) FROM test.hits WHERE CounterID = 912887; +SELECT count() FROM test.visits WHERE CounterID = 732797; RENAME TABLE test.hits TO test.visits_tmp, test.visits TO test.hits, test.visits_tmp TO test.visits; -SELECT count() FROM test.hits WHERE CounterID = 34; -SELECT sum(Sign) FROM test.visits WHERE CounterID = 34; +SELECT count() FROM test.hits WHERE CounterID = 732797; +SELECT sum(Sign) FROM test.visits WHERE CounterID = 912887; RENAME TABLE test.hits TO test.hits2, test.hits2 TO test.hits3, test.hits3 TO test.hits4, test.hits4 TO test.hits5, test.hits5 TO test.hits6, test.hits6 TO test.hits7, test.hits7 TO test.hits8, test.hits8 TO test.hits9, test.hits9 TO test.hits10; -SELECT count() FROM test.hits10 WHERE CounterID = 34; +SELECT count() FROM test.hits10 WHERE CounterID = 732797; RENAME TABLE test.hits10 TO test.hits; -SELECT count() FROM test.hits WHERE CounterID = 34; +SELECT count() FROM test.hits WHERE CounterID = 732797; RENAME TABLE test.hits TO hits, test.visits TO test.hits; -SELECT sum(Sign) FROM test.hits WHERE CounterID = 34; -SELECT count() FROM hits WHERE CounterID = 34; +SELECT sum(Sign) FROM test.hits WHERE CounterID = 912887; +SELECT count() FROM hits WHERE CounterID = 732797; RENAME TABLE test.hits TO test.visits, hits TO test.hits; -SELECT count() FROM test.hits WHERE CounterID = 34; -SELECT sum(Sign) FROM test.visits WHERE CounterID = 34; +SELECT count() FROM test.hits WHERE CounterID = 732797; +SELECT sum(Sign) FROM test.visits WHERE CounterID = 912887; diff --git a/dbms/tests/queries/1_stateful/00141_transform.reference b/dbms/tests/queries/1_stateful/00141_transform.reference index 9cd12dac687..a7ef1540cc9 100644 --- a/dbms/tests/queries/1_stateful/00141_transform.reference +++ b/dbms/tests/queries/1_stateful/00141_transform.reference @@ -1,3 +1,3 @@ -Яндекс 498635 -Google 229872 +Google 498635 +Яндекс 229872 Остальные 104472 diff --git a/dbms/tests/queries/1_stateful/00143_transform_non_const_default.reference b/dbms/tests/queries/1_stateful/00143_transform_non_const_default.reference index b9772df780b..1a8c45130f8 100644 --- a/dbms/tests/queries/1_stateful/00143_transform_non_const_default.reference +++ b/dbms/tests/queries/1_stateful/00143_transform_non_const_default.reference @@ -1,10 +1,5 @@ -Яндекс 498635 -Google 229872 -utf-8 73842 -windows-1251 28664 +Google 498635 +Яндекс 229872 +utf-8 103187 1284 -koi8-r 165 -windows-1252 148 -windows-1254 126 -iso-8859-1 102 -iso-8859-9 86 +koi8-u 1 diff --git a/dbms/tests/queries/1_stateful/00144_functions_of_aggregation_states.reference b/dbms/tests/queries/1_stateful/00144_functions_of_aggregation_states.reference index 72b11764a12..c3139530f66 100644 --- a/dbms/tests/queries/1_stateful/00144_functions_of_aggregation_states.reference +++ b/dbms/tests/queries/1_stateful/00144_functions_of_aggregation_states.reference @@ -1,7 +1,7 @@ 2014-03-17 36613 36613 2014-03-18 36531 54710 -2014-03-19 36940 69954 -2014-03-20 36462 83923 -2014-03-21 35447 96824 -2014-03-22 31555 108565 -2014-03-23 31200 119497 +2014-03-19 36940 70781 +2014-03-20 36462 84826 +2014-03-21 35447 97634 +2014-03-22 31555 109424 +2014-03-23 31200 120665 diff --git a/dbms/tests/queries/1_stateful/00145_aggregate_functions_statistics.reference b/dbms/tests/queries/1_stateful/00145_aggregate_functions_statistics.reference index 39980910bfc..3eff0a64fff 100644 --- a/dbms/tests/queries/1_stateful/00145_aggregate_functions_statistics.reference +++ b/dbms/tests/queries/1_stateful/00145_aggregate_functions_statistics.reference @@ -1,21 +1,21 @@ nan nan -159323.735527 +177781.778472 nan nan -399.153774 +421.641766 nan 0 -159323.717573 +177781.758438 nan 0 -399.153752 +421.641742 nan nan -48783.944394 +91512.198095 nan 0 -48783.938897 +91512.187783 nan nan -0.451491 +0.405886 diff --git a/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.reference b/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.reference index 3e5f4b9e260..d454ce08e8a 100644 --- a/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.reference +++ b/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.reference @@ -1,19 +1,19 @@ -2 428064 -54 105908 -143 190223 -157 161971 -187 288270 -213 1324424 -225 318611 -11503 194698 -11508 199656 -2 428333 -54 106848 -143 187608 -157 164665 -187 284689 -213 1315011 -225 319575 -11503 197005 -11508 196642 -12247 +3 412471 +40 108428 +183 189673 +196 1364985 +207 293867 +225 163657 +241 318322 +8363 190718 +15887 193341 +3 428448 +40 107178 +183 187001 +196 1311036 +207 286638 +225 164458 +241 320439 +8363 196717 +15887 197579 +22113 diff --git a/dbms/tests/queries/1_stateful/00148_monotonic_functions_and_index.reference b/dbms/tests/queries/1_stateful/00148_monotonic_functions_and_index.reference index 3f8f6c523ed..2beebb465c5 100644 --- a/dbms/tests/queries/1_stateful/00148_monotonic_functions_and_index.reference +++ b/dbms/tests/queries/1_stateful/00148_monotonic_functions_and_index.reference @@ -1,33 +1,33 @@ -1962 -1962 -1962 -1962 -1962 -1962 -1999 -17669 -1999 -409 1 17669 -409 1 17669 -3 1 1999 -3 1 1999 -1 1962 -479124 -481541 -475681 -476843 -478098 -479260 -479124 -481541 -475681 -476843 -478098 -479260 -475681 -476843 -478098 -479260 +256 +256 +256 +256 +256 +256 +256 +17322 +256 +417 1 17322 +417 1 17322 +1 1 256 +1 1 256 +1 256 +1592312 +2068010 +1113002 +1115003 +1588700 +1590701 +1592312 +2068010 +1113002 +1115003 +1588700 +1590701 +1113002 +1115003 +1588700 +1590701 1353623 1353623 1353623 diff --git a/dbms/tests/queries/1_stateful/00148_monotonic_functions_and_index.sql b/dbms/tests/queries/1_stateful/00148_monotonic_functions_and_index.sql index 84faa6e059a..f93dc5e929f 100644 --- a/dbms/tests/queries/1_stateful/00148_monotonic_functions_and_index.sql +++ b/dbms/tests/queries/1_stateful/00148_monotonic_functions_and_index.sql @@ -1,47 +1,47 @@ -SET max_rows_to_read = 50000; +SET max_rows_to_read = 60000; -SELECT count() FROM test.hits WHERE -CounterID = -34; -SELECT count() FROM test.hits WHERE abs(-CounterID) = 34; -SELECT count() FROM test.hits WHERE -abs(CounterID) = -34; -SELECT count() FROM test.hits WHERE toUInt32(CounterID) = 34; -SELECT count() FROM test.hits WHERE toInt32(CounterID) = 34; -SELECT count() FROM test.hits WHERE toFloat32(CounterID) = 34; +SELECT count() FROM test.hits WHERE -CounterID = -1731; +SELECT count() FROM test.hits WHERE abs(-CounterID) = 1731; +SELECT count() FROM test.hits WHERE -abs(CounterID) = -1731; +SELECT count() FROM test.hits WHERE toUInt32(CounterID) = 1731; +SELECT count() FROM test.hits WHERE toInt32(CounterID) = 1731; +SELECT count() FROM test.hits WHERE toFloat32(CounterID) = 1731; SET max_rows_to_read = 0; -SELECT count() FROM test.hits WHERE toInt16(CounterID) = 34; -SELECT count() FROM test.hits WHERE toInt8(CounterID) = 34; +SELECT count() FROM test.hits WHERE toInt16(CounterID) = 1731; +SELECT count() FROM test.hits WHERE toInt8(CounterID) = toInt8(1731); -SELECT count() FROM test.hits WHERE toDate(toUInt16(CounterID)) = toDate(34); +SELECT count() FROM test.hits WHERE toDate(toUInt16(CounterID)) = toDate(1731); -SELECT uniq(CounterID), uniqUpTo(5)(toInt8(CounterID)), count() FROM test.hits WHERE toInt8(CounterID + 1 - 1) = 34; -SELECT uniq(CounterID), uniqUpTo(5)(toInt8(CounterID)), count() FROM test.hits WHERE toInt8(CounterID) = 34; +SELECT uniq(CounterID), uniqUpTo(5)(toInt8(CounterID)), count() FROM test.hits WHERE toInt8(CounterID + 1 - 1) = toInt8(1731); +SELECT uniq(CounterID), uniqUpTo(5)(toInt8(CounterID)), count() FROM test.hits WHERE toInt8(CounterID) = toInt8(1731); -SELECT uniq(CounterID), uniqUpTo(5)(toInt16(CounterID)), count() FROM test.hits WHERE toInt16(CounterID + 1 - 1) = 34; -SELECT uniq(CounterID), uniqUpTo(5)(toInt16(CounterID)), count() FROM test.hits WHERE toInt16(CounterID) = 34; +SELECT uniq(CounterID), uniqUpTo(5)(toInt16(CounterID)), count() FROM test.hits WHERE toInt16(CounterID + 1 - 1) = 1731; +SELECT uniq(CounterID), uniqUpTo(5)(toInt16(CounterID)), count() FROM test.hits WHERE toInt16(CounterID) = 1731; SET max_rows_to_read = 500000; -SELECT uniq(CounterID), count() FROM test.hits WHERE toString(CounterID) = '34'; +SELECT uniq(CounterID), count() FROM test.hits WHERE toString(CounterID) = '1731'; -SET max_rows_to_read = 2000000; +SET max_rows_to_read = 2200000; -SELECT count() FROM test.hits WHERE CounterID < 101500; -SELECT count() FROM test.hits WHERE CounterID <= 101500; -SELECT count() FROM test.hits WHERE CounterID < 101500 AND CounterID > 42; -SELECT count() FROM test.hits WHERE CounterID < 101500 AND CounterID >= 42; -SELECT count() FROM test.hits WHERE CounterID <= 101500 AND CounterID > 42; -SELECT count() FROM test.hits WHERE CounterID <= 101500 AND CounterID >= 42; -SELECT count() FROM test.hits WHERE -CounterID > -101500; -SELECT count() FROM test.hits WHERE -CounterID >= -101500; -SELECT count() FROM test.hits WHERE -CounterID > -101500 AND CounterID > 42; -SELECT count() FROM test.hits WHERE -CounterID > -101500 AND CounterID >= 42; -SELECT count() FROM test.hits WHERE -CounterID >= -101500 AND CounterID > 42; -SELECT count() FROM test.hits WHERE -CounterID >= -101500 AND CounterID >= 42; -SELECT count() FROM test.hits WHERE CounterID < 101500 AND -CounterID < -42; -SELECT count() FROM test.hits WHERE CounterID < 101500 AND -CounterID <= -42; -SELECT count() FROM test.hits WHERE CounterID <= 101500 AND -CounterID < -42; -SELECT count() FROM test.hits WHERE CounterID <= 101500 AND -CounterID <= -42; +SELECT count() FROM test.hits WHERE CounterID < 732797; +SELECT count() FROM test.hits WHERE CounterID <= 732797; +SELECT count() FROM test.hits WHERE CounterID < 732797 AND CounterID > 107931; +SELECT count() FROM test.hits WHERE CounterID < 732797 AND CounterID >= 107931; +SELECT count() FROM test.hits WHERE CounterID <= 732797 AND CounterID > 107931; +SELECT count() FROM test.hits WHERE CounterID <= 732797 AND CounterID >= 107931; +SELECT count() FROM test.hits WHERE -CounterID > -732797; +SELECT count() FROM test.hits WHERE -CounterID >= -732797; +SELECT count() FROM test.hits WHERE -CounterID > -732797 AND CounterID > 107931; +SELECT count() FROM test.hits WHERE -CounterID > -732797 AND CounterID >= 107931; +SELECT count() FROM test.hits WHERE -CounterID >= -732797 AND CounterID > 107931; +SELECT count() FROM test.hits WHERE -CounterID >= -732797 AND CounterID >= 107931; +SELECT count() FROM test.hits WHERE CounterID < 732797 AND -CounterID < -107931; +SELECT count() FROM test.hits WHERE CounterID < 732797 AND -CounterID <= -107931; +SELECT count() FROM test.hits WHERE CounterID <= 732797 AND -CounterID < -107931; +SELECT count() FROM test.hits WHERE CounterID <= 732797 AND -CounterID <= -107931; SET max_rows_to_read = 0; diff --git a/dbms/tests/queries/1_stateful/00149_quantiles_timing_distributed.reference b/dbms/tests/queries/1_stateful/00149_quantiles_timing_distributed.reference index 3183819b008..8ac5f01c7cc 100644 --- a/dbms/tests/queries/1_stateful/00149_quantiles_timing_distributed.reference +++ b/dbms/tests/queries/1_stateful/00149_quantiles_timing_distributed.reference @@ -1 +1 @@ -4306270979949069156 +10726001768429413598 diff --git a/dbms/tests/queries/1_stateful/00150_quantiles_timing_precision.reference b/dbms/tests/queries/1_stateful/00150_quantiles_timing_precision.reference index 5c047df92c4..09aaf8449dc 100644 --- a/dbms/tests/queries/1_stateful/00150_quantiles_timing_precision.reference +++ b/dbms/tests/queries/1_stateful/00150_quantiles_timing_precision.reference @@ -1 +1 @@ -7234936 1824 1829 5755 0.003 +4379238 1868 1879 5755 0.006 From 76c4b86e0ad60598b4cc6aacddad73716374fa1b Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Tue, 15 Jan 2019 15:57:06 +0300 Subject: [PATCH 04/71] Add sample instruction on how to test Kafka with ClickHouse locally --- dbms/tests/instructions/kafka.txt | 45 +++++++++++++++++++++++++++++++ 1 file changed, 45 insertions(+) create mode 100644 dbms/tests/instructions/kafka.txt diff --git a/dbms/tests/instructions/kafka.txt b/dbms/tests/instructions/kafka.txt new file mode 100644 index 00000000000..69e87f38b24 --- /dev/null +++ b/dbms/tests/instructions/kafka.txt @@ -0,0 +1,45 @@ +Use this config for docker-compose: + + version: '3' + + services: + + kafka: + depends_on: + - zookeeper + hostname: kafka + image: wurstmeister/kafka + environment: + KAFKA_LISTENERS: INSIDE://:9092,OUTSIDE://:9094 + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT + KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE + KAFKA_ZOOKEEPER_CONNECT: zookeeper:2181 + ports: + - "9092:9092" + - "9094:9094" + + security_opt: + - label:disable + + zookeeper: + hostname: zookeeper + image: zookeeper + + security_opt: + - label:disable + +Start containers with `docker-compose up`. + +In clickhouse-client create table like: + + CREATE TABLE kafka ( a UInt8, b String) ENGINE = Kafka('localhost:9092', 'topic', 'group1', 'CSV') SETTINGS kafka_row_delimiter = '\n'; + +Login inside Kafka container and stream some data: + + docker exec -it bash --login + vi data.csv + cat data.csv | /opt/kafka/bin/kafka-console-producer.sh --topic topic --broker-list localhost:9092 + +Read data in clickhouse: + + SELECT * FROM kafka; From 87229baac08b73f8f0b617fff67bcd3604b6c4b7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 16 Jan 2019 15:07:26 +0300 Subject: [PATCH 05/71] commit just to trigger ci --- dbms/tests/queries/1_stateful/00007_uniq.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/1_stateful/00007_uniq.sql b/dbms/tests/queries/1_stateful/00007_uniq.sql index bc36a2b69e6..6cba046c526 100644 --- a/dbms/tests/queries/1_stateful/00007_uniq.sql +++ b/dbms/tests/queries/1_stateful/00007_uniq.sql @@ -1 +1 @@ -SELECT RegionID, uniq(UserID) AS u FROM test.hits WHERE CounterID = 800784 GROUP BY RegionID ORDER BY u DESC, RegionID LIMIT 10 +SELECT RegionID, uniq(UserID) AS u FROM test.hits WHERE CounterID = 800784 GROUP BY RegionID ORDER BY u DESC, RegionID LIMIT 10 -- nothing From 5dade4a588f97709a3b2560ead061d8044a539ba Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Wed, 16 Jan 2019 15:50:14 +0300 Subject: [PATCH 06/71] Add doc for per-column compression codecs --- .../table_engines/custom_compression_codec.md | 44 ++++++++++++++++++ docs/en/query_language/create.md | 9 +++- .../table_engines/custom_compression_codec.md | 45 +++++++++++++++++++ docs/ru/query_language/create.md | 9 +++- 4 files changed, 103 insertions(+), 4 deletions(-) create mode 100644 docs/en/operations/table_engines/custom_compression_codec.md create mode 100644 docs/ru/operations/table_engines/custom_compression_codec.md diff --git a/docs/en/operations/table_engines/custom_compression_codec.md b/docs/en/operations/table_engines/custom_compression_codec.md new file mode 100644 index 00000000000..d51b3bcf1d1 --- /dev/null +++ b/docs/en/operations/table_engines/custom_compression_codec.md @@ -0,0 +1,44 @@ + +# Column compression codecs + +Besides default data compression, defined in [server settings](../server_settings/settings.md#compression), per-column specification is also available. + +Supported compression algorithms: + +- `NONE` - no compression for data applied +- `LZ4` +- `LZ4HC(level)` - (level) - LZ4 compression algorithm with defined level. +Possible `level` range: [1,12]. Greater values stands for better compression and higher CPU usage. Recommended value range: [4,9]. +- `ZSTD(level)` - ZSTD compression algorithm with defined `level`. Possible `level` value range: [1,22]. +Greater values stands for better compression and higher CPU usage. +- `Delta(delta_bytes)` - compression approach when raw values are replace with difference with two neighbour values. Up to `delta_bytes` are used for storing delta value. +Possible `delta_bytes` values: 1, 2, 4, 8; + +Syntax example: +``` +CREATE TABLE codec_example +( + dt Date CODEC(ZSTD), /* используется уровень сжатия по-умолчанию */ + ts DateTime CODEC(LZ4HC), + float_value Float32 CODEC(NONE), + double_value Float64 CODEC(LZ4HC(2)) +) +ENGINE = MergeTree +PARTITION BY tuple() +ORDER BY dt +``` + +Codecs can be combined in a pipeline. Example below shows an optimization approach for storing timeseries metrics. +Usually, values for particular metric, stored in `path` does not differ significantly from point to point. Using delta-encoding allows to reduce disk space usage significantly. +``` +CREATE TABLE timeseries_example +( + dt Date, + ts DateTime, + path String, + value Float32 CODEC(Delta(2), ZSTD) +) +ENGINE = MergeTree +PARTITION BY dt +ORDER BY (path, ts) +``` diff --git a/docs/en/query_language/create.md b/docs/en/query_language/create.md index 9cf783154f9..387e244ce0e 100644 --- a/docs/en/query_language/create.md +++ b/docs/en/query_language/create.md @@ -17,8 +17,8 @@ The `CREATE TABLE` query can have several forms. ```sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [compression_codec], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [compression_codec], ... ) ENGINE = engine ``` @@ -80,6 +80,11 @@ If you add a new column to a table but later change its default expression, the It is not possible to set default values for elements in nested data structures. +### Column compression codecs + +Table columns can use either common compression codec, defined in server settings, or use individual one, defined in `compression_codec`. +[Detailed description](../operations/table_engines/custom_compression_codec.md). + ### Temporary Tables ClickHouse supports temporary tables which have the following characteristics: diff --git a/docs/ru/operations/table_engines/custom_compression_codec.md b/docs/ru/operations/table_engines/custom_compression_codec.md new file mode 100644 index 00000000000..8394b3200e2 --- /dev/null +++ b/docs/ru/operations/table_engines/custom_compression_codec.md @@ -0,0 +1,45 @@ + +# Форматы сжатия для колонок + +Помимо сжатия для колонок по-умолчанию, определяемого в [настройках сервера](../server_settings/settings.md#compression), +существует возможность указать формат сжатия индивидуально для каждой колонки. + +Поддерживаемые форматы: + +- `NONE` - сжатие отсутствует +- `LZ4` +- `LZ4HC(level)` - (level) - алгоритм сжатия LZ4 с указанным уровнем компрессии `level`. +Возможный диапазон значений `level`: [1,12]. Чем выше уровень, тем лучше сжатие, но тратится больше времени. Рекомендованный диапазон [4,9]. +- `ZSTD(level)` - алгоритм сжатия ZSTD с указанным уровнем компрессии `level`. Возможный диапазон значений `level`: [1,22]. +Чем выше уровень, тем лучше сжатие, но тратится больше времени. Значение по-умолчанию: +- `Delta(delta_bytes)` - способ сжатия, при котором вместо числовых значений поля сохраняется разность между двумя соседними значениями. Значение `delta_bytes` - число байт для хранения дельты. +Возможные значения: 1, 2, 4, 8; + +Пример использования: +``` +CREATE TABLE codec_example +( + dt Date CODEC(ZSTD), /* используется уровень сжатия по-умолчанию */ + ts DateTime CODEC(LZ4HC), + float_value Float32 CODEC(NONE), + double_value Float64 CODEC(LZ4HC(2)) +) +ENGINE = MergeTree +PARTITION BY tuple() +ORDER BY dt +``` + +Кодеки могут комбинироваться между собой. В примере ниже - оптимизация для хранения timeseries метрик. +Как правило, значения одной и той же метрики `path` не сильно различаются между собой, и выгоднее использовать дельта-компрессию вместо записи всего числа: +``` +CREATE TABLE timeseries_example +( + dt Date, + ts DateTime, + path String, + value Float32 CODEC(Delta(2), ZSTD) +) +ENGINE = MergeTree +PARTITION BY dt +ORDER BY (path, ts) +``` diff --git a/docs/ru/query_language/create.md b/docs/ru/query_language/create.md index 9f4cd95863c..09b0765c8f3 100644 --- a/docs/ru/query_language/create.md +++ b/docs/ru/query_language/create.md @@ -16,8 +16,8 @@ CREATE DATABASE [IF NOT EXISTS] db_name ``` sql CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] ( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [compression_codec], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [compression_codec], ... ) ENGINE = engine ``` @@ -81,6 +81,11 @@ CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name ENGINE = engine AS SELECT ... Отсутствует возможность задать значения по умолчанию для элементов вложенных структур данных. +### Форматы сжатия для колонок + +Таблица может использовать общий формат сжатия, установленный в настройках сервера, или применять к каждой колонке свой формат, указанный в `compression_codec`. +[Подробное описание](../operations/table_engines/custom_compression_codec.md). + ### Временные таблицы Во всех случаях, если указано `TEMPORARY`, то будет создана временная таблица. Временные таблицы обладают следующими особенностями: From 2d3e08fc748456ee987d9eddf09d036b8f524a98 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 17 Jan 2019 14:26:29 +0300 Subject: [PATCH 07/71] Refactor ReadBufferFromHDFS --- dbms/CMakeLists.txt | 4 +- dbms/src/IO/ReadBufferFromHDFS.cpp | 104 +++++++++++++++++++++++++++++ dbms/src/IO/ReadBufferFromHDFS.h | 96 +++++--------------------- 3 files changed, 121 insertions(+), 83 deletions(-) create mode 100644 dbms/src/IO/ReadBufferFromHDFS.cpp diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 84099810164..4463d571386 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -299,8 +299,8 @@ target_include_directories (dbms SYSTEM BEFORE PRIVATE ${DIVIDE_INCLUDE_DIR}) target_include_directories (dbms SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) if (USE_HDFS) - target_link_libraries (dbms PRIVATE ${HDFS3_LIBRARY}) - target_include_directories (dbms SYSTEM BEFORE PRIVATE ${HDFS3_INCLUDE_DIR}) + target_link_libraries (clickhouse_common_io PRIVATE ${HDFS3_LIBRARY}) + target_include_directories (clickhouse_common_io SYSTEM BEFORE PRIVATE ${HDFS3_INCLUDE_DIR}) endif() if (USE_JEMALLOC) diff --git a/dbms/src/IO/ReadBufferFromHDFS.cpp b/dbms/src/IO/ReadBufferFromHDFS.cpp new file mode 100644 index 00000000000..c4b9a2e5d4f --- /dev/null +++ b/dbms/src/IO/ReadBufferFromHDFS.cpp @@ -0,0 +1,104 @@ +#include + +#if USE_HDFS +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int NETWORK_ERROR; +} + +struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl +{ + std::string hdfs_uri; + struct hdfsBuilder * builder; + hdfsFS fs; + hdfsFile fin; + + ReadBufferFromHDFSImpl(const std::string & hdfs_name_) + : hdfs_uri(hdfs_name_) + , builder(hdfsNewBuilder()) + { + builder = hdfsNewBuilder(); + hdfs_uri = hdfs_name_; + Poco::URI uri(hdfs_name_); + auto & host = uri.getHost(); + auto port = uri.getPort(); + auto & path = uri.getPath(); + if (host.empty() || port == 0 || path.empty()) + { + throw Exception("Illegal HDFS URI: " + hdfs_uri, ErrorCodes::BAD_ARGUMENTS); + } + // set read/connect timeout, default value in libhdfs3 is about 1 hour, and too large + /// TODO Allow to tune from query Settings. + hdfsBuilderConfSetStr(builder, "input.read.timeout", "60000"); // 1 min + hdfsBuilderConfSetStr(builder, "input.connect.timeout", "60000"); // 1 min + + hdfsBuilderSetNameNode(builder, host.c_str()); + hdfsBuilderSetNameNodePort(builder, port); + fs = hdfsBuilderConnect(builder); + + if (fs == nullptr) + { + throw Exception("Unable to connect to HDFS: " + std::string(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR); + } + + fin = hdfsOpenFile(fs, path.c_str(), O_RDONLY, 0, 0, 0); + } + + ~ReadBufferFromHDFSImpl() + { + close(); + hdfsFreeBuilder(builder); + } + + void close() + { + hdfsCloseFile(fs, fin); + } + + int read(char * start, size_t size) + { + int bytes_read = hdfsRead(fs, fin, start, size); + if (bytes_read < 0) + { + throw Exception("Fail to read HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR); + } + return bytes_read; + } +}; + +ReadBufferFromHDFS::ReadBufferFromHDFS(const std::string & hdfs_name_, size_t buf_size) + : BufferWithOwnMemory(buf_size) + , impl(std::make_unique(hdfs_name_)) +{ +} + + +bool ReadBufferFromHDFS::nextImpl() +{ + int bytes_read = impl->read(internal_buffer.begin(), internal_buffer.size()); + + if (bytes_read) + working_buffer.resize(bytes_read); + else + return false; + return true; +} + +const std::string & ReadBufferFromHDFS::getHDFSUri() const +{ + return impl->hdfs_uri; +} + +ReadBufferFromHDFS::~ReadBufferFromHDFS() +{ +} + +} + +#endif diff --git a/dbms/src/IO/ReadBufferFromHDFS.h b/dbms/src/IO/ReadBufferFromHDFS.h index a97ad5ece2f..7e0656098de 100644 --- a/dbms/src/IO/ReadBufferFromHDFS.h +++ b/dbms/src/IO/ReadBufferFromHDFS.h @@ -4,94 +4,28 @@ #if USE_HDFS #include -#include -#include #include #include - -#ifndef O_DIRECT -#define O_DIRECT 00040000 -#endif +#include namespace DB { - namespace ErrorCodes - { - extern const int BAD_ARGUMENTS; - extern const int NETWORK_ERROR; - } - /** Accepts path to file and opens it, or pre-opened file descriptor. - * Closes file by himself (thus "owns" a file descriptor). - */ - class ReadBufferFromHDFS : public BufferWithOwnMemory - { - protected: - std::string hdfs_uri; - struct hdfsBuilder *builder; - hdfsFS fs; - hdfsFile fin; - public: - ReadBufferFromHDFS(const std::string & hdfs_name_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) - : BufferWithOwnMemory(buf_size), hdfs_uri(hdfs_name_) , builder(hdfsNewBuilder()) - { - Poco::URI uri(hdfs_name_); - auto & host = uri.getHost(); - auto port = uri.getPort(); - auto & path = uri.getPath(); - if (host.empty() || port == 0 || path.empty()) - { - throw Exception("Illegal HDFS URI: " + hdfs_uri, ErrorCodes::BAD_ARGUMENTS); - } - // set read/connect timeout, default value in libhdfs3 is about 1 hour, and too large - /// TODO Allow to tune from query Settings. - hdfsBuilderConfSetStr(builder, "input.read.timeout", "60000"); // 1 min - hdfsBuilderConfSetStr(builder, "input.connect.timeout", "60000"); // 1 min +/** Accepts path to file and opens it, or pre-opened file descriptor. + * Closes file by himself (thus "owns" a file descriptor). + */ +class ReadBufferFromHDFS : public BufferWithOwnMemory +{ + struct ReadBufferFromHDFSImpl; + std::unique_ptr impl; +public: + ReadBufferFromHDFS(const std::string & hdfs_name_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + ReadBufferFromHDFS(ReadBufferFromHDFS &&) = default; - hdfsBuilderSetNameNode(builder, host.c_str()); - hdfsBuilderSetNameNodePort(builder, port); - fs = hdfsBuilderConnect(builder); + bool nextImpl() override; - if (fs == nullptr) - { - throw Exception("Unable to connect to HDFS: " + std::string(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR); - } + ~ReadBufferFromHDFS() override; - fin = hdfsOpenFile(fs, path.c_str(), O_RDONLY, 0, 0, 0); - } - - ReadBufferFromHDFS(ReadBufferFromHDFS &&) = default; - - ~ReadBufferFromHDFS() override - { - close(); - hdfsFreeBuilder(builder); - } - - /// Close HDFS connection before destruction of object. - void close() - { - hdfsCloseFile(fs, fin); - } - - bool nextImpl() override - { - int bytes_read = hdfsRead(fs, fin, internal_buffer.begin(), internal_buffer.size()); - if (bytes_read < 0) - { - throw Exception("Fail to read HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR); - } - - if (bytes_read) - working_buffer.resize(bytes_read); - else - return false; - return true; - } - - const std::string & getHDFSUri() const - { - return hdfs_uri; - } - }; + const std::string & getHDFSUri() const; +}; } #endif From 07e11577d399a93105f73566c0ac7c792d11a349 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 17 Jan 2019 17:10:30 +0300 Subject: [PATCH 08/71] Add write logic and remove strange multifile read logic --- dbms/src/IO/ReadBufferFromHDFS.cpp | 8 +- dbms/src/IO/WriteBufferFromHDFS.cpp | 119 ++++++++++++++ dbms/src/IO/WriteBufferFromHDFS.h | 33 ++++ dbms/src/Storages/StorageHDFS.cpp | 152 +++++++++--------- dbms/src/Storages/StorageHDFS.h | 1 + .../integration/test_storage_hdfs/test.py | 11 ++ 6 files changed, 247 insertions(+), 77 deletions(-) create mode 100644 dbms/src/IO/WriteBufferFromHDFS.cpp create mode 100644 dbms/src/IO/WriteBufferFromHDFS.h diff --git a/dbms/src/IO/ReadBufferFromHDFS.cpp b/dbms/src/IO/ReadBufferFromHDFS.cpp index c4b9a2e5d4f..10899f24fdc 100644 --- a/dbms/src/IO/ReadBufferFromHDFS.cpp +++ b/dbms/src/IO/ReadBufferFromHDFS.cpp @@ -15,7 +15,7 @@ namespace ErrorCodes struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl { std::string hdfs_uri; - struct hdfsBuilder * builder; + hdfsBuilder * builder; hdfsFS fs; hdfsFile fin; @@ -36,6 +36,7 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl // set read/connect timeout, default value in libhdfs3 is about 1 hour, and too large /// TODO Allow to tune from query Settings. hdfsBuilderConfSetStr(builder, "input.read.timeout", "60000"); // 1 min + hdfsBuilderConfSetStr(builder, "input.write.timeout", "60000"); // 1 min hdfsBuilderConfSetStr(builder, "input.connect.timeout", "60000"); // 1 min hdfsBuilderSetNameNode(builder, host.c_str()); @@ -65,9 +66,8 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl { int bytes_read = hdfsRead(fs, fin, start, size); if (bytes_read < 0) - { - throw Exception("Fail to read HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR); - } + throw Exception("Fail to read HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), + ErrorCodes::NETWORK_ERROR); return bytes_read; } }; diff --git a/dbms/src/IO/WriteBufferFromHDFS.cpp b/dbms/src/IO/WriteBufferFromHDFS.cpp new file mode 100644 index 00000000000..4e1b916dafa --- /dev/null +++ b/dbms/src/IO/WriteBufferFromHDFS.cpp @@ -0,0 +1,119 @@ +#include + +#if USE_HDFS +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +extern const int NETWORK_ERROR; +extern const int CANNOT_FSYNC; +} + + +struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl +{ + std::string hdfs_uri; + hdfsBuilder * builder; + hdfsFS fs; + hdfsFile fout; + + WriteBufferFromHDFSImpl(const std::string & hdfs_name_) + { + builder = hdfsNewBuilder(); + hdfs_uri = hdfs_name_; + Poco::URI uri(hdfs_name_); + auto & host = uri.getHost(); + auto port = uri.getPort(); + auto & path = uri.getPath(); + if (host.empty() || port == 0 || path.empty()) + { + throw Exception("Illegal HDFS URI: " + hdfs_uri, ErrorCodes::BAD_ARGUMENTS); + } + // set read/connect timeout, default value in libhdfs3 is about 1 hour, and too large + /// TODO Allow to tune from query Settings. + hdfsBuilderConfSetStr(builder, "input.read.timeout", "60000"); // 1 min + hdfsBuilderConfSetStr(builder, "input.write.timeout", "60000"); // 1 min + hdfsBuilderConfSetStr(builder, "input.connect.timeout", "60000"); // 1 min + + hdfsBuilderSetNameNode(builder, host.c_str()); + hdfsBuilderSetNameNodePort(builder, port); + fs = hdfsBuilderConnect(builder); + + if (fs == nullptr) + { + throw Exception("Unable to connect to HDFS: " + std::string(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR); + } + + fout = hdfsOpenFile(fs, path.c_str(), O_WRONLY, 0, 0, 0); + } + + ~WriteBufferFromHDFSImpl() + { + close(); + hdfsFreeBuilder(builder); + } + + void close() + { + hdfsCloseFile(fs, fout); + } + + + int write(const char * start, size_t size) + { + int bytes_written = hdfsWrite(fs, fout, start, size); + if (bytes_written < 0) + throw Exception("Fail to write HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), + ErrorCodes::NETWORK_ERROR); + return bytes_written; + } + + void sync() + { + int result = hdfsSync(fs, fout); + if (result < 0) + throwFromErrno("Cannot HDFS sync" + hdfs_uri + " " + std::string(hdfsGetLastError()), + ErrorCodes::CANNOT_FSYNC); + } +}; + +WriteBufferFromHDFS::WriteBufferFromHDFS(const std::string & hdfs_name_, size_t buf_size) + : BufferWithOwnMemory(buf_size) + , impl(std::make_unique(hdfs_name_)) +{ +} + + +void WriteBufferFromHDFS::nextImpl() +{ + if (!offset()) + return; + + size_t bytes_written = 0; + + while (bytes_written != offset()) + bytes_written += impl->write(working_buffer.begin() + bytes_written, offset() - bytes_written); +} + + +void WriteBufferFromHDFS::sync() +{ + impl->sync(); +} + +const std::string & WriteBufferFromHDFS::getHDFSUri() const +{ + return impl->hdfs_uri; +} + +WriteBufferFromHDFS::~WriteBufferFromHDFS() +{ +} + +} +#endif diff --git a/dbms/src/IO/WriteBufferFromHDFS.h b/dbms/src/IO/WriteBufferFromHDFS.h new file mode 100644 index 00000000000..bd89405a47c --- /dev/null +++ b/dbms/src/IO/WriteBufferFromHDFS.h @@ -0,0 +1,33 @@ +#pragma once + +#include + +#if USE_HDFS +#include +#include +#include +#include + +namespace DB +{ +/** Accepts path to file and opens it, or pre-opened file descriptor. + * Closes file by himself (thus "owns" a file descriptor). + */ +class WriteBufferFromHDFS : public BufferWithOwnMemory +{ + struct WriteBufferFromHDFSImpl; + std::unique_ptr impl; +public: + WriteBufferFromHDFS(const std::string & hdfs_name_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); + WriteBufferFromHDFS(WriteBufferFromHDFS &&) = default; + + void nextImpl() override; + + ~WriteBufferFromHDFS() override; + + const std::string & getHDFSUri() const; + + void sync(); +}; +} +#endif diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index 6dd5cf4c92e..95b58910052 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -8,13 +8,12 @@ #include #include #include +#include #include #include #include #include #include -#include -#include namespace DB @@ -30,94 +29,101 @@ StorageHDFS::StorageHDFS(const String & uri_, const std::string & table_name_, const String & format_name_, const ColumnsDescription & columns_, - Context &) - : IStorage(columns_), uri(uri_), format_name(format_name_), table_name(table_name_) + Context & context_) + : IStorage(columns_) + , uri(uri_) + , format_name(format_name_) + , table_name(table_name_) + , context(context_) { } namespace { - class HDFSBlockInputStream : public IProfilingBlockInputStream + +class HDFSBlockInputStream : public IProfilingBlockInputStream +{ +public: + HDFSBlockInputStream(const String & uri, + const String & format, + const Block & sample_block, + const Context & context, + size_t max_block_size) { - public: - HDFSBlockInputStream(const String & uri, - const String & format, - const Block & sample_block, - const Context & context, - size_t max_block_size) - { - // Assume no query and fragment in uri, todo, add sanity check - String glob_file_names; - String url_prefix = uri.substr(0, uri.find_last_of('/')); - if (url_prefix.length() == uri.length()) - { - glob_file_names = uri; - url_prefix.clear(); - } - else - { - url_prefix += "/"; - glob_file_names = uri.substr(url_prefix.length()); - } + std::unique_ptr read_buf = std::make_unique(uri); + auto input_stream = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size); + reader = std::make_shared>(input_stream, std::move(read_buf)); + } - std::vector glob_names_list = parseRemoteDescription(glob_file_names, 0, glob_file_names.length(), ',' , 100/* hard coded max files */); + String getName() const override + { + return "HDFS"; + } - BlockInputStreams inputs; + Block readImpl() override + { + return reader->read(); + } - for (const auto & name : glob_names_list) - { - std::unique_ptr read_buf = std::make_unique(url_prefix + name); + Block getHeader() const override + { + return reader->getHeader(); + } - inputs.emplace_back( - std::make_shared>( - FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size), - std::move(read_buf))); - } + void readPrefixImpl() override + { + reader->readPrefix(); + } - if (inputs.size() == 0) - throw Exception("StorageHDFS inputs interpreter error", ErrorCodes::BAD_ARGUMENTS); + void readSuffixImpl() override + { + reader->readSuffix(); + } - if (inputs.size() == 1) - { - reader = inputs[0]; - } - else - { - reader = std::make_shared(inputs, nullptr, context.getSettingsRef().max_distributed_connections); - } - } +private: + BlockInputStreamPtr reader; +}; - String getName() const override - { - return "HDFS"; - } +class HDFSBlockOutputStream : public IBlockOutputStream +{ +public: + HDFSBlockOutputStream(const String & uri, + const String & format, + const Block & sample_block_, + const Context & context) + : sample_block(sample_block_) + { + write_buf = std::make_unique(uri); + writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context); + } - Block readImpl() override - { - return reader->read(); - } + Block getHeader() const override + { + return sample_block; + } - Block getHeader() const override - { - return reader->getHeader(); - } + void write(const Block & block) override + { + writer->write(block); + } - void readPrefixImpl() override - { - reader->readPrefix(); - } + void writePrefix() override + { + writer->writePrefix(); + } - void readSuffixImpl() override - { - if (auto concrete_reader = dynamic_cast(reader.get())) - concrete_reader->cancel(false); // skip Union read suffix assertion + void writeSuffix() override + { + writer->writeSuffix(); + writer->flush(); + write_buf->sync(); + } - reader->readSuffix(); - } - - private: - BlockInputStreamPtr reader; - }; +private: + Block sample_block; + std::unique_ptr write_buf; + BlockOutputStreamPtr writer; +}; } @@ -142,7 +148,7 @@ void StorageHDFS::rename(const String & /*new_path_to_db*/, const String & /*new BlockOutputStreamPtr StorageHDFS::write(const ASTPtr & /*query*/, const Settings & /*settings*/) { - throw Exception("StorageHDFS write is not supported yet", ErrorCodes::NOT_IMPLEMENTED); + return std::make_shared(uri, format_name, getSampleBlock(), context); } void registerStorageHDFS(StorageFactory & factory) diff --git a/dbms/src/Storages/StorageHDFS.h b/dbms/src/Storages/StorageHDFS.h index 44ff23c4d67..6d5141eddbc 100644 --- a/dbms/src/Storages/StorageHDFS.h +++ b/dbms/src/Storages/StorageHDFS.h @@ -48,6 +48,7 @@ private: String uri; String format_name; String table_name; + Context & context; Logger * log = &Logger::get("StorageHDFS"); }; diff --git a/dbms/tests/integration/test_storage_hdfs/test.py b/dbms/tests/integration/test_storage_hdfs/test.py index 49cf7c46c4d..0779e9c2ec2 100644 --- a/dbms/tests/integration/test_storage_hdfs/test.py +++ b/dbms/tests/integration/test_storage_hdfs/test.py @@ -45,3 +45,14 @@ def test_read_write_table(started_cluster): assert hdfs_api.read_data("/simple_table_function") == data assert node1.query("select * from hdfs('hdfs://hdfs1:9000/simple_table_function', 'TSV', 'id UInt64, text String, number Float64')") == data + + +def test_write_table(started_cluster): + hdfs_api = HDFSApi("root") + + node1.query("create table OtherHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/other_storage', 'TSV')") + node1.query("insert into OtherHDFSStorage values (10, 'tomas', 55.55), (11, 'jack', 32.54)") + + result = "10\ttomas\t55.55\n11\tjack\t32.54\n" + assert hdfs_api.read_data("/other_storage") == result + assert node1.query("select * from OtherHDFSStorage order by id") == result From 6b12ada18f1acdf6280fc3367439c0b38bde45d0 Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Thu, 17 Jan 2019 18:57:25 +0300 Subject: [PATCH 09/71] Review fixes --- .../table_engines/custom_compression_codec.md | 16 ++++++++-------- .../table_engines/custom_compression_codec.md | 14 +++++++------- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/docs/en/operations/table_engines/custom_compression_codec.md b/docs/en/operations/table_engines/custom_compression_codec.md index d51b3bcf1d1..18e49c661f3 100644 --- a/docs/en/operations/table_engines/custom_compression_codec.md +++ b/docs/en/operations/table_engines/custom_compression_codec.md @@ -1,5 +1,5 @@ -# Column compression codecs +# Column Compression Codecs Besides default data compression, defined in [server settings](../server_settings/settings.md#compression), per-column specification is also available. @@ -7,12 +7,12 @@ Supported compression algorithms: - `NONE` - no compression for data applied - `LZ4` -- `LZ4HC(level)` - (level) - LZ4 compression algorithm with defined level. -Possible `level` range: [1,12]. Greater values stands for better compression and higher CPU usage. Recommended value range: [4,9]. -- `ZSTD(level)` - ZSTD compression algorithm with defined `level`. Possible `level` value range: [1,22]. +- `LZ4HC(level)` - (level) - LZ4_HC compression algorithm with defined level. +Possible `level` range: \[3, 12\]. Default value: 9. Greater values stands for better compression and higher CPU usage. Recommended value range: [4,9]. +- `ZSTD(level)` - ZSTD compression algorithm with defined `level`. Possible `level` value range: \[1, 22\]. Default value: 1. Greater values stands for better compression and higher CPU usage. -- `Delta(delta_bytes)` - compression approach when raw values are replace with difference with two neighbour values. Up to `delta_bytes` are used for storing delta value. -Possible `delta_bytes` values: 1, 2, 4, 8; +- `Delta(delta_bytes)` - compression approach when raw values are replace with difference of two neighbour values. Up to `delta_bytes` are used for storing delta value. +Possible `delta_bytes` values: 1, 2, 4, 8. Default value for delta bytes is `sizeof(type)`, if it is equals to 1, 2, 4, 8 and equals to 1 otherwise. Syntax example: ``` @@ -21,14 +21,14 @@ CREATE TABLE codec_example dt Date CODEC(ZSTD), /* используется уровень сжатия по-умолчанию */ ts DateTime CODEC(LZ4HC), float_value Float32 CODEC(NONE), - double_value Float64 CODEC(LZ4HC(2)) + double_value Float64 CODEC(LZ4HC(9)) ) ENGINE = MergeTree PARTITION BY tuple() ORDER BY dt ``` -Codecs can be combined in a pipeline. Example below shows an optimization approach for storing timeseries metrics. +Codecs can be combined in a pipeline. Default table codec is not included into pipeline. Example below shows an optimization approach for storing timeseries metrics. Usually, values for particular metric, stored in `path` does not differ significantly from point to point. Using delta-encoding allows to reduce disk space usage significantly. ``` CREATE TABLE timeseries_example diff --git a/docs/ru/operations/table_engines/custom_compression_codec.md b/docs/ru/operations/table_engines/custom_compression_codec.md index 8394b3200e2..cda705b2e3e 100644 --- a/docs/ru/operations/table_engines/custom_compression_codec.md +++ b/docs/ru/operations/table_engines/custom_compression_codec.md @@ -8,12 +8,12 @@ - `NONE` - сжатие отсутствует - `LZ4` -- `LZ4HC(level)` - (level) - алгоритм сжатия LZ4 с указанным уровнем компрессии `level`. -Возможный диапазон значений `level`: [1,12]. Чем выше уровень, тем лучше сжатие, но тратится больше времени. Рекомендованный диапазон [4,9]. -- `ZSTD(level)` - алгоритм сжатия ZSTD с указанным уровнем компрессии `level`. Возможный диапазон значений `level`: [1,22]. -Чем выше уровень, тем лучше сжатие, но тратится больше времени. Значение по-умолчанию: +- `LZ4HC(level)` - (level) - алгоритм сжатия LZ4_HC с указанным уровнем компрессии `level`. +Возможный диапазон значений `level`: \[3, 12\]. Значение по-умолчанию: 9. Чем выше уровень, тем лучше сжатие, но тратится больше времени. Рекомендованный диапазон \[4, 9\]. +- `ZSTD(level)` - алгоритм сжатия ZSTD с указанным уровнем компрессии `level`. Возможный диапазон значений `level`: \[1, 22\]. Значение по-умолчанию: 1. +Чем выше уровень, тем лучше сжатие, но тратится больше времени. - `Delta(delta_bytes)` - способ сжатия, при котором вместо числовых значений поля сохраняется разность между двумя соседними значениями. Значение `delta_bytes` - число байт для хранения дельты. -Возможные значения: 1, 2, 4, 8; +Возможные значения: 1, 2, 4, 8. Значение по-умолчанию: если `sizeof(type)` равен 1, 2, 4, 8 - `sizeof(type)`, иначе - 1. Пример использования: ``` @@ -22,14 +22,14 @@ CREATE TABLE codec_example dt Date CODEC(ZSTD), /* используется уровень сжатия по-умолчанию */ ts DateTime CODEC(LZ4HC), float_value Float32 CODEC(NONE), - double_value Float64 CODEC(LZ4HC(2)) + double_value Float64 CODEC(LZ4HC(9)) ) ENGINE = MergeTree PARTITION BY tuple() ORDER BY dt ``` -Кодеки могут комбинироваться между собой. В примере ниже - оптимизация для хранения timeseries метрик. +Кодеки могут комбинироваться между собой. Если для колонки указана своя последовательность кодеков, то общий табличный кодек не применяется (должен быть указан в последовательности принудительно, если нужен). В примере ниже - оптимизация для хранения timeseries метрик. Как правило, значения одной и той же метрики `path` не сильно различаются между собой, и выгоднее использовать дельта-компрессию вместо записи всего числа: ``` CREATE TABLE timeseries_example From 925a79501af22e76f1807aa9972bc00f088e1568 Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Thu, 17 Jan 2019 18:58:39 +0300 Subject: [PATCH 10/71] Review fix --- docs/ru/operations/table_engines/custom_compression_codec.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/table_engines/custom_compression_codec.md b/docs/ru/operations/table_engines/custom_compression_codec.md index cda705b2e3e..844482078bb 100644 --- a/docs/ru/operations/table_engines/custom_compression_codec.md +++ b/docs/ru/operations/table_engines/custom_compression_codec.md @@ -8,7 +8,7 @@ - `NONE` - сжатие отсутствует - `LZ4` -- `LZ4HC(level)` - (level) - алгоритм сжатия LZ4_HC с указанным уровнем компрессии `level`. +- `LZ4HC(level)` - алгоритм сжатия LZ4_HC с указанным уровнем компрессии `level`. Возможный диапазон значений `level`: \[3, 12\]. Значение по-умолчанию: 9. Чем выше уровень, тем лучше сжатие, но тратится больше времени. Рекомендованный диапазон \[4, 9\]. - `ZSTD(level)` - алгоритм сжатия ZSTD с указанным уровнем компрессии `level`. Возможный диапазон значений `level`: \[1, 22\]. Значение по-умолчанию: 1. Чем выше уровень, тем лучше сжатие, но тратится больше времени. From b6ad3cf14867fb6897db9227a850a8c06b5d5b12 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 17 Jan 2019 19:15:51 +0300 Subject: [PATCH 11/71] Remove shadow variable warning --- dbms/src/Storages/StorageHDFS.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index 95b58910052..5384cf6bafd 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -131,7 +131,7 @@ private: BlockInputStreams StorageHDFS::read( const Names & /*column_names*/, const SelectQueryInfo & /*query_info*/, - const Context & context, + const Context & context_, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, unsigned /*num_streams*/) @@ -140,7 +140,7 @@ BlockInputStreams StorageHDFS::read( uri, format_name, getSampleBlock(), - context, + context_, max_block_size)}; } From 998591b30d7ade8d394b3d17dae358ab72e1d6ec Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Fri, 18 Jan 2019 10:04:16 +0800 Subject: [PATCH 12/71] fix build failure --- dbms/src/Storages/Distributed/DirectoryMonitor.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp index 1c812909676..29a77557c6b 100644 --- a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp @@ -144,10 +144,7 @@ void StorageDistributedDirectoryMonitor::run() ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::string & name, const StorageDistributed & storage) { auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(storage.global_context.getSettingsRef()); - const auto pool_factory = [&storage, &timeouts] (const std::string & host, const UInt16 port, - const Protocol::Secure secure, - const std::string & user, const std::string & password, - const std::string & default_database) + const auto pool_factory = [&storage, &timeouts] (const Cluster::Address & address) -> ConnectionPoolPtr { const auto & cluster = storage.getCluster(); const auto & shards_info = cluster->getShardsInfo(); From 382f8ebc3a18bd099a412e56ff23d7f00a65546b Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Fri, 18 Jan 2019 11:29:21 +0300 Subject: [PATCH 13/71] Review fixes --- .../table_engines/custom_compression_codec.md | 2 +- .../table_engines/custom_compression_codec.md | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/operations/table_engines/custom_compression_codec.md b/docs/en/operations/table_engines/custom_compression_codec.md index 18e49c661f3..79f9776e55a 100644 --- a/docs/en/operations/table_engines/custom_compression_codec.md +++ b/docs/en/operations/table_engines/custom_compression_codec.md @@ -28,7 +28,7 @@ PARTITION BY tuple() ORDER BY dt ``` -Codecs can be combined in a pipeline. Default table codec is not included into pipeline. Example below shows an optimization approach for storing timeseries metrics. +Codecs can be combined in a pipeline. Default table codec is not included into pipeline (if it should be applied to a column, you have to specify it explicitly in pipeline). Example below shows an optimization approach for storing timeseries metrics. Usually, values for particular metric, stored in `path` does not differ significantly from point to point. Using delta-encoding allows to reduce disk space usage significantly. ``` CREATE TABLE timeseries_example diff --git a/docs/ru/operations/table_engines/custom_compression_codec.md b/docs/ru/operations/table_engines/custom_compression_codec.md index 844482078bb..5f70cc66dac 100644 --- a/docs/ru/operations/table_engines/custom_compression_codec.md +++ b/docs/ru/operations/table_engines/custom_compression_codec.md @@ -1,7 +1,7 @@ # Форматы сжатия для колонок -Помимо сжатия для колонок по-умолчанию, определяемого в [настройках сервера](../server_settings/settings.md#compression), +Помимо сжатия для колонок по умолчанию, определяемого в [настройках сервера](../server_settings/settings.md#compression), существует возможность указать формат сжатия индивидуально для каждой колонки. Поддерживаемые форматы: @@ -9,17 +9,17 @@ - `NONE` - сжатие отсутствует - `LZ4` - `LZ4HC(level)` - алгоритм сжатия LZ4_HC с указанным уровнем компрессии `level`. -Возможный диапазон значений `level`: \[3, 12\]. Значение по-умолчанию: 9. Чем выше уровень, тем лучше сжатие, но тратится больше времени. Рекомендованный диапазон \[4, 9\]. -- `ZSTD(level)` - алгоритм сжатия ZSTD с указанным уровнем компрессии `level`. Возможный диапазон значений `level`: \[1, 22\]. Значение по-умолчанию: 1. +Возможный диапазон значений `level`: \[3, 12\]. Значение по умолчанию: 9. Чем выше уровень, тем лучше сжатие, но тратится больше времени. Рекомендованный диапазон \[4, 9\]. +- `ZSTD(level)` - алгоритм сжатия ZSTD с указанным уровнем компрессии `level`. Возможный диапазон значений `level`: \[1, 22\]. Значение по умолчанию: 1. Чем выше уровень, тем лучше сжатие, но тратится больше времени. - `Delta(delta_bytes)` - способ сжатия, при котором вместо числовых значений поля сохраняется разность между двумя соседними значениями. Значение `delta_bytes` - число байт для хранения дельты. -Возможные значения: 1, 2, 4, 8. Значение по-умолчанию: если `sizeof(type)` равен 1, 2, 4, 8 - `sizeof(type)`, иначе - 1. +Возможные значения: 1, 2, 4, 8. Значение по умолчанию: если `sizeof(type)` равен 1, 2, 4, 8 - `sizeof(type)`, иначе - 1. Пример использования: ``` CREATE TABLE codec_example ( - dt Date CODEC(ZSTD), /* используется уровень сжатия по-умолчанию */ + dt Date CODEC(ZSTD), /* используется уровень сжатия по умолчанию */ ts DateTime CODEC(LZ4HC), float_value Float32 CODEC(NONE), double_value Float64 CODEC(LZ4HC(9)) From 2f7b63cf7130761aa027cb2e5dfac2647cda95f2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 18 Jan 2019 13:55:03 +0300 Subject: [PATCH 14/71] Fix review notes --- dbms/src/IO/ReadBufferFromHDFS.cpp | 5 ----- dbms/src/IO/ReadBufferFromHDFS.h | 2 -- dbms/src/IO/WriteBufferFromHDFS.cpp | 16 ++++------------ dbms/src/IO/WriteBufferFromHDFS.h | 2 -- 4 files changed, 4 insertions(+), 21 deletions(-) diff --git a/dbms/src/IO/ReadBufferFromHDFS.cpp b/dbms/src/IO/ReadBufferFromHDFS.cpp index 10899f24fdc..4461bef3c6b 100644 --- a/dbms/src/IO/ReadBufferFromHDFS.cpp +++ b/dbms/src/IO/ReadBufferFromHDFS.cpp @@ -90,11 +90,6 @@ bool ReadBufferFromHDFS::nextImpl() return true; } -const std::string & ReadBufferFromHDFS::getHDFSUri() const -{ - return impl->hdfs_uri; -} - ReadBufferFromHDFS::~ReadBufferFromHDFS() { } diff --git a/dbms/src/IO/ReadBufferFromHDFS.h b/dbms/src/IO/ReadBufferFromHDFS.h index 7e0656098de..a7fcd117d9f 100644 --- a/dbms/src/IO/ReadBufferFromHDFS.h +++ b/dbms/src/IO/ReadBufferFromHDFS.h @@ -24,8 +24,6 @@ public: bool nextImpl() override; ~ReadBufferFromHDFS() override; - - const std::string & getHDFSUri() const; }; } #endif diff --git a/dbms/src/IO/WriteBufferFromHDFS.cpp b/dbms/src/IO/WriteBufferFromHDFS.cpp index 4e1b916dafa..7f2edc84b77 100644 --- a/dbms/src/IO/WriteBufferFromHDFS.cpp +++ b/dbms/src/IO/WriteBufferFromHDFS.cpp @@ -24,7 +24,6 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl WriteBufferFromHDFSImpl(const std::string & hdfs_name_) { - builder = hdfsNewBuilder(); hdfs_uri = hdfs_name_; Poco::URI uri(hdfs_name_); auto & host = uri.getHost(); @@ -34,6 +33,8 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl { throw Exception("Illegal HDFS URI: " + hdfs_uri, ErrorCodes::BAD_ARGUMENTS); } + + builder = hdfsNewBuilder(); // set read/connect timeout, default value in libhdfs3 is about 1 hour, and too large /// TODO Allow to tune from query Settings. hdfsBuilderConfSetStr(builder, "input.read.timeout", "60000"); // 1 min @@ -46,6 +47,7 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl if (fs == nullptr) { + hdfsFreeBuilder(builder); throw Exception("Unable to connect to HDFS: " + std::string(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR); } @@ -53,14 +55,9 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl } ~WriteBufferFromHDFSImpl() - { - close(); - hdfsFreeBuilder(builder); - } - - void close() { hdfsCloseFile(fs, fout); + hdfsDisconnect(fs); } @@ -106,11 +103,6 @@ void WriteBufferFromHDFS::sync() impl->sync(); } -const std::string & WriteBufferFromHDFS::getHDFSUri() const -{ - return impl->hdfs_uri; -} - WriteBufferFromHDFS::~WriteBufferFromHDFS() { } diff --git a/dbms/src/IO/WriteBufferFromHDFS.h b/dbms/src/IO/WriteBufferFromHDFS.h index bd89405a47c..b3b56ded18b 100644 --- a/dbms/src/IO/WriteBufferFromHDFS.h +++ b/dbms/src/IO/WriteBufferFromHDFS.h @@ -25,8 +25,6 @@ public: ~WriteBufferFromHDFS() override; - const std::string & getHDFSUri() const; - void sync(); }; } From 83c32727dcd041303b3ce8fc6bc9b3d094505048 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 18 Jan 2019 13:57:00 +0300 Subject: [PATCH 15/71] Remove strange commit --- dbms/src/IO/ReadBufferFromHDFS.h | 2 +- dbms/src/IO/WriteBufferFromHDFS.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/IO/ReadBufferFromHDFS.h b/dbms/src/IO/ReadBufferFromHDFS.h index a7fcd117d9f..6d00c8b2310 100644 --- a/dbms/src/IO/ReadBufferFromHDFS.h +++ b/dbms/src/IO/ReadBufferFromHDFS.h @@ -10,7 +10,7 @@ namespace DB { -/** Accepts path to file and opens it, or pre-opened file descriptor. +/** Accepts HDFS path to file and opens it. * Closes file by himself (thus "owns" a file descriptor). */ class ReadBufferFromHDFS : public BufferWithOwnMemory diff --git a/dbms/src/IO/WriteBufferFromHDFS.h b/dbms/src/IO/WriteBufferFromHDFS.h index b3b56ded18b..35aafc00921 100644 --- a/dbms/src/IO/WriteBufferFromHDFS.h +++ b/dbms/src/IO/WriteBufferFromHDFS.h @@ -10,7 +10,7 @@ namespace DB { -/** Accepts path to file and opens it, or pre-opened file descriptor. +/** Accepts HDFS path to file and opens it. * Closes file by himself (thus "owns" a file descriptor). */ class WriteBufferFromHDFS : public BufferWithOwnMemory From 71e58ca001c224b917880e519bd4bb7aedad6be7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 18 Jan 2019 14:01:07 +0300 Subject: [PATCH 16/71] Fix potential leak in read buffer --- dbms/src/IO/ReadBufferFromHDFS.cpp | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/dbms/src/IO/ReadBufferFromHDFS.cpp b/dbms/src/IO/ReadBufferFromHDFS.cpp index 4461bef3c6b..ab3d955f567 100644 --- a/dbms/src/IO/ReadBufferFromHDFS.cpp +++ b/dbms/src/IO/ReadBufferFromHDFS.cpp @@ -23,7 +23,6 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : hdfs_uri(hdfs_name_) , builder(hdfsNewBuilder()) { - builder = hdfsNewBuilder(); hdfs_uri = hdfs_name_; Poco::URI uri(hdfs_name_); auto & host = uri.getHost(); @@ -35,6 +34,7 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl } // set read/connect timeout, default value in libhdfs3 is about 1 hour, and too large /// TODO Allow to tune from query Settings. + builder = hdfsNewBuilder(); hdfsBuilderConfSetStr(builder, "input.read.timeout", "60000"); // 1 min hdfsBuilderConfSetStr(builder, "input.write.timeout", "60000"); // 1 min hdfsBuilderConfSetStr(builder, "input.connect.timeout", "60000"); // 1 min @@ -45,6 +45,7 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl if (fs == nullptr) { + hdfsFreeBuilder(builder); throw Exception("Unable to connect to HDFS: " + std::string(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR); } @@ -52,14 +53,9 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl } ~ReadBufferFromHDFSImpl() - { - close(); - hdfsFreeBuilder(builder); - } - - void close() { hdfsCloseFile(fs, fin); + hdfsDisconnect(fs); } int read(char * start, size_t size) From a139f375c378957d11a46b0daf1b58bb1e1e5f18 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 18 Jan 2019 14:39:36 +0300 Subject: [PATCH 17/71] Remove copypaste --- dbms/src/IO/ReadBufferFromHDFS.cpp | 1 - dbms/src/IO/WriteBufferFromHDFS.cpp | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/src/IO/ReadBufferFromHDFS.cpp b/dbms/src/IO/ReadBufferFromHDFS.cpp index ab3d955f567..e0d55355cc4 100644 --- a/dbms/src/IO/ReadBufferFromHDFS.cpp +++ b/dbms/src/IO/ReadBufferFromHDFS.cpp @@ -23,7 +23,6 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : hdfs_uri(hdfs_name_) , builder(hdfsNewBuilder()) { - hdfs_uri = hdfs_name_; Poco::URI uri(hdfs_name_); auto & host = uri.getHost(); auto port = uri.getPort(); diff --git a/dbms/src/IO/WriteBufferFromHDFS.cpp b/dbms/src/IO/WriteBufferFromHDFS.cpp index 7f2edc84b77..ca3df85bf44 100644 --- a/dbms/src/IO/WriteBufferFromHDFS.cpp +++ b/dbms/src/IO/WriteBufferFromHDFS.cpp @@ -23,8 +23,8 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl hdfsFile fout; WriteBufferFromHDFSImpl(const std::string & hdfs_name_) + : hdfs_uri(hdfs_name_) { - hdfs_uri = hdfs_name_; Poco::URI uri(hdfs_name_); auto & host = uri.getHost(); auto port = uri.getPort(); From a5af465b20922f10ef11dec102061d10f4e29a72 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 18 Jan 2019 14:47:50 +0300 Subject: [PATCH 18/71] Add option for stateless tests disabling --- dbms/tests/clickhouse-test | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 626881aafb5..32eec7271eb 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -132,6 +132,9 @@ def main(args): if 'stateful' in suite and not is_data_present(): print("Won't run stateful tests because test data wasn't loaded.") continue + if 'stateless' in suite and args.no_stateless: + print("Won't run stateless tests because they were manually disabled.") + continue # Reverse sort order: we want run newest test first. # And not reverse subtests @@ -343,6 +346,7 @@ if __name__ == '__main__': parser.add_argument('--order', default = 'desc', help = 'Run order (asc, desc, random)') parser.add_argument('--testname', action = 'store_true', default = None, dest = 'testname', help = 'Make query with test name before test run') + parser.add_argument('--no-stateless', action = 'store_true', help = 'Disable all stateless tests') parser.add_argument('--skip', nargs='+', help = "Skip these tests") parser.add_argument('--no-long', action = 'store_false', dest = 'no_long', help = 'Do not run long tests') group = parser.add_mutually_exclusive_group(required = False) From 1f7d17bad25b86339b0d58fa722807d5f3e93835 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 18 Jan 2019 15:36:20 +0300 Subject: [PATCH 19/71] Added recommended package for clickhouse-server #4091 --- debian/control | 1 + 1 file changed, 1 insertion(+) diff --git a/debian/control b/debian/control index 745764fd9df..04db4f9ae95 100644 --- a/debian/control +++ b/debian/control @@ -37,6 +37,7 @@ Description: Common files for ClickHouse Package: clickhouse-server Architecture: all Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-common-static (= ${binary:Version}), adduser +Recommends: libcap2-bin Replaces: clickhouse-server-common, clickhouse-server-base Provides: clickhouse-server-common Description: Server binary for ClickHouse From 89ffb577b1a1cff5af8b98f48d3a44b810ee0807 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Fri, 18 Jan 2019 15:48:38 +0300 Subject: [PATCH 20/71] Implement additional Kafka engine setting for allowed parsing errors --- dbms/src/Storages/Kafka/KafkaSettings.h | 5 ++-- dbms/src/Storages/Kafka/StorageKafka.cpp | 35 ++++++++++++++++++++++-- dbms/src/Storages/Kafka/StorageKafka.h | 4 ++- 3 files changed, 38 insertions(+), 6 deletions(-) diff --git a/dbms/src/Storages/Kafka/KafkaSettings.h b/dbms/src/Storages/Kafka/KafkaSettings.h index cde3221df45..c12d44d8cb7 100644 --- a/dbms/src/Storages/Kafka/KafkaSettings.h +++ b/dbms/src/Storages/Kafka/KafkaSettings.h @@ -23,11 +23,12 @@ struct KafkaSettings M(SettingString, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.") \ M(SettingString, kafka_topic_list, "", "A list of Kafka topics.") \ M(SettingString, kafka_group_name, "", "A group of Kafka consumers.") \ - M(SettingString, kafka_format, "", "Message format for Kafka engine.") \ + M(SettingString, kafka_format, "", "The message format for Kafka engine.") \ M(SettingChar, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.") \ M(SettingString, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine") \ M(SettingUInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.") \ - M(SettingUInt64, kafka_max_block_size, 0, "The maximum block size per table for Kafka engine.") + M(SettingUInt64, kafka_max_block_size, 0, "The maximum block size per table for Kafka engine.") \ + M(SettingUInt64, kafka_skip_broken_messages, 0, "Skip at least this number of consecutive broken messages from Kafka topic") #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \ TYPE NAME {DEFAULT}; diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 063fff81e2d..77e3ca7d795 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -139,6 +139,12 @@ public: { // Always skip unknown fields regardless of the context (JSON or TSKV) context.setSetting("input_format_skip_unknown_fields", 1u); + + // We don't use ratio since the number of Kafka messages may vary from stream to stream. + // Thus, ratio is meaningless. + context.setSetting("input_format_allow_errors_ratio", 1.); + context.setSetting("input_format_allow_errors_num", storage.skip_broken); + if (schema.size() > 0) context.setSetting("format_schema", schema); } @@ -248,7 +254,7 @@ StorageKafka::StorageKafka( const ColumnsDescription & columns_, const String & brokers_, const String & group_, const Names & topics_, const String & format_name_, char row_delimiter_, const String & schema_name_, - size_t num_consumers_, size_t max_block_size_) + size_t num_consumers_, size_t max_block_size_, size_t skip_broken_) : IStorage{columns_}, table_name(table_name_), database_name(database_name_), global_context(context_), topics(global_context.getMacros()->expand(topics_)), @@ -258,7 +264,8 @@ StorageKafka::StorageKafka( row_delimiter(row_delimiter_), schema_name(global_context.getMacros()->expand(schema_name_)), num_consumers(num_consumers_), max_block_size(max_block_size_), log(&Logger::get("StorageKafka (" + table_name_ + ")")), - semaphore(0, num_consumers_), mutex(), consumers() + semaphore(0, num_consumers_), mutex(), consumers(), + skip_broken(skip_broken_) { task = global_context.getSchedulePool().createTask(log->name(), [this]{ streamThread(); }); task->deactivate(); @@ -538,6 +545,8 @@ void registerStorageKafka(StorageFactory & factory) * - Row delimiter * - Schema (optional, if the format supports it) * - Number of consumers + * - Max block size for background consumption + * - Skip (at least) unreadable messages number */ // Check arguments and settings @@ -571,6 +580,7 @@ void registerStorageKafka(StorageFactory & factory) CHECK_KAFKA_STORAGE_ARGUMENT(6, kafka_schema) CHECK_KAFKA_STORAGE_ARGUMENT(7, kafka_num_consumers) CHECK_KAFKA_STORAGE_ARGUMENT(8, kafka_max_block_size) + CHECK_KAFKA_STORAGE_ARGUMENT(9, kafka_skip_broken_messages) #undef CHECK_KAFKA_STORAGE_ARGUMENT // Get and check broker list @@ -728,6 +738,7 @@ void registerStorageKafka(StorageFactory & factory) } else { + // TODO: no check if the integer is really positive throw Exception("Maximum block size must be a positive integer", ErrorCodes::BAD_ARGUMENTS); } } @@ -736,9 +747,27 @@ void registerStorageKafka(StorageFactory & factory) max_block_size = static_cast(kafka_settings.kafka_max_block_size.value); } + size_t skip_broken = 0; + if (args_count >= 9) + { + auto ast = typeid_cast(engine_args[8].get()); + if (ast && ast->value.getType() == Field::Types::UInt64) + { + skip_broken = static_cast(safeGet(ast->value)); + } + else + { + throw Exception("Number of broken messages to skip must be a non-negative integer", ErrorCodes::BAD_ARGUMENTS); + } + } + else if (kafka_settings.kafka_skip_broken_messages.changed) + { + skip_broken = static_cast(kafka_settings.kafka_skip_broken_messages.value); + } + return StorageKafka::create( args.table_name, args.database_name, args.context, args.columns, - brokers, group, topics, format, row_delimiter, schema, num_consumers, max_block_size); + brokers, group, topics, format, row_delimiter, schema, num_consumers, max_block_size, skip_broken); }); } diff --git a/dbms/src/Storages/Kafka/StorageKafka.h b/dbms/src/Storages/Kafka/StorageKafka.h index d6b324e8a85..9d5f27fb4d6 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.h +++ b/dbms/src/Storages/Kafka/StorageKafka.h @@ -80,6 +80,8 @@ private: std::mutex mutex; std::vector consumers; /// Available consumers + size_t skip_broken; + // Stream thread BackgroundSchedulePool::TaskHolder task; std::atomic stream_cancelled{false}; @@ -101,7 +103,7 @@ protected: const ColumnsDescription & columns_, const String & brokers_, const String & group_, const Names & topics_, const String & format_name_, char row_delimiter_, const String & schema_name_, - size_t num_consumers_, size_t max_block_size_); + size_t num_consumers_, size_t max_block_size_, size_t skip_broken); }; } From 7c1c6eba8be8069c681ccd29d3142eb45167bce0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 18 Jan 2019 15:48:55 +0300 Subject: [PATCH 21/71] Wrap hdfsBuilder into unique ptr --- dbms/src/IO/ReadBufferFromHDFS.cpp | 27 +++++++++++++++++---------- dbms/src/IO/WriteBufferFromHDFS.cpp | 28 ++++++++++++++++++---------- 2 files changed, 35 insertions(+), 20 deletions(-) diff --git a/dbms/src/IO/ReadBufferFromHDFS.cpp b/dbms/src/IO/ReadBufferFromHDFS.cpp index e0d55355cc4..2545c21b094 100644 --- a/dbms/src/IO/ReadBufferFromHDFS.cpp +++ b/dbms/src/IO/ReadBufferFromHDFS.cpp @@ -14,8 +14,16 @@ namespace ErrorCodes struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl { + struct HDFSBuilderDeleter + { + void operator()(hdfsBuilder * builder) + { + hdfsFreeBuilder(builder); + } + }; + std::string hdfs_uri; - hdfsBuilder * builder; + std::unique_ptr builder; hdfsFS fs; hdfsFile fin; @@ -33,19 +41,18 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl } // set read/connect timeout, default value in libhdfs3 is about 1 hour, and too large /// TODO Allow to tune from query Settings. - builder = hdfsNewBuilder(); - hdfsBuilderConfSetStr(builder, "input.read.timeout", "60000"); // 1 min - hdfsBuilderConfSetStr(builder, "input.write.timeout", "60000"); // 1 min - hdfsBuilderConfSetStr(builder, "input.connect.timeout", "60000"); // 1 min + hdfsBuilderConfSetStr(builder.get(), "input.read.timeout", "60000"); // 1 min + hdfsBuilderConfSetStr(builder.get(), "input.write.timeout", "60000"); // 1 min + hdfsBuilderConfSetStr(builder.get(), "input.connect.timeout", "60000"); // 1 min - hdfsBuilderSetNameNode(builder, host.c_str()); - hdfsBuilderSetNameNodePort(builder, port); - fs = hdfsBuilderConnect(builder); + hdfsBuilderSetNameNode(builder.get(), host.c_str()); + hdfsBuilderSetNameNodePort(builder.get(), port); + fs = hdfsBuilderConnect(builder.get()); if (fs == nullptr) { - hdfsFreeBuilder(builder); - throw Exception("Unable to connect to HDFS: " + std::string(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR); + throw Exception("Unable to connect to HDFS: " + std::string(hdfsGetLastError()), + ErrorCodes::NETWORK_ERROR); } fin = hdfsOpenFile(fs, path.c_str(), O_RDONLY, 0, 0, 0); diff --git a/dbms/src/IO/WriteBufferFromHDFS.cpp b/dbms/src/IO/WriteBufferFromHDFS.cpp index ca3df85bf44..838dedb20d9 100644 --- a/dbms/src/IO/WriteBufferFromHDFS.cpp +++ b/dbms/src/IO/WriteBufferFromHDFS.cpp @@ -17,13 +17,22 @@ extern const int CANNOT_FSYNC; struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl { + struct HDFSBuilderDeleter + { + void operator()(hdfsBuilder * builder) + { + hdfsFreeBuilder(builder); + } + }; + std::string hdfs_uri; - hdfsBuilder * builder; + std::unique_ptr builder; hdfsFS fs; hdfsFile fout; WriteBufferFromHDFSImpl(const std::string & hdfs_name_) : hdfs_uri(hdfs_name_) + , builder(hdfsNewBuilder()) { Poco::URI uri(hdfs_name_); auto & host = uri.getHost(); @@ -34,21 +43,20 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl throw Exception("Illegal HDFS URI: " + hdfs_uri, ErrorCodes::BAD_ARGUMENTS); } - builder = hdfsNewBuilder(); // set read/connect timeout, default value in libhdfs3 is about 1 hour, and too large /// TODO Allow to tune from query Settings. - hdfsBuilderConfSetStr(builder, "input.read.timeout", "60000"); // 1 min - hdfsBuilderConfSetStr(builder, "input.write.timeout", "60000"); // 1 min - hdfsBuilderConfSetStr(builder, "input.connect.timeout", "60000"); // 1 min + hdfsBuilderConfSetStr(builder.get(), "input.read.timeout", "60000"); // 1 min + hdfsBuilderConfSetStr(builder.get(), "input.write.timeout", "60000"); // 1 min + hdfsBuilderConfSetStr(builder.get(), "input.connect.timeout", "60000"); // 1 min - hdfsBuilderSetNameNode(builder, host.c_str()); - hdfsBuilderSetNameNodePort(builder, port); - fs = hdfsBuilderConnect(builder); + hdfsBuilderSetNameNode(builder.get(), host.c_str()); + hdfsBuilderSetNameNodePort(builder.get(), port); + fs = hdfsBuilderConnect(builder.get()); if (fs == nullptr) { - hdfsFreeBuilder(builder); - throw Exception("Unable to connect to HDFS: " + std::string(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR); + throw Exception("Unable to connect to HDFS: " + std::string(hdfsGetLastError()), + ErrorCodes::NETWORK_ERROR); } fout = hdfsOpenFile(fs, path.c_str(), O_WRONLY, 0, 0, 0); From 5f74094edb6f09db978dc74278ece0255a7b61b9 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Fri, 18 Jan 2019 16:15:36 +0300 Subject: [PATCH 22/71] Fix a setting description. --- dbms/src/Storages/Kafka/KafkaSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/Kafka/KafkaSettings.h b/dbms/src/Storages/Kafka/KafkaSettings.h index c12d44d8cb7..d070c82eb70 100644 --- a/dbms/src/Storages/Kafka/KafkaSettings.h +++ b/dbms/src/Storages/Kafka/KafkaSettings.h @@ -28,7 +28,7 @@ struct KafkaSettings M(SettingString, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine") \ M(SettingUInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.") \ M(SettingUInt64, kafka_max_block_size, 0, "The maximum block size per table for Kafka engine.") \ - M(SettingUInt64, kafka_skip_broken_messages, 0, "Skip at least this number of consecutive broken messages from Kafka topic") + M(SettingUInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block") #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \ TYPE NAME {DEFAULT}; From 05b71c9d60c6a8d178bf122c24b30c18b981bec7 Mon Sep 17 00:00:00 2001 From: Nikhil Raman Date: Fri, 18 Jan 2019 18:59:12 +0530 Subject: [PATCH 23/71] Fix `toStartOfFiveMinute` doc. (#4096) --- docs/en/query_language/functions/date_time_functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/query_language/functions/date_time_functions.md b/docs/en/query_language/functions/date_time_functions.md index d07457e1410..9d9f60d627e 100644 --- a/docs/en/query_language/functions/date_time_functions.md +++ b/docs/en/query_language/functions/date_time_functions.md @@ -80,7 +80,7 @@ Rounds down a date with time to the start of the minute. ## toStartOfFiveMinute -Rounds down a date with time to the start of the hour. +Rounds down a date with time to the start of the five-minute interval. ## toStartOfFifteenMinutes From 0939387514936f4cf94d8495faf777d5e5d14620 Mon Sep 17 00:00:00 2001 From: proller Date: Fri, 18 Jan 2019 18:19:53 +0300 Subject: [PATCH 24/71] Build fixes (#4076) Move copy_headers.sh and formath.sh from root. Debian: use default build dir (obj-x86_64-linux-gnu) Debian: add test after build (GLIBC_required_version) cmake: dont run copy headers if already copied --- CMakeLists.txt | 2 +- contrib/CMakeLists.txt | 17 +++++++++-------- dbms/programs/clang/CMakeLists.txt | 2 +- .../programs/clang/copy_headers.sh | 2 +- dbms/programs/server/CMakeLists.txt | 5 ++++- debian/rules | 18 +++++++++--------- .../query_language/table_functions/remote.md | 2 +- libs/libglibc-compatibility/CMakeLists.txt | 4 +++- format.sh => utils/check-style/format.sh | 6 +++++- 9 files changed, 34 insertions(+), 24 deletions(-) rename copy_headers.sh => dbms/programs/clang/copy_headers.sh (99%) rename format.sh => utils/check-style/format.sh (76%) diff --git a/CMakeLists.txt b/CMakeLists.txt index ef504c107bf..fa924694718 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -179,8 +179,8 @@ endif (TEST_COVERAGE) if (ENABLE_TESTS) message (STATUS "Tests are enabled") - enable_testing() endif () +enable_testing() # Enable for tests without binary # when installing to /usr - place configs to /etc but for /usr/local place to /usr/local/etc if (CMAKE_INSTALL_PREFIX STREQUAL "/usr") diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 8504e5facdb..e0203b1b046 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -54,6 +54,7 @@ if (USE_INTERNAL_UNWIND_LIBRARY) endif () if (USE_INTERNAL_ZLIB_LIBRARY) + set (ZLIB_ENABLE_TESTS 0 CACHE INTERNAL "") set (ZLIB_COMPAT 1 CACHE INTERNAL "") # also enables WITH_GZFILEOP set (WITH_NATIVE_INSTRUCTIONS ${ARCH_NATIVE} CACHE INTERNAL "") if (OS_FREEBSD OR ARCH_I386) @@ -74,15 +75,15 @@ if (USE_INTERNAL_ZLIB_LIBRARY) target_compile_definitions (zlibstatic PUBLIC X86_64 UNALIGNED_OK) endif () - set_target_properties(example PROPERTIES EXCLUDE_FROM_ALL 1) - if (TARGET example64) - set_target_properties(example64 PROPERTIES EXCLUDE_FROM_ALL 1) - endif () + #set_target_properties(example PROPERTIES EXCLUDE_FROM_ALL 1) + #if (TARGET example64) + # set_target_properties(example64 PROPERTIES EXCLUDE_FROM_ALL 1) + #endif () - set_target_properties(minigzip PROPERTIES EXCLUDE_FROM_ALL 1) - if (TARGET minigzip64) - set_target_properties(minigzip64 PROPERTIES EXCLUDE_FROM_ALL 1) - endif () + #set_target_properties(minigzip PROPERTIES EXCLUDE_FROM_ALL 1) + #if (TARGET minigzip64) + # set_target_properties(minigzip64 PROPERTIES EXCLUDE_FROM_ALL 1) + #endif () endif () if (USE_INTERNAL_CCTZ_LIBRARY) diff --git a/dbms/programs/clang/CMakeLists.txt b/dbms/programs/clang/CMakeLists.txt index 78bfa6b55e7..8bf9e3760e2 100644 --- a/dbms/programs/clang/CMakeLists.txt +++ b/dbms/programs/clang/CMakeLists.txt @@ -27,7 +27,7 @@ elseif (EXISTS ${INTERNAL_COMPILER_BIN_ROOT}${INTERNAL_COMPILER_EXECUTABLE}) endif () if (COPY_HEADERS_COMPILER AND OS_LINUX) - add_custom_target (copy-headers env CLANG=${COPY_HEADERS_COMPILER} BUILD_PATH=${ClickHouse_BINARY_DIR} DESTDIR=${ClickHouse_SOURCE_DIR} ${ClickHouse_SOURCE_DIR}/copy_headers.sh ${ClickHouse_SOURCE_DIR} ${TMP_HEADERS_DIR} DEPENDS ${COPY_HEADERS_DEPENDS} WORKING_DIRECTORY ${ClickHouse_SOURCE_DIR} SOURCES ${ClickHouse_SOURCE_DIR}/copy_headers.sh) + add_custom_target (copy-headers [ -f ${TMP_HEADERS_DIR}/dbms/src/Interpreters/SpecializedAggregator.h ] || env CLANG=${COPY_HEADERS_COMPILER} BUILD_PATH=${ClickHouse_BINARY_DIR} DESTDIR=${ClickHouse_SOURCE_DIR} ${CMAKE_CURRENT_SOURCE_DIR}/copy_headers.sh ${ClickHouse_SOURCE_DIR} ${TMP_HEADERS_DIR} DEPENDS ${COPY_HEADERS_DEPENDS} WORKING_DIRECTORY ${ClickHouse_SOURCE_DIR} SOURCES copy_headers.sh) if (USE_INTERNAL_LLVM_LIBRARY) set (CLANG_HEADERS_DIR "${ClickHouse_SOURCE_DIR}/contrib/llvm/clang/lib/Headers") diff --git a/copy_headers.sh b/dbms/programs/clang/copy_headers.sh similarity index 99% rename from copy_headers.sh rename to dbms/programs/clang/copy_headers.sh index a32a809774e..2f7b2f9f5cf 100755 --- a/copy_headers.sh +++ b/dbms/programs/clang/copy_headers.sh @@ -19,7 +19,7 @@ set -e # # sudo ./copy_headers.sh . /usr/share/clickhouse/headers/ -SOURCE_PATH=${1:-.} +SOURCE_PATH=${1:-../../..} DST=${2:-$SOURCE_PATH/../headers} BUILD_PATH=${BUILD_PATH=${3:-$SOURCE_PATH/build}} diff --git a/dbms/programs/server/CMakeLists.txt b/dbms/programs/server/CMakeLists.txt index d8caa07b743..9de696e417f 100644 --- a/dbms/programs/server/CMakeLists.txt +++ b/dbms/programs/server/CMakeLists.txt @@ -25,7 +25,10 @@ endif () if (OS_LINUX AND MAKE_STATIC_LIBRARIES) set (GLIBC_MAX_REQUIRED 2.4 CACHE INTERNAL "") - add_test(NAME GLIBC_required_version COMMAND bash -c "readelf -s ${CMAKE_CURRENT_BINARY_DIR}/../clickhouse-server | grep '@GLIBC' | grep -oP 'GLIBC_[\\d\\.]+' | sort | uniq | sort -r | perl -lnE 'exit 1 if $_ gt q{GLIBC_${GLIBC_MAX_REQUIRED}}'") + # temporary disabled. to enable - change 'exit 0' to 'exit $a' + add_test(NAME GLIBC_required_version COMMAND bash -c "readelf -s ${CMAKE_CURRENT_BINARY_DIR}/../clickhouse-server | perl -nE 'END {exit 0 if $a} ++$a, print if /\\x40GLIBC_(\\S+)/ and pack(q{C*}, split /\\./, \$1) gt pack q{C*}, split /\\./, q{${GLIBC_MAX_REQUIRED}}'") + + #add_test(NAME GLIBC_required_version COMMAND bash -c "readelf -s ${CMAKE_CURRENT_BINARY_DIR}/../clickhouse-server | grep '@GLIBC' | grep -oP 'GLIBC_[\\d\\.]+' | sort | uniq | sort --version-sort --reverse | perl -lnE 'warn($_), exit 1 if $_ gt q{GLIBC_${GLIBC_MAX_REQUIRED}}'") # old endif () install ( diff --git a/debian/rules b/debian/rules index 06f4316f615..cb0c5137f18 100755 --- a/debian/rules +++ b/debian/rules @@ -8,7 +8,7 @@ export DH_VERBOSE=1 export DEB_BUILD_MAINT_OPTIONS=hardening=+all,-pie # because copy_headers.sh have hardcoded path to build/include_directories.txt -BUILDDIR = build +BUILDDIR = obj-$(DEB_HOST_GNU_TYPE) CURDIR = $(shell pwd) DESTDIR = $(CURDIR)/debian/tmp @@ -80,23 +80,23 @@ ifneq ($(THREADS_COUNT),) endif %: - dh $@ $(DH_FLAGS) --buildsystem=cmake --builddirectory=$(BUILDDIR) + dh $@ $(DH_FLAGS) --buildsystem=cmake override_dh_auto_configure: dh_auto_configure -- $(CMAKE_FLAGS) override_dh_auto_build: # Fix for ninja. Do not add -O. - cd $(BUILDDIR) && $(MAKE) $(THREADS_COUNT) $(MAKE_TARGET) - #cd $(BUILDDIR) && cmake --build . -- -j$(THREADS_COUNT) # cmake return true on error + $(MAKE) $(THREADS_COUNT) -C $(BUILDDIR) $(MAKE_TARGET) +# #cd $(BUILDDIR) && cmake --build . -- -j$(THREADS_COUNT) # cmake return true on error override_dh_auto_test: - #TODO, use ENABLE_TESTS=1 - #./debian/tests_wrapper.sh +# #TODO, use ENABLE_TESTS=1 +# #./debian/tests_wrapper.sh +# cd $(BUILDDIR) && ctest $(THREADS_COUNT) -V -R GLIBC_required_version + cd $(BUILDDIR) && ctest $(THREADS_COUNT) -V -E with_server override_dh_clean: - rm -rf $(BUILDDIR) - rm -rf $(DESTDIR) rm -rf debian/copyright debian/clickhouse-client.docs debian/clickhouse-common-static.docs dh_clean @@ -130,7 +130,7 @@ override_dh_install: dh_install --list-missing --sourcedir=$(DESTDIR) override_dh_auto_install: - env DESTDIR=$(DESTDIR) $(MAKE) -C $(BUILDDIR) install + env DESTDIR=$(DESTDIR) $(MAKE) $(THREADS_COUNT) -C $(BUILDDIR) install override_dh_shlibdeps: true # We depend only on libc and dh_shlibdeps gives us wrong (too strict) dependency. diff --git a/docs/ru/query_language/table_functions/remote.md b/docs/ru/query_language/table_functions/remote.md index 555a752a94e..02aa48c7d63 100644 --- a/docs/ru/query_language/table_functions/remote.md +++ b/docs/ru/query_language/table_functions/remote.md @@ -72,6 +72,6 @@ example01-{01..02}-{1|2} Если пользователь не задан,то используется `default`. Если пароль не задан, то используется пустой пароль. -`remoteSecure` - аналогично функции `remote` но с соединением по шифрованому каналу. Порт по умолчанию - `tcp_port_secure` из конфига или 9440. +`remoteSecure` - аналогично функции `remote`, но с соединением по шифрованому каналу. Порт по умолчанию - `tcp_port_secure` из конфига или 9440. [Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/table_functions/remote/) diff --git a/libs/libglibc-compatibility/CMakeLists.txt b/libs/libglibc-compatibility/CMakeLists.txt index 15aa9fe7e3d..392db73aca3 100644 --- a/libs/libglibc-compatibility/CMakeLists.txt +++ b/libs/libglibc-compatibility/CMakeLists.txt @@ -31,4 +31,6 @@ add_library (glibc-compatibility ${GLIBC_COMPATIBILITY_SOURCES}) target_include_directories(glibc-compatibility PRIVATE libcxxabi) -add_subdirectory (tests) +if(ENABLE_TESTS) + add_subdirectory(tests) +endif() diff --git a/format.sh b/utils/check-style/format.sh similarity index 76% rename from format.sh rename to utils/check-style/format.sh index e268dc4fbec..0e7498fed03 100755 --- a/format.sh +++ b/utils/check-style/format.sh @@ -1,4 +1,8 @@ -#/usr/bin/env bash +#!/usr/bin/env bash + +# Format almost all code with current clang-format settings + +cd `readlink -f $(dirname $0)`/../.. clang_format=`bash -c "compgen -c clang-format | grep 'clang-format-[[:digit:]]' | sort --version-sort --reverse | head -n1"` if [ ! -z $clang_format ]; then From f1a7f6016535d0904f0a2372a713ceae2fe3f7c9 Mon Sep 17 00:00:00 2001 From: Boris Granveaud Date: Fri, 18 Jan 2019 16:44:53 +0100 Subject: [PATCH 25/71] new functions filesystemAvailable, filesystemFree and filesystemCapacity --- dbms/src/Functions/filesystem.cpp | 68 +++++++++++++++++++ .../registerFunctionsMiscellaneous.cpp | 2 + 2 files changed, 70 insertions(+) create mode 100644 dbms/src/Functions/filesystem.cpp diff --git a/dbms/src/Functions/filesystem.cpp b/dbms/src/Functions/filesystem.cpp new file mode 100644 index 00000000000..9859b5ed2f2 --- /dev/null +++ b/dbms/src/Functions/filesystem.cpp @@ -0,0 +1,68 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +struct FilesystemAvailable +{ + static constexpr auto name = "filesystemAvailable"; + static boost::uintmax_t get(boost::filesystem::space_info & spaceinfo) { return spaceinfo.available; } +}; + +struct FilesystemFree +{ + static constexpr auto name = "filesystemFree"; + static boost::uintmax_t get(boost::filesystem::space_info & spaceinfo) { return spaceinfo.free; } +}; + +struct FilesystemCapacity +{ + static constexpr auto name = "filesystemCapacity"; + static boost::uintmax_t get(boost::filesystem::space_info & spaceinfo) { return spaceinfo.capacity; } +}; + +template +class FilesystemImpl : public IFunction +{ +public: + static constexpr auto name = Impl::name; + + static FunctionPtr create(const Context & context) + { + return std::make_shared>(boost::filesystem::space(context.getConfigRef().getString("path"))); + } + + explicit FilesystemImpl(boost::filesystem::space_info spaceinfo_) : spaceinfo(spaceinfo_) { } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool isDeterministic() const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = DataTypeUInt64().createColumnConst(input_rows_count, static_cast(Impl::get(spaceinfo))); + } + +private: + boost::filesystem::space_info spaceinfo; +}; + + +void registerFunctionFilesystem(FunctionFactory & factory) +{ + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); +} + +} diff --git a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp index 59c49a5c950..d985fb6bf97 100644 --- a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp @@ -41,6 +41,7 @@ void registerFunctionLowCardinalityIndices(FunctionFactory &); void registerFunctionLowCardinalityKeys(FunctionFactory &); void registerFunctionsIn(FunctionFactory &); void registerFunctionJoinGet(FunctionFactory &); +void registerFunctionFilesystem(FunctionFactory &); void registerFunctionsMiscellaneous(FunctionFactory & factory) { @@ -82,6 +83,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionLowCardinalityKeys(factory); registerFunctionsIn(factory); registerFunctionJoinGet(factory); + registerFunctionFilesystem(factory); } } From 84f1fe6e75a64ae4ede7c6b483328146e9339618 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Fri, 18 Jan 2019 19:07:48 +0300 Subject: [PATCH 26/71] Docapi 4479 merge tree new syntax translate (#4085) * Update of english version of descriprion of the table function `file`. * New syntax for ReplacingMergeTree. Some improvements in text. * Significantly change article about SummingMergeTree. Article is restructured, text is changed in many places of the document. New syntax for table creation is described. * Descriptions of AggregateFunction and AggregatingMergeTree are updated. Russian version. * New syntax for new syntax of CREATE TABLE * Added english docs on Aggregating, Replacing and SummingMergeTree. * CollapsingMergeTree docs. English version. * 1. Update of CollapsingMergeTree. 2. Minor changes in markup * Update aggregatefunction.md * Update aggregatefunction.md * Update aggregatefunction.md * Update aggregatingmergetree.md * GraphiteMergeTree docs update. New syntax for creation of Replicated* tables. Minor changes in *MergeTree tables creation syntax. * Markup fix * Markup and language fixes * Clarification in the CollapsingMergeTree article * DOCAPI-4821. Sync between ru and en versions of docs. * Fixed the ambiguity in geo functions description. * Example of JOIN in ru docs * Deleted misinforming example. * Fixed links to IN operators. * Updated the description of ALTER MODIFY. * [RU] Updated ALTER MODIFY description. * DOCAPI-4479. English changes are translated into russian. * DOCAPI-4479. Minor text and markup fixes. * DOCAPI-4479. Minor text edits. --- .../en/operations/server_settings/settings.md | 4 +- docs/en/operations/settings/settings.md | 2 +- .../table_engines/collapsingmergetree.md | 18 +- .../table_engines/graphitemergetree.md | 15 +- docs/en/operations/table_engines/mergetree.md | 2 +- .../operations/table_engines/replication.md | 10 +- .../ru/operations/server_settings/settings.md | 4 +- .../table_engines/collapsingmergetree.md | 229 ++++++++++++++++-- .../table_engines/graphitemergetree.md | 110 +++++++-- docs/ru/operations/table_engines/mergetree.md | 2 +- .../operations/table_engines/replication.md | 46 ++-- docs/ru/query_language/create.md | 66 ++--- 12 files changed, 381 insertions(+), 127 deletions(-) diff --git a/docs/en/operations/server_settings/settings.md b/docs/en/operations/server_settings/settings.md index 6cb2de77ba9..fe4330fafe4 100644 --- a/docs/en/operations/server_settings/settings.md +++ b/docs/en/operations/server_settings/settings.md @@ -163,7 +163,7 @@ You can configure multiple `` clauses. For instance, you can use this ``` -## graphite_rollup +## graphite_rollup {#server_settings-graphite_rollup} Settings for thinning data for Graphite. @@ -416,7 +416,7 @@ The value 0 means that you can delete all tables without any restrictions. ``` -## merge_tree +## merge_tree {#server_settings-merge_tree} Fine tuning for tables in the [ MergeTree](../../operations/table_engines/mergetree.md). diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 36a4d6bc135..22568872092 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -22,7 +22,7 @@ The possible values are: - `allow` — Allows the use of these types of subqueries. -## fallback_to_stale_replicas_for_distributed_queries +## fallback_to_stale_replicas_for_distributed_queries {#settings-fallback_to_stale_replicas_for_distributed_queries} Forces a query to an out-of-date replica if updated data is not available. See "[Replication](../../operations/table_engines/replication.md)". diff --git a/docs/en/operations/table_engines/collapsingmergetree.md b/docs/en/operations/table_engines/collapsingmergetree.md index b6aa9b4ce2c..eeedb82c347 100644 --- a/docs/en/operations/table_engines/collapsingmergetree.md +++ b/docs/en/operations/table_engines/collapsingmergetree.md @@ -2,7 +2,7 @@ The engine inherits from [MergeTree](mergetree.md) and adds the logic of rows collapsing to data parts merge algorithm. -`CollapsingMergeTree` asynchronously deletes (collapses) pairs of rows if all of the fields in a row are equivalent excepting the particular field `Sign` which can have `1` and `-1` values. Rows without a pair are kept. For more details see the [Collapsing](#collapsing) section of the document. +`CollapsingMergeTree` asynchronously deletes (collapses) pairs of rows if all of the fields in a row are equivalent excepting the particular field `Sign` which can have `1` and `-1` values. Rows without a pair are kept. For more details see the [Collapsing](#table_engine-collapsingmergetree-collapsing) section of the document. The engine may significantly reduce the volume of storage and increase efficiency of `SELECT` query as a consequence. @@ -31,7 +31,7 @@ For a description of query parameters, see [query description](../../query_langu **Query clauses** -When creating a `CollapsingMergeTree` table, the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table. +When creating a `CollapsingMergeTree` table, the same [query clauses](mergetree.md#table_engine-mergetree-creating-a-table) are required, as when creating a `MergeTree` table.
Deprecated Method for Creating a Table @@ -55,13 +55,13 @@ All of the parameters excepting `sign` have the same meaning as in `MergeTree`.
-## Collapsing +## Collapsing {#table_engine-collapsingmergetree-collapsing} ### Data Consider the situation where you need to save continually changing data for some object. It sounds logical to have one row for an object and update it at any change, but update operation is expensive and slow for DBMS because it requires rewriting of the data in the storage. If you need to write data quickly, update not acceptable, but you can write the changes of an object sequentially as follows. -Use the particular column `Sign` when writing row. If `Sign = 1` it means that the row is a state of an object, let's call it "state" row. If `Sign = -1` it means the cancellation of the state of an object with the same attributes, let's call it "cancel" row. +Use the particular column `Sign`. If `Sign = 1` it means that the row is a state of an object, let's call it "state" row. If `Sign = -1` it means the cancellation of the state of an object with the same attributes, let's call it "cancel" row. For example, we want to calculate how much pages users checked at some site and how long they were there. At some moment of time we write the following row with the state of user activity: @@ -95,15 +95,15 @@ As we need only the last state of user activity, the rows can be deleted collapsing the invalid (old) state of an object. `CollapsingMergeTree` does this while merging of the data parts. -Why we need 2 rows for each change read in the "Algorithm" paragraph. +Why we need 2 rows for each change read in the [Algorithm](#table_engine-collapsingmergetree-collapsing-algorithm) paragraph. **Peculiar properties of such approach** 1. The program that writes the data should remember the state of an object to be able to cancel it. "Cancel" string should be the copy of "state" string with the opposite `Sign`. It increases the initial size of storage but allows to write the data quickly. 2. Long growing arrays in columns reduce the efficiency of the engine due to load for writing. The more straightforward data, the higher efficiency. -3. `SELECT` results depend strongly on the consistency of object changes history. Be accurate when preparing data for inserting. You can get unpredictable results in inconsistent data, for example, negative values for non-negative metrics such as session depth. +3. The `SELECT` results depend strongly on the consistency of object changes history. Be accurate when preparing data for inserting. You can get unpredictable results in inconsistent data, for example, negative values for non-negative metrics such as session depth. -### Algorithm +### Algorithm {#table_engine-collapsingmergetree-collapsing-algorithm} When ClickHouse merges data parts, each group of consecutive rows with the same primary key is reduced to not more than two rows, one with `Sign = 1` ("state" row) and another with `Sign = -1` ("cancel" row). In other words, entries collapse. @@ -181,8 +181,8 @@ SELECT * FROM UAct ``` What do we see and where is collapsing? -With two `INSERT` queries, we created 2 data parts. The `SELECT` query was performed in 2 threads, and we got a random order of rows. -Collapsing not occurred because there was no merge of the data parts yet. ClickHouse merges data part in an unknown moment of time which we can not predict. + +With two `INSERT` queries, we created 2 data parts. The `SELECT` query was performed in 2 threads, and we got a random order of rows. Collapsing not occurred because there was no merge of the data parts yet. ClickHouse merges data part in an unknown moment of time which we can not predict. Thus we need aggregation: diff --git a/docs/en/operations/table_engines/graphitemergetree.md b/docs/en/operations/table_engines/graphitemergetree.md index fa15ab4daaf..5e30a67c645 100644 --- a/docs/en/operations/table_engines/graphitemergetree.md +++ b/docs/en/operations/table_engines/graphitemergetree.md @@ -1,7 +1,7 @@ # GraphiteMergeTree -This engine is designed for rollup (thinning and aggregating/averaging) [Graphite](http://graphite.readthedocs.io/en/latest/index.html) data. It may be helpful to developers who want to use ClickHouse as a data store for Graphite. +This engine is designed for thinning and aggregating/averaging (rollup) [Graphite](http://graphite.readthedocs.io/en/latest/index.html) data. It may be helpful to developers who want to use ClickHouse as a data store for Graphite. You can use any ClickHouse table engine to store the Graphite data if you don't need rollup, but if you need a rollup use `GraphiteMergeTree`. The engine reduces the volume of storage and increases the efficiency of queries from Graphite. @@ -29,9 +29,9 @@ For a description of request parameters, see [request description](../../query_l A table for the Graphite date should have the following columns: - Column with the metric name (Graphite sensor). Data type: `String`. -- Column with the time for measuring the metric. Data type: `DateTime`. +- Column with the time of measuring the metric. Data type: `DateTime`. - Column with the value of the metric. Data type: any numeric. -- Column with the version of the metric with the same name and time of measurement. Data type: any numeric. +- Column with the version of the metric. Data type: any numeric. ClickHouse saves the rows with the highest version or the last written if versions are the same. Other rows are deleted during the merge of data parts. @@ -43,7 +43,7 @@ The names of these columns should be set in the rollup configuration. **Query clauses** -When creating a `GraphiteMergeTree` table, the same [clauses](mergetree.md) are required, as when creating a `MergeTree` table. +When creating a `GraphiteMergeTree` table, the same [clauses](mergetree.md#table_engine-mergetree-creating-a-table) are required, as when creating a `MergeTree` table.
Deprecated Method for Creating a Table @@ -69,7 +69,7 @@ All of the parameters excepting `config_section` have the same meaning as in `Me ## Rollup configuration -The settings for rollup are defined by the [graphite_rollup](../server_settings/settings.md) parameter in the server configuration. The name of the parameter could be any. You can create several configurations and use them for different tables. +The settings for rollup are defined by the [graphite_rollup](../server_settings/settings.md#server_settings-graphite_rollup) parameter in the server configuration. The name of the parameter could be any. You can create several configurations and use them for different tables. Rollup configuration structure: @@ -102,10 +102,9 @@ Fields for `pattern` and `default` sections: The `required-columns`: - `path_column_name` — Column with the metric name (Graphite sensor). -- `time_column_name` — Column with the time for measuring the metric. +- `time_column_name` — Column with the time of measuring the metric. - `value_column_name` — Column with the value of the metric at the time set in `time_column_name`. -- `version_column_name` — Column with the version timestamp of the metric with the same name and time remains in the database. - +- `version_column_name` — Column with the version of the metric. Example of settings: diff --git a/docs/en/operations/table_engines/mergetree.md b/docs/en/operations/table_engines/mergetree.md index abac921f9df..ec2c05b786f 100644 --- a/docs/en/operations/table_engines/mergetree.md +++ b/docs/en/operations/table_engines/mergetree.md @@ -26,7 +26,7 @@ Main features: The [Merge](merge.md) engine does not belong to the `*MergeTree` family. -## Creating a Table +## Creating a Table {#table_engine-mergetree-creating-a-table} ``` CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] diff --git a/docs/en/operations/table_engines/replication.md b/docs/en/operations/table_engines/replication.md index 7a0c35acbd2..75b1866e01e 100644 --- a/docs/en/operations/table_engines/replication.md +++ b/docs/en/operations/table_engines/replication.md @@ -18,9 +18,9 @@ Compressed data for `INSERT` and `ALTER` queries is replicated (for more informa `CREATE`, `DROP`, `ATTACH`, `DETACH` and `RENAME` queries are executed on a single server and are not replicated: -- `The CREATE TABLE` query creates a new replicatable table on the server where the query is run. If this table already exists on other servers, it adds a new replica. -- `The DROP TABLE` query deletes the replica located on the server where the query is run. -- `The RENAME` query renames the table on one of the replicas. In other words, replicated tables can have different names on different replicas. +- The `CREATE TABLE` query creates a new replicatable table on the server where the query is run. If this table already exists on other servers, it adds a new replica. +- The `DROP TABLE` query deletes the replica located on the server where the query is run. +- The `RENAME` query renames the table on one of the replicas. In other words, replicated tables can have different names on different replicas. To use replication, set the addresses of the ZooKeeper cluster in the config file. Example: @@ -47,7 +47,7 @@ You can specify any existing ZooKeeper cluster and the system will use a directo If ZooKeeper isn't set in the config file, you can't create replicated tables, and any existing replicated tables will be read-only. -ZooKeeper is not used in `SELECT` queries because replication does not affect the performance of `SELECT` and queries run just as fast as they do for non-replicated tables. When querying distributed replicated tables, ClickHouse behavior is controlled by the settings [max_replica_delay_for_distributed_queries](../settings/settings.md#settings-max_replica_delay_for_distributed_queries) and [fallback_to_stale_replicas_for_distributed_queries](../settings/settings.md). +ZooKeeper is not used in `SELECT` queries because replication does not affect the performance of `SELECT` and queries run just as fast as they do for non-replicated tables. When querying distributed replicated tables, ClickHouse behavior is controlled by the settings [max_replica_delay_for_distributed_queries](../settings/settings.md#settings-max_replica_delay_for_distributed_queries) and [fallback_to_stale_replicas_for_distributed_queries](../settings/settings.md#settings-fallback_to_stale_replicas_for_distributed_queries). For each `INSERT` query, approximately ten entries are added to ZooKeeper through several transactions. (To be more precise, this is for each inserted block of data; an INSERT query contains one block or one block per `max_insert_block_size = 1048576` rows.) This leads to slightly longer latencies for `INSERT` compared to non-replicated tables. But if you follow the recommendations to insert data in batches of no more than one `INSERT` per second, it doesn't create any problems. The entire ClickHouse cluster used for coordinating one ZooKeeper cluster has a total of several hundred `INSERTs` per second. The throughput on data inserts (the number of rows per second) is just as high as for non-replicated data. @@ -59,7 +59,7 @@ By default, an INSERT query waits for confirmation of writing the data from only Each block of data is written atomically. The INSERT query is divided into blocks up to `max_insert_block_size = 1048576` rows. In other words, if the `INSERT` query has less than 1048576 rows, it is made atomically. -Data blocks are deduplicated. For multiple writes of the same data block (data blocks of the same size containing the same rows in the same order), the block is only written once. The reason for this is in case of network failures when the client application doesn't know if the data was written to the DB, so the `INSERT` query can simply be repeated. It doesn't matter which replica INSERTs were sent to with identical data. `INSERTs` are idempotent. Deduplication parameters are controlled by [merge_tree](../server_settings/settings.md) server settings. +Data blocks are deduplicated. For multiple writes of the same data block (data blocks of the same size containing the same rows in the same order), the block is only written once. The reason for this is in case of network failures when the client application doesn't know if the data was written to the DB, so the `INSERT` query can simply be repeated. It doesn't matter which replica INSERTs were sent to with identical data. `INSERTs` are idempotent. Deduplication parameters are controlled by [merge_tree](../server_settings/settings.md#server_settings-merge_tree) server settings. During replication, only the source data to insert is transferred over the network. Further data transformation (merging) is coordinated and performed on all the replicas in the same way. This minimizes network usage, which means that replication works well when replicas reside in different datacenters. (Note that duplicating data in different datacenters is the main goal of replication.) diff --git a/docs/ru/operations/server_settings/settings.md b/docs/ru/operations/server_settings/settings.md index 68b7c4becc0..75008f875d5 100644 --- a/docs/ru/operations/server_settings/settings.md +++ b/docs/ru/operations/server_settings/settings.md @@ -164,7 +164,7 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat ``` -## graphite_rollup +## graphite_rollup {#server_settings-graphite_rollup} Настройка прореживания данных для Graphite. @@ -416,7 +416,7 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat ``` -## merge_tree +## merge_tree {#server_settings-merge_tree} Тонкая настройка таблиц семейства [MergeTree](../../operations/table_engines/mergetree.md). diff --git a/docs/ru/operations/table_engines/collapsingmergetree.md b/docs/ru/operations/table_engines/collapsingmergetree.md index 9b28deb880a..cc938de2af3 100644 --- a/docs/ru/operations/table_engines/collapsingmergetree.md +++ b/docs/ru/operations/table_engines/collapsingmergetree.md @@ -1,37 +1,222 @@ # CollapsingMergeTree {#table_engine-collapsingmergetree} -*Движок достаточно специфичен для Яндекс.Метрики.* +Движок наследует функциональность от [MergeTree](mergetree.md) и добавляет в алгоритм слияния кусков данных логику сворачивания (удаления) строк. -Отличается от `MergeTree` тем, что позволяет автоматически удалять - "схлопывать" некоторые пары строк при слиянии. +`CollapsingMergeTree` асинхронно удаляет (сворачивает) пары строк, если все поля в строке эквивалентны, за исключением специального поля `Sign`, которое может принимать значения `1` и `-1`. Строки без пары сохраняются. Подробнее смотрите раздел [Сворачивание (удаление) строк](#table_engine-collapsingmergetree-collapsing). -В Яндекс.Метрике есть обычные логи (например, лог хитов) и логи изменений. Логи изменений используются, чтобы инкрементально считать статистику по постоянно меняющимся данным. Например - логи изменений визитов, логи изменений истории посетителей. Визиты в Яндекс.Метрике постоянно меняются - например, увеличивается количество хитов в визите. Изменением какого либо объекта будем называть пару (?старые значения, ?новые значения). Старые значения могут отсутствовать, если объект создался. Новые значения могут отсутствовать, если объект удалился. Если объект изменился, но был раньше и не удалился - присутствует оба значения. В лог изменений, для каждого изменения, пишется от одной до двух записей. Каждая запись содержит все те же атрибуты, что и сам объект, и ещё специальный атрибут, который позволяет отличить старые и новые значения. Видно, что при изменении объектов, в лог изменений лишь дописываются новые записи и не трогаются уже имеющиеся. +Движок может значительно уменьшить объем хранения и, как следствие, повысить эффективность запросов `SELECT`. -Лог изменений позволяет инкрементально считать почти любую статистику. Для этого надо учитывать "новые" строки с положительным знаком, и "старые" строки с отрицательным знаком. То есть, возможно инкрементально считать все статистики, алгебраическая структура которых содержит операцию взятия обратного элемента. Большинство статистик именно такие. Также удаётся посчитать "идемпотентные" статистики, например, количество уникальных посетителей, так как при изменении визитов, уникальные посетители не удаляются. +## Создание таблицы -Это - основная идея, благодаря которой Яндекс.Метрика работает в реальном времени. - -CollapsingMergeTree принимает дополнительный параметр - имя столбца типа Int8, содержащего "знак" строки. Пример: - -``` sql -CollapsingMergeTree(EventDate, (CounterID, EventDate, intHash32(UniqID), VisitID), 8192, Sign) +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = CollapsingMergeTree(sign) +[PARTITION BY expr] +[ORDER BY expr] +[SAMPLE BY expr] +[SETTINGS name=value, ...] ``` -Здесь `Sign` - столбец, содержащий -1 для "старых" значений и 1 для "новых" значений. +Подробности про `CREATE TABLE` смотрите в [описании запроса](../../query_language/create.md). -При слиянии, для каждой группы идущих подряд одинаковых значений первичного ключа (столбцов, по которым сортируются данные), остаётся не более одной строки со значением столбца sign_column = -1 ("отрицательной строки") и не более одной строки со значением столбца sign_column = 1 ("положительной строки"). То есть - производится схлопывание записей из лога изменений. +**Параметры CollapsingMergeTree** -Если количество положительных и отрицательных строк совпадает - то пишет первую отрицательную и последнюю положительную строку. -Если положительных на 1 больше, чем отрицательных - то пишет только последнюю положительную строку. -Если отрицательных на 1 больше, чем положительных - то пишет только первую отрицательную строку. -Иначе - логическая ошибка, и ни одна из таких строк не пишется. (Логическая ошибка может возникать, если случайно один кусок лога был вставлен более одного раза. Поэтому, об ошибке всего лишь пишется в лог сервера, и слияние продолжает работать.) +- `sign` — Имя столбца с типом строки: `1` — строка состояния, `-1` — строка отмены состояния. -Как видно, от схлопывания не должны меняться результаты расчётов статистик. -Изменения постепенно схлопываются так что в конце-концов, для почти каждого объекта, остаются лишь его последние значения. -По сравнению с MergeTree, движок CollapsingMergeTree позволяет в несколько раз уменьшить объём данных. + Тип данных столбца — `Int8`. -Существует несколько способов получения полностью "схлопнутых" данных из таблицы типа `CollapsingMergeTree`: +**Секции запроса** +При создании таблицы `CollapsingMergeTree` используются те же [секции](mergetree.md#table_engine-mergetree-creating-a-table) запроса, что и при создании таблицы `MergeTree`. -1. Написать запрос с GROUP BY и агрегатными функциями, учитывающими знак. Например, чтобы посчитать количество, надо вместо count() написать sum(Sign); чтобы посчитать сумму чего-либо, надо вместо sum(x) написать sum(Sign \* x) и т. п., а также добавить HAVING sum(Sign) `>` 0. Не все величины можно посчитать подобным образом. Например, агрегатные функции min, max не могут быть переписаны. -2. Если необходимо вынимать данные без агрегации (например, проверить наличие строк, самые новые значения которых удовлетворяют некоторым условиям), можно использовать модификатор FINAL для секции FROM. Это вариант существенно менее эффективен. +
Устаревший способ создания таблицы + +!!! attention + Не используйте этот способ в новых проектах и по возможности переведите старые проекты на способ описанный выше. + +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE [=] CollapsingMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, sign) +``` + +Все параметры, кроме `ver` имеют то же значение, что и в `MergeTree`. + +- `sign` — Имя столбца с типом строки: `1` — строка состояния, `-1` — строка отмены состояния. + + Тип данных столбца — `Int8`. + +
+ +## Сворачивание (удаление) строк {table_engine-collapsingmergetree-collapsing} + +### Данные + +Рассмотрим ситуацию, когда необходимо сохранять постоянно изменяющиеся данные для какого-либо объекта. Кажется логичным иметь одну строку для объекта и обновлять её при любом изменении, однако операция обновления является дорогостоящей и медленной для СУБД, поскольку требует перезаписи данных в хранилище. Если необходимо быстро записать данные, обновление не допустимо, но можно записать изменения объекта последовательно как описано ниже. + +Используйте специальный столбец `Sign`. Если `Sign = 1`, то это означает, что строка является состоянием объекта, назовём её строкой состояния. Если `Sign = -1`, то это означает отмену состояния объекта с теми же атрибутами, назовём её строкой отмены состояния. + +Например, мы хотим рассчитать, сколько страниц проверили пользователи на каком-то сайте и как долго они там находились. В какой-то момент времени мы пишем следующую строку с состоянием действий пользователя: + +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + +Через некоторое время мы регистрируем изменение активности пользователя и записываем его следующими двумя строками. + +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ -1 │ +│ 4324182021466249494 │ 6 │ 185 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + +Первая строка отменяет предыдущее состояние объекта (пользователя). Она должен повторять все поля отменённого состояния за исключением `Sign`. + +Вторая строка содержит текущее состояние. + +Поскольку нам нужно только последнее состояние активности пользователя, строки + +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ 1 │ +│ 4324182021466249494 │ 5 │ 146 │ -1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + +можно удалить, сворачивая (удаляя) устаревшее состояние объекта. `CollapsingMergeTree` выполняет это при слиянии кусков данных. + +Зачем нужны 2 строки для каждого изменения, читайте в параграфе [Алгоритм](#table_engine-collapsingmergetree-collapsing-algorithm). + +**Особенности подхода** + +1. Программа, которая записывает данные, должна помнить состояние объекта, чтобы иметь возможность отменить его. Строка отмены состояния должна быть копией предыдущей строки состояния с противоположным значением `Sign`. Это увеличивает начальный размер хранилища, но позволяет быстро записывать данные. +2. Длинные растущие массивы в Столбцах снижают эффективность работы движка за счёт нагрузки на запись. Чем проще данные, тем выше эффективность. +3. Результаты запроса `SELECT` сильно зависят от согласованности истории изменений объекта. Будьте точны при подготовке данных для вставки. Можно получить непредсказуемые результаты для несогласованных данных, например отрицательные значения для неотрицательных метрик, таких как глубина сеанса. + +### Алгоритм {#table_engine-collapsingmergetree-collapsing-algorithm} + +Когда ClickHouse объединяет куски данных, каждая группа последовательных строк с одним и тем же первичным ключом уменьшается до не более чем двух строк, одна из которых имеет `Sign = 1` (строка состояния), а другая строка с `Sign = -1` (строка отмены состояния). Другими словами, записи сворачиваются. + +Для каждого результирующего куска данных ClickHouse сохраняет: + + 1. Первую строку отмены состояния и последнюю строку состояния, если количество строк обоих видов совпадает. + + 1. Последнюю строку состояния, если строк состояния на одну больше, чем строк отмены состояния. + + 1. Первую строку отмены состояния, если их на одну больше, чем строк состояния. + + 1. Ни в одну из строк во всех остальных случаях. + + Слияние продолжается, но ClickHouse рассматривает эту ситуацию как логическую ошибку и записывает её в журнал сервера. Эта ошибка может возникать, если одни и те же данные вставлялись несколько раз. + +Как видно, от сворачивания не должны меняться результаты расчётов статистик. +Изменения постепенно сворачиваются так, что остаются лишь последнее состояние почти каждого объекта. + +Столбец `Sign` необходим, поскольку алгоритм слияния не гарантирует, что все строки с одинаковым первичным ключом будут находиться в одном результирующем куске данных и даже на одном физическом сервере. ClickHouse выполняет запросы `SELECT` несколькими потоками, и он не может предсказать порядок строк в результате. Если необходимо получить полностью свёрнутые данные из таблицы `CollapsingMergeTree`, то необходимо агрегирование. + +Для завершения свертывания добавьте в запрос секцию`GROUP BY` и агрегатные функции, которые учитывают знак. Например, для расчета количества используйте `sum(Sign)` вместо`count()`. Чтобы вычислить сумму чего-либо, используйте `sum(Sign * x)` вместо`sum(х)`, и так далее, а также добавьте `HAVING sum(Sign) > 0` . + +Таким образом можно вычислять агрегации `count`, `sum` и `avg`. Если объект имеет хотя бы одно не свёрнутое состояние, то может быть вычислена агрегация `uniq`. Агрегации `min` и `max` невозможно вычислить, поскольку `CollapsingMergeTree` не сохраняет историю значений свернутых состояний. + +Если необходимо выбирать данные без агрегации (например, проверить наличие строк, последние значения которых удовлетворяют некоторым условиям), можно использовать модификатор `FINAL` для секции `FROM`. Это вариант существенно менее эффективен. + +## Пример использования + +Example data: + +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ 1 │ +│ 4324182021466249494 │ 5 │ 146 │ -1 │ +│ 4324182021466249494 │ 6 │ 185 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + +Создание таблицы: + +```sql +CREATE TABLE UAct +( + UserID UInt64, + PageViews UInt8, + Duration UInt8, + Sign Int8 +) +ENGINE = CollapsingMergeTree(Sign) +ORDER BY UserID +``` + +Insertion of the data: + +```sql +INSERT INTO UAct VALUES (4324182021466249494, 5, 146, 1) +``` + +```sql +INSERT INTO UAct VALUES (4324182021466249494, 5, 146, -1),(4324182021466249494, 6, 185, 1) +``` + +Мы используем два запроса `INSERT` для создания двух различных кусков данных. Если вставить данные одним запросом, ClickHouse создаёт один кусок данных и никогда не будет выполнять слияние. + +Получение данных: + +``` +SELECT * FROM UAct +``` + +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ -1 │ +│ 4324182021466249494 │ 6 │ 185 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + +Что мы видим и где сворачивание? + +Двумя запросами `INSERT`, мы создали два куска данных. Запрос `SELECT` был выполнен в 2 потока, и мы получили случайный порядок строк. Сворачивание не произошло, так как слияние кусков данных еще не произошло. ClickHouse объединяет куски данных в неизвестный момент времени, который мы не можем предсказать. + +Таким образом, нам нужна агрегация: + +```sql +SELECT + UserID, + sum(PageViews * Sign) AS PageViews, + sum(Duration * Sign) AS Duration +FROM UAct +GROUP BY UserID +HAVING sum(Sign) > 0 +``` + +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┐ +│ 4324182021466249494 │ 6 │ 185 │ +└─────────────────────┴───────────┴──────────┘ +``` + +Если нам не нужна агрегация, но мы хотим принудительно выполнить свёртку данных, можно использовать модификатор `FINAL` для секции `FROM`. + +```sql +SELECT * FROM UAct FINAL +``` + +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 6 │ 185 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + +Такой способ выбора данных очень неэффективен. Не используйте его для больших таблиц. [Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/collapsingmergetree/) diff --git a/docs/ru/operations/table_engines/graphitemergetree.md b/docs/ru/operations/table_engines/graphitemergetree.md index 52e0d98df81..816fddff7f2 100644 --- a/docs/ru/operations/table_engines/graphitemergetree.md +++ b/docs/ru/operations/table_engines/graphitemergetree.md @@ -1,58 +1,122 @@ - # GraphiteMergeTree -Движок предназначен для rollup (прореживания и агрегирования/усреднения) данных [Graphite](http://graphite.readthedocs.io/en/latest/index.html). Он может быть интересен разработчикам, которые хотят использовать ClickHouse как хранилище данных для Graphite. +Движок предназначен для прореживания и агрегирования/усреднения (rollup) данных [Graphite](http://graphite.readthedocs.io/en/latest/index.html). Он может быть интересен разработчикам, которые хотят использовать ClickHouse как хранилище данных для Graphite. -Graphite хранит в ClickHouse полные данные, а получать их может следующими способами: +Если rollup не требуется, то для хранения данных Graphite можно использовать любой движок таблиц ClickHouse, в противном случае используйте `GraphiteMergeTree`. Движок уменьшает объем хранения и повышает эффективность запросов от Graphite. -- Без прореживания. +Движок наследует свойства от [MergeTree](mergetree.md). - Используется движок [MergeTree](mergetree.md). +## Создание таблицы -- С прореживанием. +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + Path String, + Time DateTime, + Value , + Version + ... +) ENGINE = GraphiteMergeTree(config_section) +[PARTITION BY expr] +[ORDER BY expr] +[SAMPLE BY expr] +[SETTINGS name=value, ...] +``` - Используется движок `GraphiteMergeTree`. +Описание параметров запроса смотрите в [описании запроса](../../query_language/create.md). -Движок наследует свойства MergeTree. Настройки прореживания данных задаются параметром [graphite_rollup](../server_settings/settings.md) в конфигурации сервера . +Таблица для данных Graphite должна содержать следующие столбцы: -## Использование движка +- Колонка с названием метрики (Graphite sensor). Тип данных: `String`. -Таблица с данными Graphite должна содержать как минимум следующие поля: +- Столбец со временем измерения метрики. Тип данных `DateTime`. -- `Path` - имя метрики (сенсора Graphite). -- `Time` - время измерения. -- `Value` - значение метрики в момент времени Time. -- `Version` - настройка, которая определяет какое значение метрики с одинаковыми Path и Time останется в базе. +- Столбец со значением метрики. Тип данных: любой числовой. -Шаблон правил rollup: +- Столбец с версией метрики. Тип данных: любой числовой. + + ClickHouse сохраняет строки с последней версией или последнюю записанную строку, если версии совпадают. Другие строки удаляются при слиянии кусков данных. + +Имена этих столбцов должны быть заданы в конфигурации rollup. + +**Параметры GraphiteMergeTree** + +- `config_section` — имя раздела в конфигурационном файле, в котором находятся правила rollup. + +**Секции запроса** + +При создании таблицы `GraphiteMergeTree` используются те же [секции](mergetree.md#table_engine-mergetree-creating-a-table) запроса, что при создании таблицы `MergeTree`. + +
Устаревший способ создания таблицы + +!!! attention +Не используйте этот способ в новых проектах и по возможности переведите старые проекты на способ описанный выше. + +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + EventDate Date, + Path String, + Time DateTime, + Value , + Version + ... +) ENGINE [=] GraphiteMergeTree(date-column [, sampling_expression], (primary, key), index_granularity, config_section) +``` + +Все параметры, кроме `config_section` имеют то же значение, что в `MergeTree`. + +- `config_section` — имя раздела в конфигурационном файле, в котором находятся правила rollup. + +
+ +## Конфигурация rollup + +Настройки для прореживания данных задаются параметром [graphite_rollup](../server_settings/settings.md#server_settings-graphite_rollup) Имя параметра может быть любым. Можно создать несколько конфигураций и использовать их для разных таблиц. + +Структура конфигурации rollup: ``` +required-columns pattern regexp function - age -> precision + age + precision ... pattern ... default function - age -> precision + age + precision ... ``` -При обработке записи ClickHouse проверит правила в секции `pattern`. Если имя метрики соответствует шаблону `regexp`, то применяются правила из `pattern`, в противном случае из `default`. +При обработке строки ClickHouse проверяет правила в разделе `pattern`. Если имя метрики соответствует шаблону `regexp`, то применяются правила из раздела `pattern`, в противном случае из раздела `default`. -Поля шаблона правил. +Правила определяются с помощью полей `function` и `age + precision`. -- `age` - Минимальный возраст данных в секундах. -- `function` - Имя агрегирующей функции, которую следует применить к данным, чей возраст оказался в интервале `[age, age + precision]`. -- `precision` - Точность определения возраста данных в секундах. -- `regexp` - Шаблон имени метрики. +Поля для разделов `pattenrn` и `default`: + +- `regexp` – шаблон имени метрики. +- `age` – минимальный возраст данных в секундах. +- `precision` – точность определения возраста данных в секундах. +- `function` – имя агрегирующей функции, которую следует применить к данным, чей возраст оказался в интервале `[age, age + precision]`. + +`required-columns`: + +- `path_column_name` — колонка с названием метрики (Graphite sensor). +- `time_column_name` — столбец со временем измерения метрики. +- `value_column_name` — столбец со значением метрики в момент времени, установленный в `time_column_name`. +- `version_column_name` — столбец с версией метрики. Пример настройки: ```xml + Path + Time + Value + Version click_cost any diff --git a/docs/ru/operations/table_engines/mergetree.md b/docs/ru/operations/table_engines/mergetree.md index f1f65b94442..d0410ccea58 100644 --- a/docs/ru/operations/table_engines/mergetree.md +++ b/docs/ru/operations/table_engines/mergetree.md @@ -24,7 +24,7 @@ При необходимости можно задать способ сэмплирования данных в таблице. -## Создание таблицы +## Создание таблицы {#table_engine-mergetree-creating-a-table} ```sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] diff --git a/docs/ru/operations/table_engines/replication.md b/docs/ru/operations/table_engines/replication.md index d80ef509986..fe61e3a232c 100644 --- a/docs/ru/operations/table_engines/replication.md +++ b/docs/ru/operations/table_engines/replication.md @@ -1,4 +1,4 @@ -# Репликация данных {#table_engines-replication} +# Репликация данных {#table_engines-репликация} Репликация поддерживается только для таблиц семейства MergeTree: @@ -7,6 +7,7 @@ - ReplicatedReplacingMergeTree - ReplicatedAggregatingMergeTree - ReplicatedCollapsingMergeTree +- ReplicatedVersionedCollapsingMergeTree - ReplicatedGraphiteMergeTree Репликация работает на уровне отдельных таблиц, а не всего сервера. То есть, на сервере могут быть расположены одновременно реплицируемые и не реплицируемые таблицы. @@ -15,11 +16,11 @@ Реплицируются сжатые данные запросов `INSERT`, `ALTER` (см. подробности в описании запроса [ALTER](../../query_language/alter.md#query_language_queries_alter)). -Запросы `CREATE`, `DROP`, `ATTACH`, `DETACH`, `RENAME` выполняются на одном сервере и не реплицируются: +Запросы `CREATE`, `DROP`, `ATTACH`, `DETACH` и `RENAME` выполняются на одном сервере и не реплицируются: -- `CREATE TABLE` создаёт новую реплицируемую таблицу на том сервере, где выполняется запрос, а если на других серверах такая таблица уже есть - добавляет новую реплику. +- Запрос `CREATE TABLE` создаёт новую реплицируемую таблицу на том сервере, где его выполнили. Если таблица уже существует на других серверах, запрос добавляет новую реплику. - `DROP TABLE` удаляет реплику, расположенную на том сервере, где выполняется запрос. -- `RENAME` переименовывает таблицу на одной из реплик - то есть, реплицируемые таблицы на разных репликах могут называться по разному. +- Запрос `RENAME` переименовывает таблицу на одной реплик. Другими словами, реплицируемые таблицы на разных репликах могут называться по-разному. Чтобы использовать репликацию, укажите в конфигурационном файле адреса ZooKeeper кластера. Пример: @@ -46,19 +47,19 @@ Если в конфигурационном файле не настроен ZooKeeper, то вы не сможете создать реплицируемые таблицы, а уже имеющиеся реплицируемые таблицы будут доступны в режиме только на чтение. -При запросах `SELECT`, ZooKeeper не используется, т.е. репликация не влияет на производительность `SELECT` и запросы работают так же быстро, как и для нереплицируемых таблиц. При запросах к распределенным реплицированным таблицам поведение ClickHouse регулируется настройками [max_replica_delay_for_distributed_queries](../settings/settings.md#settings_settings_max_replica_delay_for_distributed_queries) и [fallback_to_stale_replicas_for_distributed_queries](../settings/settings.md). +При запросах `SELECT`, ZooKeeper не используется, т.е. репликация не влияет на производительность `SELECT` и запросы работают так же быстро, как и для нереплицируемых таблиц. При запросах к распределенным реплицированным таблицам поведение ClickHouse регулируется настройками [max_replica_delay_for_distributed_queries](../settings/settings.md#settings-max_replica_delay_for_distributed_queries) and [fallback_to_stale_replicas_for_distributed_queries](../settings/settings.md). -При каждом запросе `INSERT` (точнее, на каждый вставляемый блок данных; запрос INSERT содержит один блок, или по блоку на каждые `max_insert_block_size = 1048576` строк), делается около десятка записей в ZooKeeper в рамках нескольких транзакций. Это приводит к некоторому увеличению задержек при `INSERT`, по сравнению с нереплицируемыми таблицами. Но если придерживаться обычных рекомендаций - вставлять данные пачками не более одного `INSERT` в секунду, то это не составляет проблем. На всём кластере ClickHouse, использующим для координации один кластер ZooKeeper, может быть в совокупности несколько сотен `INSERT` в секунду. Пропускная способность при вставке данных (количество строчек в секунду) такая же высокая, как для нереплицируемых таблиц. +При каждом запросе `INSERT`, делается около десятка записей в ZooKeeper в рамках нескольких транзакций. (Чтобы быть более точным, это для каждого вставленного блока данных; запрос INSERT содержит один блок или один блок на `max_insert_block_size = 1048576` строк.) Это приводит к некоторому увеличению задержек при `INSERT`, по сравнению с нереплицируемыми таблицами. Но если придерживаться обычных рекомендаций - вставлять данные пачками не более одного `INSERT` в секунду, то это не составляет проблем. На всём кластере ClickHouse, использующим для координации один кластер ZooKeeper, может быть в совокупности несколько сотен `INSERT` в секунду. Пропускная способность при вставке данных (количество строчек в секунду) такая же высокая, как для нереплицируемых таблиц. Для очень больших кластеров, можно использовать разные кластеры ZooKeeper для разных шардов. Впрочем, на кластере Яндекс.Метрики (примерно 300 серверов) такой необходимости не возникает. -Репликация асинхронная, мульти-мастер. Запросы `INSERT` (а также `ALTER`) можно отправлять на любой доступный сервер. Данные вставятся на сервер, где выполнен запрос, а затем скопируются на остальные серверы. В связи с асинхронностью, только что вставленные данные появляются на остальных репликах с небольшой задержкой. Если часть реплик недоступна, данные на них запишутся тогда, когда они станут доступны. Если реплика доступна, то задержка составляет столько времени, сколько требуется для передачи блока сжатых данных по сети. +Репликация асинхронная, мульти-мастер. Запросы `INSERT` и `ALTER` можно направлять на любой доступный сервер. Данные вставятся на сервер, где выполнен запрос, а затем скопируются на остальные серверы. В связи с асинхронностью, только что вставленные данные появляются на остальных репликах с небольшой задержкой. Если часть реплик недоступна, данные на них запишутся тогда, когда они станут доступны. Если реплика доступна, то задержка составляет столько времени, сколько требуется для передачи блока сжатых данных по сети. -По умолчанию, запрос INSERT ждёт подтверждения записи только от одной реплики. Если данные были успешно записаны только на одну реплику, и сервер с этой репликой перестал существовать, то записанные данные будут потеряны. Вы можете включить подтверждение записи от нескольких реплик, используя настройку [insert_quorum](../settings/settings.md). +По умолчанию, запрос INSERT ждёт подтверждения записи только от одной реплики. Если данные были успешно записаны только на одну реплику, и сервер с этой репликой перестал существовать, то записанные данные будут потеряны. Вы можете включить подтверждение записи от нескольких реплик, используя настройку `insert_quorum`. Каждый блок данных записывается атомарно. Запрос INSERT разбивается на блоки данных размером до `max_insert_block_size = 1048576` строк. То есть, если в запросе `INSERT` менее 1048576 строк, то он делается атомарно. -Блоки данных дедуплицируются. При многократной записи одного и того же блока данных (блоков данных одинакового размера, содержащих одни и те же строчки в одном и том же порядке), блок будет записан только один раз. Это сделано для того, чтобы в случае сбоя в сети, когда клиентское приложение не может понять, были ли данные записаны в БД, можно было просто повторить запрос `INSERT`. При этом не имеет значения, на какую реплику будут отправлены INSERT-ы с одинаковыми данными. То есть, обеспечивается идемпотентность `INSERT`. Параметры дедупликации регулируются настройками сервера [merge_tree](../server_settings/settings.md). +Блоки данных дедуплицируются. При многократной записи одного и того же блока данных (блоков данных одинакового размера, содержащих одни и те же строчки в одном и том же порядке), блок будет записан только один раз. Это сделано для того, чтобы в случае сбоя в сети, когда клиентское приложение не может понять, были ли данные записаны в БД, можно было просто повторить запрос `INSERT`. При этом не имеет значения, на какую реплику будут отправлены INSERT-ы с одинаковыми данными. Запрос `INSERT` идемпотентный. Параметры дедуплицирования регулируются настройками сервера [merge_tree](../server_settings/settings.md#server_settings-merge_tree) При репликации, по сети передаются только исходные вставляемые данные. Дальнейшие преобразования данных (слияния) координируются и делаются на всех репликах одинаковым образом. За счёт этого минимизируется использование сети, и благодаря этому, репликация хорошо работает при расположении реплик в разных датацентрах. (Стоит заметить, что дублирование данных в разных датацентрах, по сути, является основной задачей репликации). @@ -66,12 +67,14 @@ Система следит за синхронностью данных на репликах и умеет восстанавливаться после сбоя. Восстановление после сбоя автоматическое (в случае небольших различий в данных) или полуавтоматическое (когда данные отличаются слишком сильно, что может свидетельствовать об ошибке конфигурации). - ## Создание реплицируемых таблиц В начало имени движка таблицы добавляется `Replicated`. Например, `ReplicatedMergeTree`. -Также добавляются два параметра в начало списка параметров - путь к таблице в ZooKeeper, имя реплики в ZooKeeper. +**Параметры Replicated\*MergeTree** + +- `zoo_path` — путь к таблице в ZooKeeper. +- `replica_name` — имя реплики в ZooKeeper. Пример: @@ -100,7 +103,7 @@ CREATE TABLE table_name
-Как видно в примере, эти параметры могут содержать подстановки в фигурных скобках. Подставляемые значения достаются из конфигурационного файла, из секции macros. Пример: +Как видно в примере, эти параметры могут содержать подстановки в фигурных скобках. Подставляемые значения достаются из конфигурационного файла, из секции `macros`. Пример: ```xml @@ -113,13 +116,13 @@ CREATE TABLE table_name Путь к таблице в ZooKeeper должен быть разным для каждой реплицируемой таблицы. В том числе, для таблиц на разных шардах, должны быть разные пути. В данном случае, путь состоит из следующих частей: -`/clickhouse/tables/` - общий префикс. Рекомендуется использовать именно его. +`/clickhouse/tables/` — общий префикс. Рекомендуется использовать именно его. -`{layer}-{shard}` - идентификатор шарда. В данном примере он состоит из двух частей, так как на кластере Яндекс.Метрики используется двухуровневое шардирование. Для большинства задач, оставьте только подстановку {shard}, которая будет раскрываться в идентификатор шарда. +`{layer}-{shard}` — идентификатор шарда. В данном примере он состоит из двух частей, так как на кластере Яндекс.Метрики используется двухуровневое шардирование. Для большинства задач, оставьте только подстановку {shard}, которая будет раскрываться в идентификатор шарда. `hits` - имя узла для таблицы в ZooKeeper. Разумно делать его таким же, как имя таблицы. Оно указывается явно, так как, в отличие от имени таблицы, оно не меняется после запроса RENAME. -Имя реплики - то, что идентифицирует разные реплики одной и той же таблицы. Можно использовать для него имя сервера, как показано в примере. Впрочем, достаточно, чтобы имя было уникально лишь в пределах каждого шарда. +Имя реплики — то, что идентифицирует разные реплики одной и той же таблицы. Можно использовать для него имя сервера, как показано в примере. Впрочем, достаточно, чтобы имя было уникально лишь в пределах каждого шарда. Можно не использовать подстановки, а указать соответствующие параметры явно. Это может быть удобным для тестирования и при настройке маленьких кластеров. Однако в этом случае нельзя пользоваться распределенными DDL-запросами (`ON CLUSTER`). @@ -129,7 +132,7 @@ CREATE TABLE table_name Если вы добавляете новую реплику после того, как таблица на других репликах уже содержит некоторые данные, то после выполнения запроса, данные на новую реплику будут скачаны с других реплик. То есть, новая реплика синхронизирует себя с остальными. -Для удаления реплики, выполните запрос `DROP TABLE`. При этом, удаляется только одна реплика - расположенная на том сервере, где вы выполняете запрос. +Для удаления реплики, выполните запрос `DROP TABLE`. При этом, удаляется только одна реплика — расположенная на том сервере, где вы выполняете запрос. ## Восстановление после сбоя @@ -143,9 +146,9 @@ CREATE TABLE table_name Стоит заметить, что ClickHouse не делает самостоятельно никаких деструктивных действий типа автоматического удаления большого количества данных. -При старте сервера (или создании новой сессии с ZooKeeper), проверяется только количество и размеры всех файлов. Если у файлов совпадают размеры, но изменены байты где-то посередине, то это обнаруживается не сразу, а только при попытке их прочитать при каком-либо запросе `SELECT` - запрос кинет исключение о несоответствующей чексумме или размере сжатого блока. В этом случае, куски данных добавляются в очередь на проверку, и при необходимости, скачиваются с реплик. +При старте сервера (или создании новой сессии с ZooKeeper), проверяется только количество и размеры всех файлов. Если у файлов совпадают размеры, но изменены байты где-то посередине, то это обнаруживается не сразу, а только при попытке их прочитать при каком-либо запросе `SELECT`. Запрос кинет исключение о несоответствующей чексумме или размере сжатого блока. В этом случае, куски данных добавляются в очередь на проверку, и при необходимости, скачиваются с реплик. -Если обнаруживается, что локальный набор данных слишком сильно отличается от ожидаемого, то срабатывает защитный механизм - сервер сообщает об этом в лог и отказывается запускаться. Это сделано, так как такой случай может свидетельствовать об ошибке конфигурации - например, если реплика одного шарда была случайно сконфигурирована, как реплика другого шарда. Тем не менее, пороги защитного механизма поставлены довольно низкими, и такая ситуация может возникнуть и при обычном восстановлении после сбоя. В этом случае, восстановление делается полуавтоматически - "по кнопке". +Если обнаруживается, что локальный набор данных слишком сильно отличается от ожидаемого, то срабатывает защитный механизм. Сервер сообщает об этом в лог и отказывается запускаться. Это сделано, так как такой случай может свидетельствовать об ошибке конфигурации - например, если реплика одного шарда была случайно сконфигурирована, как реплика другого шарда. Тем не менее, пороги защитного механизма поставлены довольно низкими, и такая ситуация может возникнуть и при обычном восстановлении после сбоя. В этом случае, восстановление делается полуавтоматически - "по кнопке". Для запуска восстановления, создайте в ZooKeeper узел `/path_to_table/replica_name/flags/force_restore_data` с любым содержимым или выполните команду для восстановления всех реплицируемых таблиц: @@ -166,11 +169,10 @@ sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data Затем запустите сервер (перезапустите, если уже запущен). Данные будут скачаны с реплик. -В качестве альтернативного варианта восстановления, вы можете удалить из ZooKeeper информацию о потерянной реплике - `/path_to_table/replica_name`, и затем создать реплику заново, как написано в разделе "[Создание реплицируемых таблиц](#sozdanie-replitsiruemykh-tablits)". +В качестве альтернативного варианта восстановления, вы можете удалить из ZooKeeper информацию о потерянной реплике (`/path_to_table/replica_name`), и затем создать реплику заново, как написано в разделе [Создание реплицированных таблиц](#creating-replicated-tables) . Отсутствует ограничение на использование сетевой полосы при восстановлении. Имейте это ввиду, если восстанавливаете сразу много реплик. - ## Преобразование из MergeTree в ReplicatedMergeTree Здесь и далее, под `MergeTree` подразумеваются все движки таблиц семейства `MergeTree`, так же для `ReplicatedMergeTree`. @@ -189,8 +191,8 @@ sudo -u clickhouse touch /var/lib/clickhouse/flags/force_restore_data Если вы хотите избавиться от таблицы `ReplicatedMergeTree`, не запуская сервер, то -- удалите соответствующий файл `.sql` в директории с метаданными (`/var/lib/clickhouse/metadata/`); -- удалите соответствующий путь в ZooKeeper (`/path_to_table/replica_name`); +- удалите соответствующий файл `.sql` в директории с метаданными (`/var/lib/clickhouse/metadata/`); +- удалите соответствующий путь в ZooKeeper (`/path_to_table/replica_name`); После этого, вы можете запустить сервер, создать таблицу типа `MergeTree`, перенести данные в её директорию, и перезапустить сервер. diff --git a/docs/ru/query_language/create.md b/docs/ru/query_language/create.md index 09b0765c8f3..74269ad4823 100644 --- a/docs/ru/query_language/create.md +++ b/docs/ru/query_language/create.md @@ -1,20 +1,20 @@ ## CREATE DATABASE + Создание базы данных db_name -``` sql +```sql CREATE DATABASE [IF NOT EXISTS] db_name ``` `База данных` - это просто директория для таблиц. Если написано `IF NOT EXISTS`, то запрос не будет возвращать ошибку, если база данных уже существует. - - ## CREATE TABLE + Запрос `CREATE TABLE` может иметь несколько форм. -``` sql -CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ( name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [compression_codec], name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [compression_codec], @@ -28,14 +28,14 @@ CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster] Описание столбца, это `name type`, в простейшем случае. Пример: `RegionID UInt32`. Также могут быть указаны выражения для значений по умолчанию - смотрите ниже. -``` sql -CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name AS [db2.]name2 [ENGINE = engine] +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name AS [db2.]name2 [ENGINE = engine] ``` Создаёт таблицу с такой же структурой, как другая таблица. Можно указать другой движок для таблицы. Если движок не указан, то будет выбран такой же движок, как у таблицы `db2.name2`. -``` sql -CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name ENGINE = engine AS SELECT ... +```sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = engine AS SELECT ... ``` Создаёт таблицу со структурой, как результат запроса `SELECT`, с движком engine, и заполняет её данными из SELECT-а. @@ -46,7 +46,6 @@ CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name ENGINE = engine AS SELECT ... ### Значения по умолчанию - В описании столбца, может быть указано выражение для значения по умолчанию, одного из следующих видов: `DEFAULT expr`, `MATERIALIZED expr`, `ALIAS expr`. Пример: `URLDomain String DEFAULT domain(URL)`. @@ -67,7 +66,7 @@ CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name ENGINE = engine AS SELECT ... Материализованное выражение. Такой столбец не может быть указан при INSERT, то есть, он всегда вычисляется. При INSERT без указания списка столбцов, такие столбцы не рассматриваются. -Также этот столбец не подставляется при использовании звёздочки в запросе SELECT - чтобы сохранить инвариант, что дамп, полученный путём `SELECT *`, можно вставить обратно в таблицу INSERT-ом без указания списка столбцов. +Также этот столбец не подставляется при использовании звёздочки в запросе SELECT. Это необходимо, чтобы сохранить инвариант, что дамп, полученный путём `SELECT *`, можно вставить обратно в таблицу INSERT-ом без указания списка столбцов. `ALIAS expr` @@ -88,58 +87,64 @@ CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db.]name ENGINE = engine AS SELECT ... ### Временные таблицы -Во всех случаях, если указано `TEMPORARY`, то будет создана временная таблица. Временные таблицы обладают следующими особенностями: +ClickHouse поддерживает временные таблицы со следующими характеристиками: + - временные таблицы исчезают после завершения сессии; в том числе, при обрыве соединения; -- временная таблица создаётся с движком Memory; все остальные движки таблиц не поддерживаются; -- для временной таблицы нет возможности указать БД: она создаётся вне баз данных; +- Временная таблица использует только модуль памяти. +- Невозможно указать базу данных для временной таблицы. Временные таблицы создается вне баз данных. - если временная таблица имеет то же имя, что и некоторая другая, то, при упоминании в запросе без указания БД, будет использована временная таблица; - при распределённой обработке запроса, используемые в запросе временные таблицы, передаются на удалённые серверы. +Чтобы создать временную таблицу, используйте следующий синтаксис: + +```sql +CREATE TEMPORARY TABLE [IF NOT EXISTS] table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) +``` + В большинстве случаев, временные таблицы создаются не вручную, а при использовании внешних данных для запроса, или при распределённом `(GLOBAL) IN`. Подробнее см. соответствующие разделы -Распределенные DDL запросы (секция ON CLUSTER) ----------------------------------------------- +## Распределенные DDL запросы (секция ON CLUSTER) Запросы `CREATE`, `DROP`, `ALTER`, `RENAME` поддерживают возможность распределенного выполнения на кластере. -Например, следующий запрос создает `Distributed`-таблицу `all_hits` на каждом хосте кластера `cluster`: +Например, следующий запрос создает распределенную (Distributed) таблицу `all_hits` на каждом хосте в `cluster`: -``` sql +```sql CREATE TABLE IF NOT EXISTS all_hits ON CLUSTER cluster (p Date, i Int32) ENGINE = Distributed(cluster, default, hits) ``` -Для корректного выполнения таких запросов необходимо на каждом хосте иметь одинаковое определение кластера (для упрощения синхронизации конфигов можете использовать подстановки из ZooKeeper), также необходимо подключение к ZooKeeper серверам. -Локальная версия запроса в конечном итоге будет выполнена на каждом хосте кластера, даже если некоторые хосты в данный момент не доступны, гарантируется упорядоченность выполнения запросов в рамках одного хоста. -Пока не поддерживаются `ALTER`-запросы для реплицированных таблиц. +Для корректного выполнения таких запросов необходимо на каждом хосте иметь одинаковое определение кластера (для упрощения синхронизации конфигов можете использовать подстановки из ZooKeeper). Также необходимо подключение к ZooKeeper серверам. +Локальная версия запроса в конечном итоге будет выполнена на каждом хосте кластера, даже если некоторые хосты в данный момент не доступны. Гарантируется упорядоченность выполнения запросов в рамках одного хоста. Для реплицированных таблиц не поддерживаются запросы `ALTER`. ## CREATE VIEW -``` sql -CREATE [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]name [TO[db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... +```sql +CREATE [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]table_name [TO[db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... ``` Создаёт представление. Представления бывают двух видов - обычные и материализованные (MATERIALIZED). -При создании материализованного представления, нужно обязательно указать ENGINE - движок таблицы для хранения данных. - -Материализованное представление работает следующим образом: при вставлении данных в таблицу, указанную в SELECT, часть вставленных данных конвертируется запросом, а результат вставляется в представление. - Обычные представления не хранят никаких данных, а всего лишь производят чтение из другой таблицы. То есть, обычное представление - не более чем сохранённый запрос. При чтении из представления, этот сохранённый запрос, используется в качестве подзапроса в секции FROM. Для примера, пусть вы создали представление: -``` sql +```sql CREATE VIEW view AS SELECT ... ``` и написали запрос: -``` sql +```sql SELECT a, b, c FROM view ``` Этот запрос полностью эквивалентен использованию подзапроса: -``` sql +```sql SELECT a, b, c FROM (SELECT ...) ``` @@ -159,5 +164,4 @@ SELECT a, b, c FROM (SELECT ...) Отсутствует отдельный запрос для удаления представлений. Чтобы удалить представление, следует использовать `DROP TABLE`. - [Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/create/) From 66bd80703e6c3e3475218c79c5e8c5d15b8c0fe5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 18 Jan 2019 19:17:24 +0300 Subject: [PATCH 27/71] Fix clang warning --- dbms/src/IO/ReadBufferFromHDFS.cpp | 4 ++-- dbms/src/IO/WriteBufferFromHDFS.cpp | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/IO/ReadBufferFromHDFS.cpp b/dbms/src/IO/ReadBufferFromHDFS.cpp index 2545c21b094..fcd2894b374 100644 --- a/dbms/src/IO/ReadBufferFromHDFS.cpp +++ b/dbms/src/IO/ReadBufferFromHDFS.cpp @@ -16,9 +16,9 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl { struct HDFSBuilderDeleter { - void operator()(hdfsBuilder * builder) + void operator()(hdfsBuilder * builder_ptr) { - hdfsFreeBuilder(builder); + hdfsFreeBuilder(builder_ptr); } }; diff --git a/dbms/src/IO/WriteBufferFromHDFS.cpp b/dbms/src/IO/WriteBufferFromHDFS.cpp index 838dedb20d9..fca7ea05ff7 100644 --- a/dbms/src/IO/WriteBufferFromHDFS.cpp +++ b/dbms/src/IO/WriteBufferFromHDFS.cpp @@ -19,9 +19,9 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl { struct HDFSBuilderDeleter { - void operator()(hdfsBuilder * builder) + void operator()(hdfsBuilder * builder_ptr) { - hdfsFreeBuilder(builder); + hdfsFreeBuilder(builder_ptr); } }; From a74df23b089d6e7377cdf1c6dbe7d103317b13f7 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 18 Jan 2019 19:18:46 +0300 Subject: [PATCH 28/71] Minor docs fixes (#4098) * CLICKHOUSE-4063: less manual html @ index.md * CLICKHOUSE-4063: recommend markdown="1" in README.md * CLICKHOUSE-4003: manually purge custom.css for now * CLICKHOUSE-4064: expand
before any print (including to pdf) * CLICKHOUSE-3927: rearrange interfaces/formats.md a bit * CLICKHOUSE-3306: add few http headers * Remove copy-paste introduced in #3392 * Hopefully better chinese fonts #3392 * get rid of tabs @ custom.css * Apply comments and patch from #3384 * Add jdbc.md to ToC and some translation, though it still looks badly incomplete * minor punctuation * Add some backlinks to official website from mirrors that just blindly take markdown sources * Do not make fonts extra light * find . -name '*.md' -type f | xargs -I{} perl -pi -e 's//g' {} * find . -name '*.md' -type f | xargs -I{} perl -pi -e 's/ sql/g' {} * Remove outdated stuff from roadmap.md * Not so light font on front page too * Refactor Chinese formats.md to match recent changes in other languages * Update some links on front page * Remove some outdated comment * Add twitter link to front page * More front page links tuning * Add Amsterdam meetup link * Smaller font to avoid second line * Add Amsterdam link to README.md * Proper docs nav translation * Back to 300 font-weight except Chinese * fix docs build * Update Amsterdam link * remove symlinks * more zh punctuation * apply lost comment by @zhang2014 * Apply comments by @zhang2014 from #3417 * Remove Beijing link * rm incorrect symlink * restore content of docs/zh/operations/table_engines/index.md * CLICKHOUSE-3751: stem terms while searching docs * CLICKHOUSE-3751: use English stemmer in non-English docs too * CLICKHOUSE-4135 fix * Remove past meetup link * Add blog link to top nav * Add ContentSquare article link * Add form link to front page + refactor some texts * couple markup fixes * minor * Introduce basic ODBC driver page in docs * More verbose 3rd party libs disclaimer * Put third-party stuff into a separate folder * Separate third-party stuff in ToC too * Update links * Move stuff that is not really (only) a client library into a separate page * Add clickhouse-hdfs-loader link * Some introduction for "interfaces" section * Rewrite tcp.md * http_interface.md -> http.md * fix link * Remove unconvenient error for now * try to guess anchor instead of failing * remove symlink * Remove outdated info from introduction * remove ru roadmap.md * replace ru roadmap.md with symlink * Update roadmap.md * lost file * Title case in toc_en.yml * Sync "Functions" ToC section with en * Remove reference to pretty old ClickHouse release from docs * couple lost symlinks in fa * Close quote in proper place * Rewrite en/getting_started/index.md * Sync en<>ru getting_started/index.md * minor changes * Some gui.md refactoring * Translate DataGrip section to ru * Translate DataGrip section to zh * Translate DataGrip section to fa * Translate DBeaver section to fa * Translate DBeaver section to zh * Split third-party GUI to open-source and commercial * Mention some RDBMS integrations + ad-hoc translation fixes * Add rel="external nofollow" to outgoing links from docs * Lost blank lines * Fix class name * More rel="external nofollow" * Apply suggestions by @sundy-li * Mobile version of front page improvements * test * test 2 * test 3 * Update LICENSE * minor docs fix * Highlight current article as suggested by @sundy-li * fix link destination * Introduce backup.md (only "en" for now) * Mention INSERT+SELECT in backup.md * Some improvements for replication.md * Add backup.md to toc * Mention clickhouse-backup tool * Mention LightHouse in third-party GUI list * Introduce interfaces/third-party/proxy.md * Add clickhouse-bulk to proxy.md * Major extension of integrations.md contents * fix link target * remove unneeded file * better toc item name * fix markdown * better ru punctuation * Add yet another possible backup approach * Simplify copying permalinks to headers * Support non-eng link anchors in docs + update some deps * Generate anchors for single-page mode automatically * Remove anchors to top of pages * Remove anchors that nobody links to * build fixes * fix few links * restore css * fix some links * restore gifs * fix lost words * more docs fixes * docs fixes * NULL anchor * update urllib3 dependency * more fixes * Remove excessive content from print version * Try short license again * Back to long license for now * Introduce anchor integrity checks for single-page docs * Add --save-raw-single-page option to build.py (helps to debug incorrect anchors) * fix kafka engine links * fix one class of broken anchors * fix some broken links * Add https://github.com/hatarist/clickhouse-cli to third-party section (in gui.md for now, maybe will add cli.md later) * fix one more class of links to nowhere * less duplicate anchors * get rid of weird anchors * fix anchor * fix link * fix couple links * rearrange integrations.md a bit + sync zh version * Mention nagios plugin in other languages * port summingmergetree.md fix to zh * Make doc links to nowhere fatal * additional check in markdown extension * add option to skip pdf * 2019 have started * minor fixes * Inline custom_compression_codec.md into create.md for now --- .../table_engines/custom_compression_codec.md | 44 ----------------- docs/en/query_language/create.md | 45 +++++++++++++++-- .../table_engines/custom_compression_codec.md | 45 ----------------- docs/ru/query_language/create.md | 49 +++++++++++++++++-- 4 files changed, 87 insertions(+), 96 deletions(-) delete mode 100644 docs/en/operations/table_engines/custom_compression_codec.md delete mode 100644 docs/ru/operations/table_engines/custom_compression_codec.md diff --git a/docs/en/operations/table_engines/custom_compression_codec.md b/docs/en/operations/table_engines/custom_compression_codec.md deleted file mode 100644 index 79f9776e55a..00000000000 --- a/docs/en/operations/table_engines/custom_compression_codec.md +++ /dev/null @@ -1,44 +0,0 @@ - -# Column Compression Codecs - -Besides default data compression, defined in [server settings](../server_settings/settings.md#compression), per-column specification is also available. - -Supported compression algorithms: - -- `NONE` - no compression for data applied -- `LZ4` -- `LZ4HC(level)` - (level) - LZ4_HC compression algorithm with defined level. -Possible `level` range: \[3, 12\]. Default value: 9. Greater values stands for better compression and higher CPU usage. Recommended value range: [4,9]. -- `ZSTD(level)` - ZSTD compression algorithm with defined `level`. Possible `level` value range: \[1, 22\]. Default value: 1. -Greater values stands for better compression and higher CPU usage. -- `Delta(delta_bytes)` - compression approach when raw values are replace with difference of two neighbour values. Up to `delta_bytes` are used for storing delta value. -Possible `delta_bytes` values: 1, 2, 4, 8. Default value for delta bytes is `sizeof(type)`, if it is equals to 1, 2, 4, 8 and equals to 1 otherwise. - -Syntax example: -``` -CREATE TABLE codec_example -( - dt Date CODEC(ZSTD), /* используется уровень сжатия по-умолчанию */ - ts DateTime CODEC(LZ4HC), - float_value Float32 CODEC(NONE), - double_value Float64 CODEC(LZ4HC(9)) -) -ENGINE = MergeTree -PARTITION BY tuple() -ORDER BY dt -``` - -Codecs can be combined in a pipeline. Default table codec is not included into pipeline (if it should be applied to a column, you have to specify it explicitly in pipeline). Example below shows an optimization approach for storing timeseries metrics. -Usually, values for particular metric, stored in `path` does not differ significantly from point to point. Using delta-encoding allows to reduce disk space usage significantly. -``` -CREATE TABLE timeseries_example -( - dt Date, - ts DateTime, - path String, - value Float32 CODEC(Delta(2), ZSTD) -) -ENGINE = MergeTree -PARTITION BY dt -ORDER BY (path, ts) -``` diff --git a/docs/en/query_language/create.md b/docs/en/query_language/create.md index 387e244ce0e..7a1660e670c 100644 --- a/docs/en/query_language/create.md +++ b/docs/en/query_language/create.md @@ -80,10 +80,49 @@ If you add a new column to a table but later change its default expression, the It is not possible to set default values for elements in nested data structures. -### Column compression codecs +# Column Compression Codecs -Table columns can use either common compression codec, defined in server settings, or use individual one, defined in `compression_codec`. -[Detailed description](../operations/table_engines/custom_compression_codec.md). +Besides default data compression, defined in [server settings](../operations/server_settings/settings.md#compression), per-column specification is also available. + +Supported compression algorithms: + +- `NONE` - no compression for data applied +- `LZ4` +- `LZ4HC(level)` - (level) - LZ4\_HC compression algorithm with defined level. +Possible `level` range: \[3, 12\]. Default value: 9. Greater values stands for better compression and higher CPU usage. Recommended value range: [4,9]. +- `ZSTD(level)` - ZSTD compression algorithm with defined `level`. Possible `level` value range: \[1, 22\]. Default value: 1. +Greater values stands for better compression and higher CPU usage. +- `Delta(delta_bytes)` - compression approach when raw values are replace with difference of two neighbour values. Up to `delta_bytes` are used for storing delta value. +Possible `delta_bytes` values: 1, 2, 4, 8. Default value for delta bytes is `sizeof(type)`, if it is equals to 1, 2, 4, 8 and equals to 1 otherwise. + +Syntax example: +``` +CREATE TABLE codec_example +( + dt Date CODEC(ZSTD), /* используется уровень сжатия по-умолчанию */ + ts DateTime CODEC(LZ4HC), + float_value Float32 CODEC(NONE), + double_value Float64 CODEC(LZ4HC(9)) +) +ENGINE = MergeTree +PARTITION BY tuple() +ORDER BY dt +``` + +Codecs can be combined in a pipeline. Default table codec is not included into pipeline (if it should be applied to a column, you have to specify it explicitly in pipeline). Example below shows an optimization approach for storing timeseries metrics. +Usually, values for particular metric, stored in `path` does not differ significantly from point to point. Using delta-encoding allows to reduce disk space usage significantly. +``` +CREATE TABLE timeseries_example +( + dt Date, + ts DateTime, + path String, + value Float32 CODEC(Delta(2), ZSTD) +) +ENGINE = MergeTree +PARTITION BY dt +ORDER BY (path, ts) +``` ### Temporary Tables diff --git a/docs/ru/operations/table_engines/custom_compression_codec.md b/docs/ru/operations/table_engines/custom_compression_codec.md deleted file mode 100644 index 5f70cc66dac..00000000000 --- a/docs/ru/operations/table_engines/custom_compression_codec.md +++ /dev/null @@ -1,45 +0,0 @@ - -# Форматы сжатия для колонок - -Помимо сжатия для колонок по умолчанию, определяемого в [настройках сервера](../server_settings/settings.md#compression), -существует возможность указать формат сжатия индивидуально для каждой колонки. - -Поддерживаемые форматы: - -- `NONE` - сжатие отсутствует -- `LZ4` -- `LZ4HC(level)` - алгоритм сжатия LZ4_HC с указанным уровнем компрессии `level`. -Возможный диапазон значений `level`: \[3, 12\]. Значение по умолчанию: 9. Чем выше уровень, тем лучше сжатие, но тратится больше времени. Рекомендованный диапазон \[4, 9\]. -- `ZSTD(level)` - алгоритм сжатия ZSTD с указанным уровнем компрессии `level`. Возможный диапазон значений `level`: \[1, 22\]. Значение по умолчанию: 1. -Чем выше уровень, тем лучше сжатие, но тратится больше времени. -- `Delta(delta_bytes)` - способ сжатия, при котором вместо числовых значений поля сохраняется разность между двумя соседними значениями. Значение `delta_bytes` - число байт для хранения дельты. -Возможные значения: 1, 2, 4, 8. Значение по умолчанию: если `sizeof(type)` равен 1, 2, 4, 8 - `sizeof(type)`, иначе - 1. - -Пример использования: -``` -CREATE TABLE codec_example -( - dt Date CODEC(ZSTD), /* используется уровень сжатия по умолчанию */ - ts DateTime CODEC(LZ4HC), - float_value Float32 CODEC(NONE), - double_value Float64 CODEC(LZ4HC(9)) -) -ENGINE = MergeTree -PARTITION BY tuple() -ORDER BY dt -``` - -Кодеки могут комбинироваться между собой. Если для колонки указана своя последовательность кодеков, то общий табличный кодек не применяется (должен быть указан в последовательности принудительно, если нужен). В примере ниже - оптимизация для хранения timeseries метрик. -Как правило, значения одной и той же метрики `path` не сильно различаются между собой, и выгоднее использовать дельта-компрессию вместо записи всего числа: -``` -CREATE TABLE timeseries_example -( - dt Date, - ts DateTime, - path String, - value Float32 CODEC(Delta(2), ZSTD) -) -ENGINE = MergeTree -PARTITION BY dt -ORDER BY (path, ts) -``` diff --git a/docs/ru/query_language/create.md b/docs/ru/query_language/create.md index 74269ad4823..6f1c5d3811c 100644 --- a/docs/ru/query_language/create.md +++ b/docs/ru/query_language/create.md @@ -1,6 +1,6 @@ ## CREATE DATABASE -Создание базы данных db_name +Создание базы данных db\_name. ```sql CREATE DATABASE [IF NOT EXISTS] db_name @@ -80,10 +80,51 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = engine AS SELECT ... Отсутствует возможность задать значения по умолчанию для элементов вложенных структур данных. -### Форматы сжатия для колонок +# Форматы сжатия для колонок + +Помимо сжатия для колонок по умолчанию, определяемого в [настройках сервера](../operations/server_settings/settings.md#compression), +существует возможность указать формат сжатия индивидуально для каждой колонки. + +Поддерживаемые форматы: + +- `NONE` - сжатие отсутствует +- `LZ4` +- `LZ4HC(level)` - алгоритм сжатия LZ4\_HC с указанным уровнем компрессии `level`. +Возможный диапазон значений `level`: \[3, 12\]. Значение по умолчанию: 9. Чем выше уровень, тем лучше сжатие, но тратится больше времени. Рекомендованный диапазон \[4, 9\]. +- `ZSTD(level)` - алгоритм сжатия ZSTD с указанным уровнем компрессии `level`. Возможный диапазон значений `level`: \[1, 22\]. Значение по умолчанию: 1. +Чем выше уровень, тем лучше сжатие, но тратится больше времени. +- `Delta(delta_bytes)` - способ сжатия, при котором вместо числовых значений поля сохраняется разность между двумя соседними значениями. Значение `delta_bytes` - число байт для хранения дельты. +Возможные значения: 1, 2, 4, 8. Значение по умолчанию: если `sizeof(type)` равен 1, 2, 4, 8 - `sizeof(type)`, иначе - 1. + +Пример использования: +``` +CREATE TABLE codec_example +( + dt Date CODEC(ZSTD), /* используется уровень сжатия по умолчанию */ + ts DateTime CODEC(LZ4HC), + float_value Float32 CODEC(NONE), + double_value Float64 CODEC(LZ4HC(9)) +) +ENGINE = MergeTree +PARTITION BY tuple() +ORDER BY dt +``` + +Кодеки могут комбинироваться между собой. Если для колонки указана своя последовательность кодеков, то общий табличный кодек не применяется (должен быть указан в последовательности принудительно, если нужен). В примере ниже - оптимизация для хранения timeseries метрик. +Как правило, значения одной и той же метрики `path` не сильно различаются между собой, и выгоднее использовать дельта-компрессию вместо записи всего числа: +``` +CREATE TABLE timeseries_example +( + dt Date, + ts DateTime, + path String, + value Float32 CODEC(Delta(2), ZSTD) +) +ENGINE = MergeTree +PARTITION BY dt +ORDER BY (path, ts) +``` -Таблица может использовать общий формат сжатия, установленный в настройках сервера, или применять к каждой колонке свой формат, указанный в `compression_codec`. -[Подробное описание](../operations/table_engines/custom_compression_codec.md). ### Временные таблицы From 112f62f49f3db9357165634ecd27d8e2bb02ce67 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 18 Jan 2019 19:35:43 +0300 Subject: [PATCH 29/71] Fixed linking order of glibc-compatibility library --- CMakeLists.txt | 5 ++++- dbms/CMakeLists.txt | 5 ----- libs/libglibc-compatibility/CMakeLists.txt | 2 ++ 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index ef504c107bf..f6a191d0efc 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -83,7 +83,10 @@ if (CMAKE_SYSTEM_PROCESSOR MATCHES "amd64|x86_64") if (OS_LINUX) option (GLIBC_COMPATIBILITY "Set to TRUE to enable compatibility with older glibc libraries. Only for x86_64, Linux. Implies USE_INTERNAL_MEMCPY." ON) - endif() + if (GLIBC_COMPATIBILITY) + link_libraries(glibc-compatibility) + endif () + endif () endif () if (GLIBC_COMPATIBILITY) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 84099810164..9dd83c56dc6 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -318,11 +318,6 @@ target_include_directories (clickhouse_common_io BEFORE PRIVATE ${COMMON_INCLUDE add_subdirectory (programs) add_subdirectory (tests) -if (GLIBC_COMPATIBILITY AND NOT CLICKHOUSE_SPLIT_BINARY) - MESSAGE(STATUS "Some symbols from glibc will be replaced for compatibility") - target_link_libraries(dbms PUBLIC glibc-compatibility) -endif() - if (ENABLE_TESTS) macro (grep_gtest_sources BASE_DIR DST_VAR) # Cold match files that are not in tests/ directories diff --git a/libs/libglibc-compatibility/CMakeLists.txt b/libs/libglibc-compatibility/CMakeLists.txt index 15aa9fe7e3d..0a3dd74123e 100644 --- a/libs/libglibc-compatibility/CMakeLists.txt +++ b/libs/libglibc-compatibility/CMakeLists.txt @@ -31,4 +31,6 @@ add_library (glibc-compatibility ${GLIBC_COMPATIBILITY_SOURCES}) target_include_directories(glibc-compatibility PRIVATE libcxxabi) +set_target_properties(glibc-compatibility PROPERTIES LINK_LIBRARIES "") + add_subdirectory (tests) From ed124c955a2135f44424a3069d96ba1a5aeac7e3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 18 Jan 2019 21:20:07 +0300 Subject: [PATCH 30/71] Link all libraries to "glibc-compatibility" --- CMakeLists.txt | 1 + libs/libglibc-compatibility/CMakeLists.txt | 7 +++++++ 2 files changed, 8 insertions(+) diff --git a/CMakeLists.txt b/CMakeLists.txt index f6a191d0efc..63bf39f8383 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -84,6 +84,7 @@ if (CMAKE_SYSTEM_PROCESSOR MATCHES "amd64|x86_64") if (OS_LINUX) option (GLIBC_COMPATIBILITY "Set to TRUE to enable compatibility with older glibc libraries. Only for x86_64, Linux. Implies USE_INTERNAL_MEMCPY." ON) if (GLIBC_COMPATIBILITY) + message (STATUS "Some symbols from glibc will be replaced for compatibility") link_libraries(glibc-compatibility) endif () endif () diff --git a/libs/libglibc-compatibility/CMakeLists.txt b/libs/libglibc-compatibility/CMakeLists.txt index 0a3dd74123e..0c819654271 100644 --- a/libs/libglibc-compatibility/CMakeLists.txt +++ b/libs/libglibc-compatibility/CMakeLists.txt @@ -31,6 +31,13 @@ add_library (glibc-compatibility ${GLIBC_COMPATIBILITY_SOURCES}) target_include_directories(glibc-compatibility PRIVATE libcxxabi) +# glibc-compatibility does not depend on any libraries but is linked to all libraries implicitly. +# Avoid linking of the library to itself. set_target_properties(glibc-compatibility PROPERTIES LINK_LIBRARIES "") +# Garbage. Rough explanation: some libraries want to install itself and CMake forces us to also install the glibc-compatibility library. +install(TARGETS glibc-compatibility EXPORT CapnProtoTargets ARCHIVE DESTINATION "/tmp") +install(TARGETS glibc-compatibility EXPORT protobuf-targets ARCHIVE DESTINATION "/tmp") +install(TARGETS glibc-compatibility EXPORT double-conversionTargets ARCHIVE DESTINATION "/tmp") + add_subdirectory (tests) From 9f1837f01f05a96c84f0654019645948bfd6aa7c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 18 Jan 2019 21:27:46 +0300 Subject: [PATCH 31/71] Update CMakeLists.txt --- CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 0a452668acf..69a4a509c5b 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -81,7 +81,7 @@ option (ENABLE_TESTS "Enables tests" ON) if (CMAKE_SYSTEM_PROCESSOR MATCHES "amd64|x86_64") option (USE_INTERNAL_MEMCPY "Use internal implementation of 'memcpy' function instead of provided by libc. Only for x86_64." ON) - if (OS_LINUX) + if (OS_LINUX AND NOT UNBUNDLED) option (GLIBC_COMPATIBILITY "Set to TRUE to enable compatibility with older glibc libraries. Only for x86_64, Linux. Implies USE_INTERNAL_MEMCPY." ON) if (GLIBC_COMPATIBILITY) message (STATUS "Some symbols from glibc will be replaced for compatibility") From 33d5f0c8dd9675051cf3ff961b30f6841ac92928 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 18 Jan 2019 21:57:11 +0300 Subject: [PATCH 32/71] Fix potential null pointer dereference --- dbms/src/IO/ReadBufferFromHDFS.cpp | 6 ++++++ dbms/src/IO/WriteBufferFromHDFS.cpp | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/dbms/src/IO/ReadBufferFromHDFS.cpp b/dbms/src/IO/ReadBufferFromHDFS.cpp index fcd2894b374..804f2357b86 100644 --- a/dbms/src/IO/ReadBufferFromHDFS.cpp +++ b/dbms/src/IO/ReadBufferFromHDFS.cpp @@ -56,6 +56,12 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl } fin = hdfsOpenFile(fs, path.c_str(), O_RDONLY, 0, 0, 0); + + if (fin == nullptr) + { + throw Exception("Unable to open HDFS file: " + path + " error: " + std::string(hdfsGetLastError()), + ErrorCodes::NETWORK_ERROR); + } } ~ReadBufferFromHDFSImpl() diff --git a/dbms/src/IO/WriteBufferFromHDFS.cpp b/dbms/src/IO/WriteBufferFromHDFS.cpp index fca7ea05ff7..ffedfb6dca4 100644 --- a/dbms/src/IO/WriteBufferFromHDFS.cpp +++ b/dbms/src/IO/WriteBufferFromHDFS.cpp @@ -60,6 +60,12 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl } fout = hdfsOpenFile(fs, path.c_str(), O_WRONLY, 0, 0, 0); + if (fout == nullptr) + { + throw Exception("Unable to open HDFS file: " + path + " error: " + std::string(hdfsGetLastError()), + ErrorCodes::NETWORK_ERROR); + } + } ~WriteBufferFromHDFSImpl() From 57f6e81687b06784f846d40156de16f7b6a3e023 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 18 Jan 2019 22:41:45 +0300 Subject: [PATCH 33/71] Updated "ssl" submodule --- contrib/ssl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/ssl b/contrib/ssl index dbbbcdbbd17..ba8de796195 160000 --- a/contrib/ssl +++ b/contrib/ssl @@ -1 +1 @@ -Subproject commit dbbbcdbbd17785566f8f9c107b714f9e213d7293 +Subproject commit ba8de796195ff9d8bb0249ce289b83226b848b77 From 0572a62939282c0ee19dc94403a4cd5ad8085728 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 18 Jan 2019 22:42:29 +0300 Subject: [PATCH 34/71] Removed useless code in CMakeLists for "ssl" --- contrib/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index e0203b1b046..01d62a15847 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -106,7 +106,6 @@ if (USE_INTERNAL_SSL_LIBRARY) if (NOT MAKE_STATIC_LIBRARIES) set (BUILD_SHARED 1) endif () - set (USE_SHARED ${USE_STATIC_LIBRARIES}) set (LIBRESSL_SKIP_INSTALL 1) add_subdirectory (ssl) target_include_directories(${OPENSSL_CRYPTO_LIBRARY} SYSTEM PUBLIC ${OPENSSL_INCLUDE_DIR}) From 2752cf7eafa55ea411c42559efa13a6deae7dda4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Jan 2019 01:15:58 +0300 Subject: [PATCH 35/71] Added a test #4097 --- dbms/tests/queries/0_stateless/00824_filesystem.reference | 1 + dbms/tests/queries/0_stateless/00824_filesystem.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00824_filesystem.reference create mode 100644 dbms/tests/queries/0_stateless/00824_filesystem.sql diff --git a/dbms/tests/queries/0_stateless/00824_filesystem.reference b/dbms/tests/queries/0_stateless/00824_filesystem.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00824_filesystem.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/00824_filesystem.sql b/dbms/tests/queries/0_stateless/00824_filesystem.sql new file mode 100644 index 00000000000..cd4d69a703e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00824_filesystem.sql @@ -0,0 +1 @@ +SELECT filesystemCapacity() >= filesystemFree() AND filesystemFree() >= filesystemAvailable() AND filesystemAvailable() >= 0; From f4ae6a4cda2151fa39b98d328843e2236cf80d51 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Jan 2019 02:52:21 +0300 Subject: [PATCH 36/71] Removed useless install --- CMakeLists.txt | 2 +- contrib/CMakeLists.txt | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 69a4a509c5b..4265cc126f1 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -255,7 +255,7 @@ include (libs/libdaemon/cmake/find_unwind.cmake) include (cmake/print_flags.cmake) -add_subdirectory (contrib) +add_subdirectory (contrib EXCLUDE_FROM_ALL) add_subdirectory (libs) add_subdirectory (utils) add_subdirectory (dbms) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 01d62a15847..7b66bfce063 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -55,6 +55,7 @@ endif () if (USE_INTERNAL_ZLIB_LIBRARY) set (ZLIB_ENABLE_TESTS 0 CACHE INTERNAL "") + set (SKIP_INSTALL_ALL 1 CACHE INTERNAL "") set (ZLIB_COMPAT 1 CACHE INTERNAL "") # also enables WITH_GZFILEOP set (WITH_NATIVE_INSTRUCTIONS ${ARCH_NATIVE} CACHE INTERNAL "") if (OS_FREEBSD OR ARCH_I386) @@ -106,7 +107,7 @@ if (USE_INTERNAL_SSL_LIBRARY) if (NOT MAKE_STATIC_LIBRARIES) set (BUILD_SHARED 1) endif () - set (LIBRESSL_SKIP_INSTALL 1) + set (LIBRESSL_SKIP_INSTALL 1 CACHE INTERNAL "") add_subdirectory (ssl) target_include_directories(${OPENSSL_CRYPTO_LIBRARY} SYSTEM PUBLIC ${OPENSSL_INCLUDE_DIR}) target_include_directories(${OPENSSL_SSL_LIBRARY} SYSTEM PUBLIC ${OPENSSL_INCLUDE_DIR}) From 3feb4efdf2cb748db7f98e869939989373e9eca1 Mon Sep 17 00:00:00 2001 From: proller Date: Sat, 19 Jan 2019 19:48:54 +0300 Subject: [PATCH 37/71] Disable rdkafka in macos --- cmake/find_rdkafka.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/find_rdkafka.cmake b/cmake/find_rdkafka.cmake index 1c93b99a344..52a865cd50b 100644 --- a/cmake/find_rdkafka.cmake +++ b/cmake/find_rdkafka.cmake @@ -1,4 +1,4 @@ -if (NOT ARCH_ARM AND NOT ARCH_32) +if (NOT ARCH_ARM AND NOT ARCH_32 AND NOT APPLE) option (ENABLE_RDKAFKA "Enable kafka" ON) endif () From 21f1c4d1ceeeea81c2f24030f3e08878f550a5d0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 19 Jan 2019 23:17:19 +0300 Subject: [PATCH 38/71] Better code and tests for bad cases --- dbms/src/IO/HDFSCommon.cpp | 43 ++++++++++++ dbms/src/IO/HDFSCommon.h | 38 +++++++++++ dbms/src/IO/ReadBufferFromHDFS.cpp | 67 +++++-------------- dbms/src/IO/WriteBufferFromHDFS.cpp | 60 +++++------------ .../integration/test_storage_hdfs/test.py | 18 +++++ 5 files changed, 132 insertions(+), 94 deletions(-) create mode 100644 dbms/src/IO/HDFSCommon.cpp create mode 100644 dbms/src/IO/HDFSCommon.h diff --git a/dbms/src/IO/HDFSCommon.cpp b/dbms/src/IO/HDFSCommon.cpp new file mode 100644 index 00000000000..34cef18068f --- /dev/null +++ b/dbms/src/IO/HDFSCommon.cpp @@ -0,0 +1,43 @@ +#include + +#if USE_HDFS +#include +namespace DB +{ +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +extern const int NETWORK_ERROR; +} +HDFSBuilderPtr createHDFSBuilder(const Poco::URI & uri) +{ + auto & host = uri.getHost(); + auto port = uri.getPort(); + auto & path = uri.getPath(); + if (host.empty() || port == 0 || path.empty()) + throw Exception("Illegal HDFS URI: " + uri.toString(), ErrorCodes::BAD_ARGUMENTS); + + HDFSBuilderPtr builder(hdfsNewBuilder()); + if (builder == nullptr) + throw Exception("Unable to create builder to connect to HDFS: " + uri.toString() + " " + std::string(hdfsGetLastError()), + ErrorCodes::NETWORK_ERROR); + hdfsBuilderConfSetStr(builder.get(), "input.read.timeout", "60000"); // 1 min + hdfsBuilderConfSetStr(builder.get(), "input.write.timeout", "60000"); // 1 min + hdfsBuilderConfSetStr(builder.get(), "input.connect.timeout", "60000"); // 1 min + + hdfsBuilderSetNameNode(builder.get(), host.c_str()); + hdfsBuilderSetNameNodePort(builder.get(), port); + return builder; +} + +HDFSFSPtr createHDFSFS(hdfsBuilder * builder) +{ + HDFSFSPtr fs(hdfsBuilderConnect(builder)); + if (fs == nullptr) + throw Exception("Unable to connect to HDFS: " + std::string(hdfsGetLastError()), + ErrorCodes::NETWORK_ERROR); + + return fs; +} +} +#endif diff --git a/dbms/src/IO/HDFSCommon.h b/dbms/src/IO/HDFSCommon.h new file mode 100644 index 00000000000..8c526d908bb --- /dev/null +++ b/dbms/src/IO/HDFSCommon.h @@ -0,0 +1,38 @@ +#include +#include +#include +#include + +#if USE_HDFS +#include + +namespace DB +{ +namespace detail +{ +struct HDFSBuilderDeleter +{ + void operator()(hdfsBuilder * builder_ptr) + { + hdfsFreeBuilder(builder_ptr); + } +}; +struct HDFSFsDeleter +{ + void operator()(hdfsFS fs_ptr) + { + hdfsDisconnect(fs_ptr); + } +}; + +} + +using HDFSBuilderPtr = std::unique_ptr; +using HDFSFSPtr = std::unique_ptr, detail::HDFSFsDeleter>; + +// set read/connect timeout, default value in libhdfs3 is about 1 hour, and too large +/// TODO Allow to tune from query Settings. +HDFSBuilderPtr createHDFSBuilder(const Poco::URI & hdfs_uri); +HDFSFSPtr createHDFSFS(hdfsBuilder * builder); +} +#endif diff --git a/dbms/src/IO/ReadBufferFromHDFS.cpp b/dbms/src/IO/ReadBufferFromHDFS.cpp index 804f2357b86..39f7d1fcc7e 100644 --- a/dbms/src/IO/ReadBufferFromHDFS.cpp +++ b/dbms/src/IO/ReadBufferFromHDFS.cpp @@ -2,82 +2,51 @@ #if USE_HDFS #include +#include #include namespace DB { namespace ErrorCodes { - extern const int BAD_ARGUMENTS; extern const int NETWORK_ERROR; + extern const int CANNOT_OPEN_FILE; } struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl { - struct HDFSBuilderDeleter - { - void operator()(hdfsBuilder * builder_ptr) - { - hdfsFreeBuilder(builder_ptr); - } - }; - - std::string hdfs_uri; - std::unique_ptr builder; - hdfsFS fs; + Poco::URI hdfs_uri; hdfsFile fin; - + HDFSBuilderPtr builder; + HDFSFSPtr fs; + ReadBufferFromHDFSImpl(const std::string & hdfs_name_) : hdfs_uri(hdfs_name_) - , builder(hdfsNewBuilder()) + , builder(createHDFSBuilder(hdfs_uri)) + , fs(createHDFSFS(builder.get())) { - Poco::URI uri(hdfs_name_); - auto & host = uri.getHost(); - auto port = uri.getPort(); - auto & path = uri.getPath(); - if (host.empty() || port == 0 || path.empty()) - { - throw Exception("Illegal HDFS URI: " + hdfs_uri, ErrorCodes::BAD_ARGUMENTS); - } - // set read/connect timeout, default value in libhdfs3 is about 1 hour, and too large - /// TODO Allow to tune from query Settings. - hdfsBuilderConfSetStr(builder.get(), "input.read.timeout", "60000"); // 1 min - hdfsBuilderConfSetStr(builder.get(), "input.write.timeout", "60000"); // 1 min - hdfsBuilderConfSetStr(builder.get(), "input.connect.timeout", "60000"); // 1 min - hdfsBuilderSetNameNode(builder.get(), host.c_str()); - hdfsBuilderSetNameNodePort(builder.get(), port); - fs = hdfsBuilderConnect(builder.get()); - - if (fs == nullptr) - { - throw Exception("Unable to connect to HDFS: " + std::string(hdfsGetLastError()), - ErrorCodes::NETWORK_ERROR); - } - - fin = hdfsOpenFile(fs, path.c_str(), O_RDONLY, 0, 0, 0); + auto & path = hdfs_uri.getPath(); + fin = hdfsOpenFile(fs.get(), path.c_str(), O_RDONLY, 0, 0, 0); if (fin == nullptr) - { throw Exception("Unable to open HDFS file: " + path + " error: " + std::string(hdfsGetLastError()), - ErrorCodes::NETWORK_ERROR); - } - } - - ~ReadBufferFromHDFSImpl() - { - hdfsCloseFile(fs, fin); - hdfsDisconnect(fs); + ErrorCodes::CANNOT_OPEN_FILE); } int read(char * start, size_t size) { - int bytes_read = hdfsRead(fs, fin, start, size); + int bytes_read = hdfsRead(fs.get(), fin, start, size); if (bytes_read < 0) - throw Exception("Fail to read HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), + throw Exception("Fail to read HDFS file: " + hdfs_uri.toString() + " " + std::string(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR); return bytes_read; } + + ~ReadBufferFromHDFSImpl() + { + hdfsCloseFile(fs.get(), fin); + } }; ReadBufferFromHDFS::ReadBufferFromHDFS(const std::string & hdfs_name_, size_t buf_size) diff --git a/dbms/src/IO/WriteBufferFromHDFS.cpp b/dbms/src/IO/WriteBufferFromHDFS.cpp index ffedfb6dca4..09da3004b3e 100644 --- a/dbms/src/IO/WriteBufferFromHDFS.cpp +++ b/dbms/src/IO/WriteBufferFromHDFS.cpp @@ -2,6 +2,7 @@ #if USE_HDFS #include +#include #include namespace DB @@ -9,86 +10,55 @@ namespace DB namespace ErrorCodes { -extern const int BAD_ARGUMENTS; extern const int NETWORK_ERROR; +extern const int CANNOT_OPEN_FILE; extern const int CANNOT_FSYNC; } struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl { - struct HDFSBuilderDeleter - { - void operator()(hdfsBuilder * builder_ptr) - { - hdfsFreeBuilder(builder_ptr); - } - }; - - std::string hdfs_uri; - std::unique_ptr builder; - hdfsFS fs; + Poco::URI hdfs_uri; hdfsFile fout; + HDFSBuilderPtr builder; + HDFSFSPtr fs; WriteBufferFromHDFSImpl(const std::string & hdfs_name_) : hdfs_uri(hdfs_name_) - , builder(hdfsNewBuilder()) + , builder(createHDFSBuilder(hdfs_uri)) + , fs(createHDFSFS(builder.get())) { - Poco::URI uri(hdfs_name_); - auto & host = uri.getHost(); - auto port = uri.getPort(); - auto & path = uri.getPath(); - if (host.empty() || port == 0 || path.empty()) - { - throw Exception("Illegal HDFS URI: " + hdfs_uri, ErrorCodes::BAD_ARGUMENTS); - } + auto & path = hdfs_uri.getPath(); + fout = hdfsOpenFile(fs.get(), path.c_str(), O_WRONLY, 0, 0, 0); - // set read/connect timeout, default value in libhdfs3 is about 1 hour, and too large - /// TODO Allow to tune from query Settings. - hdfsBuilderConfSetStr(builder.get(), "input.read.timeout", "60000"); // 1 min - hdfsBuilderConfSetStr(builder.get(), "input.write.timeout", "60000"); // 1 min - hdfsBuilderConfSetStr(builder.get(), "input.connect.timeout", "60000"); // 1 min - - hdfsBuilderSetNameNode(builder.get(), host.c_str()); - hdfsBuilderSetNameNodePort(builder.get(), port); - fs = hdfsBuilderConnect(builder.get()); - - if (fs == nullptr) - { - throw Exception("Unable to connect to HDFS: " + std::string(hdfsGetLastError()), - ErrorCodes::NETWORK_ERROR); - } - - fout = hdfsOpenFile(fs, path.c_str(), O_WRONLY, 0, 0, 0); if (fout == nullptr) { throw Exception("Unable to open HDFS file: " + path + " error: " + std::string(hdfsGetLastError()), - ErrorCodes::NETWORK_ERROR); + ErrorCodes::CANNOT_OPEN_FILE); } } ~WriteBufferFromHDFSImpl() { - hdfsCloseFile(fs, fout); - hdfsDisconnect(fs); + hdfsCloseFile(fs.get(), fout); } int write(const char * start, size_t size) { - int bytes_written = hdfsWrite(fs, fout, start, size); + int bytes_written = hdfsWrite(fs.get(), fout, start, size); if (bytes_written < 0) - throw Exception("Fail to write HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), + throw Exception("Fail to write HDFS file: " + hdfs_uri.toString() + " " + std::string(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR); return bytes_written; } void sync() { - int result = hdfsSync(fs, fout); + int result = hdfsSync(fs.get(), fout); if (result < 0) - throwFromErrno("Cannot HDFS sync" + hdfs_uri + " " + std::string(hdfsGetLastError()), + throwFromErrno("Cannot HDFS sync" + hdfs_uri.toString() + " " + std::string(hdfsGetLastError()), ErrorCodes::CANNOT_FSYNC); } }; diff --git a/dbms/tests/integration/test_storage_hdfs/test.py b/dbms/tests/integration/test_storage_hdfs/test.py index 0779e9c2ec2..173c2d77b3f 100644 --- a/dbms/tests/integration/test_storage_hdfs/test.py +++ b/dbms/tests/integration/test_storage_hdfs/test.py @@ -56,3 +56,21 @@ def test_write_table(started_cluster): result = "10\ttomas\t55.55\n11\tjack\t32.54\n" assert hdfs_api.read_data("/other_storage") == result assert node1.query("select * from OtherHDFSStorage order by id") == result + +def test_bad_hdfs_uri(started_cluster): + try: + node1.query("create table BadStorage1 (id UInt32, name String, weight Float64) ENGINE = HDFS('hads:hgsdfs100500:9000/other_storage', 'TSV')") + except Exception as ex: + print ex + assert 'Illegal HDFS URI' in str(ex) + try: + node1.query("create table BadStorage2 (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs100500:9000/other_storage', 'TSV')") + except Exception as ex: + print ex + assert 'Unable to create builder to connect to HDFS' in str(ex) + + try: + node1.query("create table BadStorage3 (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/<>', 'TSV')") + except Exception as ex: + print ex + assert 'Unable to open HDFS file' in str(ex) From f1bb9fc05a7766c84832f23ee98cf422c1009b79 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Jan 2019 00:05:20 +0300 Subject: [PATCH 39/71] Removed useless code --- libs/libcommon/include/common/preciseExp10.h | 12 +-------- libs/libcommon/src/preciseExp10.c | 27 -------------------- libs/libdaemon/src/BaseDaemon.cpp | 2 +- 3 files changed, 2 insertions(+), 39 deletions(-) diff --git a/libs/libcommon/include/common/preciseExp10.h b/libs/libcommon/include/common/preciseExp10.h index 9b757a3a57d..137205c484c 100644 --- a/libs/libcommon/include/common/preciseExp10.h +++ b/libs/libcommon/include/common/preciseExp10.h @@ -8,19 +8,9 @@ * Note: the function names are different to avoid confusion with symbols from the system libm. */ -#include /// for __THROW - -// freebsd have no __THROW -#if !defined(__THROW) -#define __THROW -#endif - extern "C" { -double preciseExp10(double x) __THROW; -double precisePow10(double x) __THROW; -float preciseExp10f(float x) __THROW; -float precisePow10f(float x) __THROW; +double preciseExp10(double x); } diff --git a/libs/libcommon/src/preciseExp10.c b/libs/libcommon/src/preciseExp10.c index 62793ff66bb..49c508e6fed 100644 --- a/libs/libcommon/src/preciseExp10.c +++ b/libs/libcommon/src/preciseExp10.c @@ -190,30 +190,3 @@ double preciseExp10(double x) } return pow(10.0, x); } - -float preciseExp10f(float x) -{ - static const float p10[] = { - 1e-7f, 1e-6f, 1e-5f, 1e-4f, 1e-3f, 1e-2f, 1e-1f, - 1, 1e1, 1e2, 1e3, 1e4, 1e5, 1e6, 1e7 - }; - float n, y = modff(x, &n); - union {float f; uint32_t i;} u = {n}; - /* fabsf(n) < 8 without raising invalid on nan */ - if ((u.i>>23 & 0xff) < 0x7f+3) { - if (!y) return p10[(int)n+7]; - y = exp2f(3.32192809488736234787031942948939f * y); - return y * p10[(int)n+7]; - } - return exp2(3.32192809488736234787031942948939 * x); -} - -double precisePow10(double x) -{ - return preciseExp10(x); -} - -float precisePow10f(float x) -{ - return preciseExp10f(x); -} diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index bad38c78529..838730185ad 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -7,8 +7,8 @@ #include #include -#include #include +#include #include #include #include From e22e19ddc943907a842e92a3299215e7917b0278 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Jan 2019 00:35:46 +0300 Subject: [PATCH 40/71] Added missing glibc compatibility functions --- libs/libglibc-compatibility/CMakeLists.txt | 5 +- .../glibc-compatibility.c | 26 +- libs/libglibc-compatibility/musl/atomic.h | 318 ++++++++++++++++++ .../libglibc-compatibility/musl/atomic_arch.h | 123 +++++++ .../musl/sched_getcpu.c | 45 +++ .../musl/timespec_get.c | 12 + 6 files changed, 526 insertions(+), 3 deletions(-) create mode 100644 libs/libglibc-compatibility/musl/atomic.h create mode 100644 libs/libglibc-compatibility/musl/atomic_arch.h create mode 100644 libs/libglibc-compatibility/musl/sched_getcpu.c create mode 100644 libs/libglibc-compatibility/musl/timespec_get.c diff --git a/libs/libglibc-compatibility/CMakeLists.txt b/libs/libglibc-compatibility/CMakeLists.txt index dadf6f03fc3..060f4b7a89b 100644 --- a/libs/libglibc-compatibility/CMakeLists.txt +++ b/libs/libglibc-compatibility/CMakeLists.txt @@ -20,7 +20,10 @@ musl/glob.c musl/exp2f.c musl/pwritev.c musl/getrandom.c -musl/fcntl.c) +musl/fcntl.c +musl/timespec_get.c +musl/sched_getcpu.c +) if (MAKE_STATIC_LIBRARIES) set (GLIBC_COMPATIBILITY_SOURCES ${GLIBC_COMPATIBILITY_SOURCES} diff --git a/libs/libglibc-compatibility/glibc-compatibility.c b/libs/libglibc-compatibility/glibc-compatibility.c index 74bf4346a8c..ca5ceb9c5b1 100644 --- a/libs/libglibc-compatibility/glibc-compatibility.c +++ b/libs/libglibc-compatibility/glibc-compatibility.c @@ -1,5 +1,5 @@ -/** Allows to build programs with libc 2.18 and run on systems with at least libc 2.4, - * such as Ubuntu Lucid or CentOS 6. +/** Allows to build programs with libc 2.27 and run on systems with at least libc 2.4, + * such as Ubuntu Hardy or CentOS 5. * * Also look at http://www.lightofdawn.org/wiki/wiki.cgi/NewAppsOnOldGlibc */ @@ -85,6 +85,28 @@ int __vasprintf_chk(char **s, int unused, const char *fmt, va_list ap) return vasprintf(s, fmt, ap); } +int __asprintf_chk(char **result_ptr, int unused, const char *format, ...) +{ + int ret; + va_list ap; + va_start (ap, format); + ret = vasprintf(result_ptr, format, ap); + va_end (ap); + return ret; +} + +int vdprintf(int fd, const char *format, va_list ap); + +int __dprintf_chk (int d, int unused, const char *format, ...) +{ + int ret; + va_list ap; + va_start (ap, format); + ret = vdprintf(d, format, ap); + va_end (ap); + return ret; +} + size_t fread(void *ptr, size_t size, size_t nmemb, void *stream); size_t __fread_chk(void *ptr, size_t unused, size_t size, size_t nmemb, void *stream) diff --git a/libs/libglibc-compatibility/musl/atomic.h b/libs/libglibc-compatibility/musl/atomic.h new file mode 100644 index 00000000000..f938879b077 --- /dev/null +++ b/libs/libglibc-compatibility/musl/atomic.h @@ -0,0 +1,318 @@ +#ifndef _ATOMIC_H +#define _ATOMIC_H + +#include + +#include "atomic_arch.h" + +#ifdef a_ll + +#ifndef a_pre_llsc +#define a_pre_llsc() +#endif + +#ifndef a_post_llsc +#define a_post_llsc() +#endif + +#ifndef a_cas +#define a_cas a_cas +static inline int a_cas(volatile int *p, int t, int s) +{ + int old; + a_pre_llsc(); + do old = a_ll(p); + while (old==t && !a_sc(p, s)); + a_post_llsc(); + return old; +} +#endif + +#ifndef a_swap +#define a_swap a_swap +static inline int a_swap(volatile int *p, int v) +{ + int old; + a_pre_llsc(); + do old = a_ll(p); + while (!a_sc(p, v)); + a_post_llsc(); + return old; +} +#endif + +#ifndef a_fetch_add +#define a_fetch_add a_fetch_add +static inline int a_fetch_add(volatile int *p, int v) +{ + int old; + a_pre_llsc(); + do old = a_ll(p); + while (!a_sc(p, (unsigned)old + v)); + a_post_llsc(); + return old; +} +#endif + +#ifndef a_fetch_and +#define a_fetch_and a_fetch_and +static inline int a_fetch_and(volatile int *p, int v) +{ + int old; + a_pre_llsc(); + do old = a_ll(p); + while (!a_sc(p, old & v)); + a_post_llsc(); + return old; +} +#endif + +#ifndef a_fetch_or +#define a_fetch_or a_fetch_or +static inline int a_fetch_or(volatile int *p, int v) +{ + int old; + a_pre_llsc(); + do old = a_ll(p); + while (!a_sc(p, old | v)); + a_post_llsc(); + return old; +} +#endif + +#endif + +#ifdef a_ll_p + +#ifndef a_cas_p +#define a_cas_p a_cas_p +static inline void *a_cas_p(volatile void *p, void *t, void *s) +{ + void *old; + a_pre_llsc(); + do old = a_ll_p(p); + while (old==t && !a_sc_p(p, s)); + a_post_llsc(); + return old; +} +#endif + +#endif + +#ifndef a_cas +#error missing definition of a_cas +#endif + +#ifndef a_swap +#define a_swap a_swap +static inline int a_swap(volatile int *p, int v) +{ + int old; + do old = *p; + while (a_cas(p, old, v) != old); + return old; +} +#endif + +#ifndef a_fetch_add +#define a_fetch_add a_fetch_add +static inline int a_fetch_add(volatile int *p, int v) +{ + int old; + do old = *p; + while (a_cas(p, old, (unsigned)old+v) != old); + return old; +} +#endif + +#ifndef a_fetch_and +#define a_fetch_and a_fetch_and +static inline int a_fetch_and(volatile int *p, int v) +{ + int old; + do old = *p; + while (a_cas(p, old, old&v) != old); + return old; +} +#endif +#ifndef a_fetch_or +#define a_fetch_or a_fetch_or +static inline int a_fetch_or(volatile int *p, int v) +{ + int old; + do old = *p; + while (a_cas(p, old, old|v) != old); + return old; +} +#endif + +#ifndef a_and +#define a_and a_and +static inline void a_and(volatile int *p, int v) +{ + a_fetch_and(p, v); +} +#endif + +#ifndef a_or +#define a_or a_or +static inline void a_or(volatile int *p, int v) +{ + a_fetch_or(p, v); +} +#endif + +#ifndef a_inc +#define a_inc a_inc +static inline void a_inc(volatile int *p) +{ + a_fetch_add(p, 1); +} +#endif + +#ifndef a_dec +#define a_dec a_dec +static inline void a_dec(volatile int *p) +{ + a_fetch_add(p, -1); +} +#endif + +#ifndef a_store +#define a_store a_store +static inline void a_store(volatile int *p, int v) +{ +#ifdef a_barrier + a_barrier(); + *p = v; + a_barrier(); +#else + a_swap(p, v); +#endif +} +#endif + +#ifndef a_barrier +#define a_barrier a_barrier +static void a_barrier() +{ + volatile int tmp = 0; + a_cas(&tmp, 0, 0); +} +#endif + +#ifndef a_spin +#define a_spin a_barrier +#endif + +#ifndef a_and_64 +#define a_and_64 a_and_64 +static inline void a_and_64(volatile uint64_t *p, uint64_t v) +{ + union { uint64_t v; uint32_t r[2]; } u = { v }; + if (u.r[0]+1) a_and((int *)p, u.r[0]); + if (u.r[1]+1) a_and((int *)p+1, u.r[1]); +} +#endif + +#ifndef a_or_64 +#define a_or_64 a_or_64 +static inline void a_or_64(volatile uint64_t *p, uint64_t v) +{ + union { uint64_t v; uint32_t r[2]; } u = { v }; + if (u.r[0]) a_or((int *)p, u.r[0]); + if (u.r[1]) a_or((int *)p+1, u.r[1]); +} +#endif + +#ifndef a_cas_p +typedef char a_cas_p_undefined_but_pointer_not_32bit[-sizeof(char) == 0xffffffff ? 1 : -1]; +#define a_cas_p a_cas_p +static inline void *a_cas_p(volatile void *p, void *t, void *s) +{ + return (void *)a_cas((volatile int *)p, (int)t, (int)s); +} +#endif + +#ifndef a_or_l +#define a_or_l a_or_l +static inline void a_or_l(volatile void *p, long v) +{ + if (sizeof(long) == sizeof(int)) a_or(p, v); + else a_or_64(p, v); +} +#endif + +#ifndef a_crash +#define a_crash a_crash +static inline void a_crash() +{ + *(volatile char *)0=0; +} +#endif + +#ifndef a_ctz_32 +#define a_ctz_32 a_ctz_32 +static inline int a_ctz_32(uint32_t x) +{ +#ifdef a_clz_32 + return 31-a_clz_32(x&-x); +#else + static const char debruijn32[32] = { + 0, 1, 23, 2, 29, 24, 19, 3, 30, 27, 25, 11, 20, 8, 4, 13, + 31, 22, 28, 18, 26, 10, 7, 12, 21, 17, 9, 6, 16, 5, 15, 14 + }; + return debruijn32[(x&-x)*0x076be629 >> 27]; +#endif +} +#endif + +#ifndef a_ctz_64 +#define a_ctz_64 a_ctz_64 +static inline int a_ctz_64(uint64_t x) +{ + static const char debruijn64[64] = { + 0, 1, 2, 53, 3, 7, 54, 27, 4, 38, 41, 8, 34, 55, 48, 28, + 62, 5, 39, 46, 44, 42, 22, 9, 24, 35, 59, 56, 49, 18, 29, 11, + 63, 52, 6, 26, 37, 40, 33, 47, 61, 45, 43, 21, 23, 58, 17, 10, + 51, 25, 36, 32, 60, 20, 57, 16, 50, 31, 19, 15, 30, 14, 13, 12 + }; + if (sizeof(long) < 8) { + uint32_t y = x; + if (!y) { + y = x>>32; + return 32 + a_ctz_32(y); + } + return a_ctz_32(y); + } + return debruijn64[(x&-x)*0x022fdd63cc95386dull >> 58]; +} +#endif + +static inline int a_ctz_l(unsigned long x) +{ + return (sizeof(long) < 8) ? a_ctz_32(x) : a_ctz_64(x); +} + +#ifndef a_clz_64 +#define a_clz_64 a_clz_64 +static inline int a_clz_64(uint64_t x) +{ +#ifdef a_clz_32 + if (x>>32) + return a_clz_32(x>>32); + return a_clz_32(x) + 32; +#else + uint32_t y; + int r; + if (x>>32) y=x>>32, r=0; else y=x, r=32; + if (y>>16) y>>=16; else r |= 16; + if (y>>8) y>>=8; else r |= 8; + if (y>>4) y>>=4; else r |= 4; + if (y>>2) y>>=2; else r |= 2; + return r | !(y>>1); +#endif +} +#endif + +#endif diff --git a/libs/libglibc-compatibility/musl/atomic_arch.h b/libs/libglibc-compatibility/musl/atomic_arch.h new file mode 100644 index 00000000000..da4e2037548 --- /dev/null +++ b/libs/libglibc-compatibility/musl/atomic_arch.h @@ -0,0 +1,123 @@ +#define a_cas a_cas +static inline int a_cas(volatile int *p, int t, int s) +{ + __asm__ __volatile__ ( + "lock ; cmpxchg %3, %1" + : "=a"(t), "=m"(*p) : "a"(t), "r"(s) : "memory" ); + return t; +} + +#define a_cas_p a_cas_p +static inline void *a_cas_p(volatile void *p, void *t, void *s) +{ + __asm__( "lock ; cmpxchg %3, %1" + : "=a"(t), "=m"(*(void *volatile *)p) + : "a"(t), "r"(s) : "memory" ); + return t; +} + +#define a_swap a_swap +static inline int a_swap(volatile int *p, int v) +{ + __asm__ __volatile__( + "xchg %0, %1" + : "=r"(v), "=m"(*p) : "0"(v) : "memory" ); + return v; +} + +#define a_fetch_add a_fetch_add +static inline int a_fetch_add(volatile int *p, int v) +{ + __asm__ __volatile__( + "lock ; xadd %0, %1" + : "=r"(v), "=m"(*p) : "0"(v) : "memory" ); + return v; +} + +#define a_and a_and +static inline void a_and(volatile int *p, int v) +{ + __asm__ __volatile__( + "lock ; and %1, %0" + : "=m"(*p) : "r"(v) : "memory" ); +} + +#define a_or a_or +static inline void a_or(volatile int *p, int v) +{ + __asm__ __volatile__( + "lock ; or %1, %0" + : "=m"(*p) : "r"(v) : "memory" ); +} + +#define a_and_64 a_and_64 +static inline void a_and_64(volatile uint64_t *p, uint64_t v) +{ + __asm__ __volatile( + "lock ; and %1, %0" + : "=m"(*p) : "r"(v) : "memory" ); +} + +#define a_or_64 a_or_64 +static inline void a_or_64(volatile uint64_t *p, uint64_t v) +{ + __asm__ __volatile__( + "lock ; or %1, %0" + : "=m"(*p) : "r"(v) : "memory" ); +} + +#define a_inc a_inc +static inline void a_inc(volatile int *p) +{ + __asm__ __volatile__( + "lock ; incl %0" + : "=m"(*p) : "m"(*p) : "memory" ); +} + +#define a_dec a_dec +static inline void a_dec(volatile int *p) +{ + __asm__ __volatile__( + "lock ; decl %0" + : "=m"(*p) : "m"(*p) : "memory" ); +} + +#define a_store a_store +static inline void a_store(volatile int *p, int x) +{ + __asm__ __volatile__( + "mov %1, %0 ; lock ; orl $0,(%%rsp)" + : "=m"(*p) : "r"(x) : "memory" ); +} + +#define a_barrier a_barrier +static inline void a_barrier() +{ + __asm__ __volatile__( "" : : : "memory" ); +} + +#define a_spin a_spin +static inline void a_spin() +{ + __asm__ __volatile__( "pause" : : : "memory" ); +} + +#define a_crash a_crash +static inline void a_crash() +{ + __asm__ __volatile__( "hlt" : : : "memory" ); +} + +#define a_ctz_64 a_ctz_64 +static inline int a_ctz_64(uint64_t x) +{ + __asm__( "bsf %1,%0" : "=r"(x) : "r"(x) ); + return x; +} + +#define a_clz_64 a_clz_64 +static inline int a_clz_64(uint64_t x) +{ + __asm__( "bsr %1,%0 ; xor $63,%0" : "=r"(x) : "r"(x) ); + return x; +} diff --git a/libs/libglibc-compatibility/musl/sched_getcpu.c b/libs/libglibc-compatibility/musl/sched_getcpu.c new file mode 100644 index 00000000000..d0e171dac95 --- /dev/null +++ b/libs/libglibc-compatibility/musl/sched_getcpu.c @@ -0,0 +1,45 @@ +#define _GNU_SOURCE +#include +#include +#include +#include "syscall.h" +#include "atomic.h" + +#ifdef VDSO_GETCPU_SYM + +void *__vdsosym(const char *, const char *); + +static void *volatile vdso_func; + +typedef long (*getcpu_f)(unsigned *, unsigned *, void *); + +static long getcpu_init(unsigned *cpu, unsigned *node, void *unused) +{ + void *p = __vdsosym(VDSO_GETCPU_VER, VDSO_GETCPU_SYM); + getcpu_f f = (getcpu_f)p; + a_cas_p(&vdso_func, (void *)getcpu_init, p); + return f ? f(cpu, node, unused) : -ENOSYS; +} + +static void *volatile vdso_func = (void *)getcpu_init; + +#endif + +int sched_getcpu(void) +{ + int r; + unsigned cpu; + +#ifdef VDSO_GETCPU_SYM + getcpu_f f = (getcpu_f)vdso_func; + if (f) { + r = f(&cpu, 0, 0); + if (!r) return cpu; + if (r != -ENOSYS) return __syscall_ret(r); + } +#endif + + r = __syscall(SYS_getcpu, &cpu, 0, 0); + if (!r) return cpu; + return __syscall_ret(r); +} diff --git a/libs/libglibc-compatibility/musl/timespec_get.c b/libs/libglibc-compatibility/musl/timespec_get.c new file mode 100644 index 00000000000..03c5a77b7b0 --- /dev/null +++ b/libs/libglibc-compatibility/musl/timespec_get.c @@ -0,0 +1,12 @@ +#include + +int __clock_gettime(clockid_t, struct timespec *); + +/* There is no other implemented value than TIME_UTC; all other values + * are considered erroneous. */ +int timespec_get(struct timespec * ts, int base) +{ + if (base != TIME_UTC) return 0; + int ret = __clock_gettime(CLOCK_REALTIME, ts); + return ret < 0 ? 0 : base; +} From cd4f93ff2a4acd0b59ac655c18bb776e50442294 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Jan 2019 00:41:59 +0300 Subject: [PATCH 41/71] Added missing glibc compatibility functions --- libs/libglibc-compatibility/musl/timespec_get.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/libs/libglibc-compatibility/musl/timespec_get.c b/libs/libglibc-compatibility/musl/timespec_get.c index 03c5a77b7b0..4a01a49e0a8 100644 --- a/libs/libglibc-compatibility/musl/timespec_get.c +++ b/libs/libglibc-compatibility/musl/timespec_get.c @@ -1,12 +1,12 @@ #include -int __clock_gettime(clockid_t, struct timespec *); +int clock_gettime(clockid_t, struct timespec *); /* There is no other implemented value than TIME_UTC; all other values * are considered erroneous. */ int timespec_get(struct timespec * ts, int base) { if (base != TIME_UTC) return 0; - int ret = __clock_gettime(CLOCK_REALTIME, ts); + int ret = clock_gettime(CLOCK_REALTIME, ts); return ret < 0 ? 0 : base; } From 0fd1eca8cc1d9e758c203a7a6a998b82f051c578 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Jan 2019 01:43:49 +0300 Subject: [PATCH 42/71] Fixed test --- .../00024_random_counters.reference | 2000 ++++++++--------- .../1_stateful/00024_random_counters.sql | 2000 ++++++++--------- 2 files changed, 2000 insertions(+), 2000 deletions(-) diff --git a/dbms/tests/queries/1_stateful/00024_random_counters.reference b/dbms/tests/queries/1_stateful/00024_random_counters.reference index 3b181f11e2a..f11b66aa5b5 100644 --- a/dbms/tests/queries/1_stateful/00024_random_counters.reference +++ b/dbms/tests/queries/1_stateful/00024_random_counters.reference @@ -1,1000 +1,1000 @@ -1 1 2 35 -1 1 1 0 -1 1 1 0 -1 1 1 11 -6 8 22 1926 -1 1 3 4 -1 1 1 23 -1 1 1 0 -1 3 31 1762 -1 2 9 2340 -4 4 4 0 -4 6 40 2088 -1 1 3 9 -1 1 1 11 -1 1 1 11 -3 3 3 0 -5 5 9 298 -2 2 2 58 -5 8 20 294 -82 93 250 25679 -12 12 66 1506 -1 2 12 1192 -1 1 1 11 -1 1 23 702 -1 1 1 0 -1 1 8 335 -1 1 1 0 -1 1 3 217 -3 3 4 278 -1 1 1 12 -7 12 25 594 -1 1 7 1496 -5 5 12 289 -1 1 2 60 -1 1 1 0 -1 1 1 11 -5 5 24 1871 -3 3 3 33 -3 3 7 65 -1 1 2 31 -1 1 11 448 -1 1 1 11 -7 9 33 1836 -1 1 7 301 -2 2 30 627 -7 7 9 158 -6 8 19 2809 -3 4 14 684 -1 1 1 0 -1 1 1 11 -1 1 1 0 -1 1 1 11 -2 2 4 47 -3 3 3 0 -2 2 28 1703 -7 7 13 94 -2 2 14 416 -2 2 2 32 -2 3 39 2194 -1 1 8 1002 -8 9 20 1033 -2 2 4 253 -3 3 5 986 -1 1 1 8 -1 1 1 0 -4 4 15 444 -3 3 3 45 -2 2 4 70 -1 1 7 139 -1 1 1 16 -6 7 38 1986 -6 13 21 1237 -26 34 49 3819 -27 29 40 882 -1 3 3 5 -1 1 1 11 -1 1 11 276 -1 2 2 22 -1 1 3 11 -1 2 6 189 -4 8 12 50 -1 1 3 30 -1 3 3 0 -1 1 1 11 -1 1 1 11 -1 1 1 11 -5 9 12 287 -1 1 1 0 -1 1 1 11 -2 2 6 848 -1 1 2 96 -1 1 1 0 -1 1 1 121 -2 2 2 0 -1 1 1 0 -2 2 2 19 -2 2 8 305 -2 2 2 19 -12 12 67 7827 -1 1 6 49 -1 1 1 11 -2 2 2 30 -13 48 162 5879 -1 1 1 0 -14 16 99 3428 -1 2 7 645 -4 5 14 1062 -1 1 1 11 -164 212 986 73006 -5 6 6 47 -1 1 2 20 -58 77 102 9209 -1 1 1 11 -7 7 12 187 -1 1 2 96 -1 1 1 11 -12 15 31 4746 -2 2 2 19 -1 2 2 23 -1 1 3 53 -1 1 1 11 -2 5 6 117 -2 2 4 49 -14 34 136 16428 -1 1 2 54 -1 1 5 1115 -1 1 1 0 -7 7 8 21 -1 1 1 0 -28 31 66 298 -3 5 5 0 -6 10 43 2347 -5 5 5 52 -4 4 46 2400 -5 5 5 0 -1 1 1 11 -1 1 1 17 -24 37 124 12370 -1 1 1 0 -1 1 1 11 -2 3 3 0 -3 3 9 39 -1 1 1 0 -1 1 2 45 -19 25 210 9807 -7 7 7 60 -1 1 1 19 -1 1 1 0 -1 1 1 0 -1 1 6 57 -2 3 5 369 -1 1 1 0 -1 4 4 38 -1 1 1 0 -9 24 82 5362 -2 2 2 0 -3 3 5 174 -2 3 4 1761 -2 2 4 24 -2 3 23 848 -21 28 60 5255 -4 4 6 112 -2 2 2 0 -125 148 303 22391 -689 2520 4204 800685 -6 6 6 0 -8 8 12 712 -1 1 6 108 -2 2 24 528 -5 7 94 5924 -1 1 1 0 -1 3 4 1822 -346 684 1504 146668 -11 14 33 1210 -1 1 1 0 -2 2 5 343 -1 1 6 277 -2 2 2 4 -3 4 4 37 -21 26 105 5345 -1 1 1 0 -1 1 1 0 -1 1 1 0 -1 1 1 0 -1 1 11 1202 -2 3 3 22 -3 3 3 0 -1 3 7 41 -6 6 29 2928 -1 1 1 0 -4 7 26 2243 -2 2 2 0 -1 1 2 42 -2 2 7 911 -1 1 2 92 -1 1 1 11 -121 272 1804 81031 -2 2 2 29 -1 1 1 0 -1 1 6 8 -1 1 1 0 -2 2 10 575 -1 1 1 0 -3 22 167 16008 -1 1 1 11 -3 3 3 33 -1 2 8 33 -1 1 1 22 -9 15 17 1289 -1 1 4 484 -1 1 1 0 -16 17 41 1687 -1 1 1 26 -1 1 6 143 -8 20 72 2685 -1 1 14 392 -1 1 1 11 -1 14 14 167 -11 11 22 894 -1 1 6 59 -1 1 1 8 -17 23 86 6438 -1 5 27 1613 -1 1 1 19 -17 26 44 5994 -1 1 1 11 -1 2 6 34 -1 1 3 12 -4 4 14 688 -2 2 20 384 -2 2 4 132 -14 37 95 5108 -4 5 7 193 -3 3 7 1324 -1 1 1 0 -1 2 4 1981 -2 2 4 66 -39 48 78 9745 -1 17 165 34059 -1 1 1 11 -4 35 539 46431 -1 1 1 19 -1 1 1 8 -1 1 3 275 -1 1 1 0 -1 1 1 11 -1 27 41 9920 -1 1 2 35 -1 1 1 253 -1 1 1 11 -4 5 12 1385 -1 1 2 216 -2 4 25 1332 -6 6 14 1193 -1 1 1 22 -3 5 5 11 -1 1 4 88 -1 1 1 0 -2 2 2 0 -1 1 1 19 -5 5 5 55 -1 1 1 0 -1 1 3 158 -1 1 3 5 -4 4 27 290 -1 1 1 11 -1 1 3 13 -2 2 2 0 -8 8 36 3345 -3 4 18 913 -1 1 1 11 -1 2 4 1637 -1 1 5 209 -1 4 19 3312 -1 1 3 23 -1 1 7 1990 -2 2 7 444 -2 2 2 119 -2 2 2 11 -4 6 44 3147 -2 2 2 11 -2 2 4 33 -1 1 1 18 -1 1 1 11 -31 83 230 25720 -30 38 98 6870 -2 2 3 29 -1 1 1 0 -1 1 6 178 -1 4 9 415 -32 37 45 4039 -1 1 1 11 -4 4 9 1568 -1 1 1 0 -38 44 81 2938 -5 14 25 655 -5 7 7 0 -5 5 7 1217 -1 1 3 121 -3 4 9 281 -24 34 106 9053 -5 7 9 56 -2 2 2 19 -10 10 10 97 -4 4 4 51 -1 1 6 165 -64 77 347 15480 -28 42 75 12045 -3 4 5 65 -1 1 1 11 -3 6 6 19 -1 1 1 0 -2 2 3 492 -6 6 12 120 -1 1 1 18 -6 6 8 118 -1 1 1 0 -21 31 119 4907 -1 1 3 371 -1 1 1 0 -1 1 3 110 -1 1 7 242 -1 2 28 343 -19 21 44 1143 -53 57 91 4678 -1 1 1 0 -29 33 252 11991 -2 2 2 11 -115 128 188 12022 -1 1 1 0 -5 7 26 1365 -1 1 10 59 -3 3 11 947 -1 1 4 226 -1 1 2 37 -30 48 80 3567 -1 1 1 0 -1 2 2 34 -38 58 165 13686 -3 3 25 1621 -1 1 12 955 -2 2 2 0 -16 17 31 1597 -3 3 3 0 -2 2 4 1025 -13 20 45 1533 -2 2 16 458 -3 5 11 789 -2 2 2 0 -4 5 5 78 -213 425 1385 117704 -13 15 26 2345 -2 2 3 66 -3 3 10 100 -1 1 1 0 -1 1 7 108 -1 1 11 882 -5 7 14 146 -1 1 1 0 -5 5 9 4035 -1 1 1 11 -1 2 2 22 -1 2 2 0 -16 17 35 4198 -3 3 3 0 -1 1 1 11 -1 1 1 0 -1 1 1 11 -3 3 9 438 -1 1 11 435 -38 50 150 24709 -3 6 28 667 -1 1 1 0 -6 9 38 1405 -45 57 358 35596 -2 2 2 42 -3 3 3 8 -1 1 1 11 -9 13 57 10251 -3 3 7 719 -9 10 12 90 -1 1 1 11 -6 6 9 476 -1 1 1 11 -1 2 3 234 -1 3 5 84 -1 1 1 11 -1 3 15 802 -1 1 1 0 -1 1 1 11 -1 1 1 11 -1 2 10 205 -1 1 1 0 -2 2 2 11 -3 3 5 98 -1 1 1 8 -1 1 3 54 -15 16 27 3419 -1 1 1 8 -3 6 22 2861 -1 1 7 76 -1 1 1 11 -1 2 2 0 -1 2 8 63 -6 8 12 484 -1 1 1 19 -12 24 113 6755 -3 5 39 2528 -1 1 1 0 -2 2 2 0 -2 5 143 2710 -2 2 2 22 -1 1 1 11 -3 3 5 78 -1 1 1 0 -3 3 3 11 -2 2 2 32 -81 87 124 42448 -1 1 1 11 -1 1 1 8 -1 1 1 0 -3 3 31 421 -1 1 2 117 -1 1 1 0 -1 27 38 5645 -1 1 1 11 -1 1 2 57 -1 1 2 49 -20 25 34 1305 -1 1 12 2165 -6 7 23 3702 -1 1 3 10 -35 35 126 10411 -5 8 66 3619 -1 2 6 253 -5 5 7 289 -3 3 5 538 -1 1 1 0 -1 1 1 0 -1 1 1 11 -1 1 3 18 -1 1 1 0 -3 3 4 57 -1 1 1 13 -5 5 7 117 -1 1 2 118 -5 5 5 21 -1 1 1 11 -1 1 1 0 -3 4 10 287 -4 4 9 2430 -1 1 2 30 -3 3 3 41 -19 20 48 1815 -1 1 1 21 -1 1 1 8 -1 3 11 1380 -1 1 1 0 -1 1 1 0 -3 3 25 764 -1 1 7 102 -1 1 1 11 -41 99 317 41164 -1 2 5 2468 -3 5 9 154 -1 1 6 107 -2 4 28 1212 -1 1 1 0 -12 17 19 277 -2 2 5 114 -1 1 9 798 -17 22 57 1481 -1 1 1 0 -2 2 7 2935 -3 4 10 1736 -4 4 26 1448 -1 1 3 91 -1 1 1 0 -1 1 1 0 -1 1 1 0 -1 1 3 47 -2 2 2 0 -1 1 2 83 -1 1 1 21 -2 2 11 763 -2 3 3 35 -33 39 84 4160 -1 1 1 11 -17 18 56 617 -1 1 6 110 -5 5 5 66 -1 1 1 8 -1 1 7 1333 -1 1 1 11 -4 4 12 1263 -5 5 20 4238 -1 1 1 11 -1 1 1 11 -3 4 11 1662 -1 1 1 0 -1 1 1 10 -3 3 3 30 -1 1 1 11 -1 1 1 11 -5 5 19 4103 -2 4 4 11 -2 2 2 0 -6 7 45 4746 -1 1 3 809 -1 2 110 2211 -1 1 1 0 -1 1 1 0 -1 1 3 813 -1 1 1 11 -6 6 14 1117 -2 3 18 993 -3 3 14 2668 -4 4 15 1046 -55 56 56 829 -2 2 2 0 -1 2 2 40 -1 1 1 0 -1 1 1 11 -1 1 3 1407 -3 3 8 206 -1 1 1 0 -1 2 3 373 -2 2 5 53 -11 11 11 0 -13 32 80 10316 -1 1 1 8 -2 2 2 0 -14 21 129 6740 -1 2 31 1035 -18 22 182 6324 -14 15 73 2943 -1 1 13 254 -1 2 12 599 -1 1 1 0 -1 1 1 0 -1 1 1 0 -3 3 8 106 -1 1 1 95 -1 1 1 0 -5 20 1460 96399 -7 8 16 202 -16 18 43 1602 -3 3 3 0 -1 2 4 52 -1 1 1 11 -1 1 1 11 -3 19 266 27049 -1 1 1 8 -1 1 1 11 -17 25 30 3561 -1 1 1 8 -1 1 1 11 -1 2 9 165 -1 1 1 11 -4 5 5 49 -97 100 185 6082 -3 5 31 1387 -3 4 31 4777 -1 1 1 0 -2 2 8 273 -1 1 1 20 -1 1 1 15 -58 65 101 3789 -4 5 7 41 -1 1 7 847 -1 2 2 0 -1 1 1 0 -1 1 3 4 -4 4 10 789 -1 1 1 0 -5 5 8 1200 -1 1 16 136 -1 1 1 0 -1 1 1 11 -1 1 1 11 -2 2 5 193 -1 4 4 0 -1 1 1 0 -1 1 3 110 -1 1 4 127 -6 7 10 2957 -2 2 4 52 -1 1 5 880 -2 2 3 168 -3 6 38 3866 -1 1 1 19 -1 1 1 11 -1 1 12 784 -1 1 3 20 -1 1 1 19 -1 1 1 9 -1 2 6 1026 -1 1 1 20 -1 1 1 11 -1 1 1 9 -1 1 8 281 -2 2 2 11 -1 3 3 33 -2 3 4 54 -2 2 2 38 -1 3 3 0 -1 1 1 19 -4 4 6 21 -1 4 4 50 -1 1 1 8 -1 1 1 0 -4 4 4 0 -1 1 1 19 -65 81 166 13564 -161 261 434 16985 -1 1 1 0 -1 7 7 63 -3 3 7 156 -2 5 5 0 -1 1 3 18 -1 1 1 0 -1 1 1 79 -35 36 424 8362 -1 1 7 689 -1 1 3 1994 -6 6 7 642 -3 3 3 28 -2 4 4 0 -1 7 48 1043 -2 3 3 11 -3 3 10 229 -1 1 1 0 -1 2 111 1932 -9 12 26 2788 -13 22 37 1697 -32 33 37 1452 -1 1 3 1507 -21 41 73 4716 -1 1 1 0 -1 1 1 0 -1 1 1 15 -1 1 1 18 -1 1 1 16 -4 4 4 173 -8 10 10 0 -4 4 6 63 -2 4 10 204 -2 2 2 19 -1 1 1 0 -1 1 1 0 -1 1 1 11 -1 1 1 0 -2 2 4 15 -2 3 10 123 -1 1 2 173 -1 1 1 11 -21 25 63 2482 -12 12 13 743 -40 82 182 10634 -1 1 2 173 -2 2 4 62 -149 159 243 12601 -2 2 3 129 -1 1 7 793 -2 11 13 20 -1 1 2 63 -1 1 1 11 -1 1 1 20 -1 1 1 11 -1 1 1 0 -3 3 8 1473 -1 2 4 51 -3 3 3 22 -1 1 10 1785 -8 9 10 67 -58 59 85 12924 -5 10 16 3338 -1 1 1 0 -1 2 2 44 -2 2 4 202 -1 1 7 291 -1 1 5 329 -3 3 7 1705 -1 1 1 11 -12 18 75 6272 -1 1 1 11 -4 4 4 63 -2 3 3 0 -1 3 7 135 -1 1 1 19 -1 1 1 63 -18 21 42 5809 -114 122 298 13665 -2 2 2 26 -1 1 1 21 -15 18 25 882 -1 1 1 0 -1 13 13 0 -8 9 29 2258 -20 22 26 189 -1 1 1 11 -5 7 7 0 -1 1 3 92 -1 1 6 321 -1 1 14 810 -4 5 14 287 -2 2 6 936 -5 5 5 61 -2 2 2 0 -1 1 1 19 -1 6 13 660 -2 2 2 0 -3 4 12 224 -1 1 1 0 -3 3 3 17 -4 4 7 65 -2 2 2 8 -26 30 39 844 -2 2 2 0 -3 3 5 37 -3 3 28 1226 -1 1 1 2 -1 1 3 259 -1 1 1 0 -3 4 26 477 -1 1 3 62 -3 3 38 1208 -37 45 201 6855 -1 1 2 110 -1 1 1 0 -1 1 3 1235 -1 1 1 21 -3 3 3 47 -1 1 6 1743 -1 1 1 11 -1 1 1 31 -31 39 176 8502 -1 1 1 0 -1 1 26 191 -1 1 1 0 -6 6 24 2502 -4 4 7 190 -3 3 7 1313 -1 1 2 21 -2 2 2 0 -1 1 2 57 -1 18 18 203 -1 1 1 8 -2 2 2 0 -2 2 2 79 -1 1 1 0 -1 1 1 11 -1 2 2 22 -2 2 3 2464 -1 1 3 55 -1 1 1 0 -1 1 1 11 -1 1 1 0 -1 1 3 99 -6 16 30 9372 -1 1 1 11 -1 1 1 0 -1 1 5 339 -1 1 1 11 -4 4 5 390 -24 101 608 51846 -3 3 12 599 -1 1 2 43 -1 1 1 28 -2 2 2 11 -1 1 1 0 -3 3 11 944 -2 6 11 3996 -2 2 5 102 -1 1 2 419 -1 1 1 0 -1 1 1 21 -1 1 1 11 -10 14 115 4031 -3 4 6 141 -6 10 69 1642 -7 8 26 435 -1 1 1 0 -1 1 1 0 -4 5 25 219 -1 1 7 157 -4 4 8 187 -1 1 5 505 -2 2 4 91 -1 4 19 3531 -1 2 2 0 -1 1 6 556 -1 1 1 11 -1 2 2 59 -1 1 6 35 -2 3 11 894 -2 2 4 68 -1 1 1 0 -1 1 1 0 -1 1 1 19 -2 3 6 1644 -1 1 1 11 -5 7 20 195 -57 75 422 46687 -1 1 1 11 -488 786 2802 174088 -1 1 1 32 -65 117 278 31034 -1 1 3 27 -131 136 174 6766 -7 9 15 1894 -1 1 5 293 -2 2 4 52 -3 3 5 16 -1 1 1 0 -1 1 3 102 -1 1 1 0 -1 1 7 127 -2 2 2 0 -1 1 3 641 -4 4 4 19 -2 2 4 122 -10 10 94 1765 -1 1 1 0 -3 3 6 185 -1 1 2 231 -1 1 1 11 -2 2 7 95 -1 1 1 23 -3 3 7 605 -2 3 11 55 -2 2 6 213 -5 10 26 2117 -1 1 4 77 -1 1 1 0 -7 10 19 1257 -3 3 3 0 -1 2 4 1798 -1 1 1 0 -1 1 1 11 -13 18 48 2205 -4 4 4 56 -1 1 2 164 -3 3 7 90 -1 1 1 11 -12 14 26 1101 -3 3 12 668 -1 1 6 205 -14 17 96 3247 -2 2 2 33 -3 3 3 0 -1 1 1 0 -6 8 21 1440 -1 1 3 151 -9 9 56 7312 -1 1 1 11 -1 1 14 59 -1 1 1 0 -2 2 2 0 -5 8 36 708 -1 1 1 19 -4 5 30 1111 -7 8 25 1843 -1 1 1 11 -6 12 38 3316 -2 2 2 0 -5 5 7 115 -1 1 3 10 -1 9 417 25980 -2 3 3 0 -12 12 12 410 -3 3 12 464 -1 1 3 30 -29 33 121 16070 -1 3 3 22 -1 1 1 19 -1 1 2 658 -1 1 1 8 -3 3 3 0 -1 1 1 11 -1 1 1 25 -2 2 8 751 -1 1 1 0 -2 2 4 590 -1 1 1 0 -1 1 1 11 -1 1 2 126 -20 21 31 3498 -4 4 4 30 -1 1 1 11 -1 1 3 35 -1 1 2 11 -1 1 4 50 -5 5 8 254 -3 3 3 0 -1 1 3 915 -2 3 3 50 -2 2 5 1097 -1 1 1 11 -1 2 11 510 -3 3 12 1310 -1 1 3 62 -3 3 3 34 -50 62 566 32493 -1 1 6 291 -2 2 6 1261 -2 2 6 95 -43 55 207 12664 -10 11 13 91 -1 1 2 458 -5 5 12 648 -1 1 1 11 -3 3 32 784 -2 2 2 0 -35 121 376 38864 -2 9 9 0 -1 1 10 191 -1 1 1 1 -1 1 1 0 -1 3 6 188 -1 1 1 11 -1 1 1 0 -1 1 1 11 -3 4 4 22 -2 2 2 0 -1 1 1 11 -5 6 11 805 -2 2 2 0 -1 2 97 1587 -2 3 29 2991 -2 2 4 113 -1 2 4 129 -2 3 10 224 -1 1 7 2870 -1 1 1 0 -19 37 54 3931 -1 1 1 0 -2 2 31 902 -1 1 1 11 -2 2 4 63 -3 3 3 0 -5 5 7 38 -2 2 9 137 -1 1 6 108 -1 1 3 22 -1 1 3 6 -34 46 491 18726 -6 6 12 1409 -3 4 20 1007 -1 3 61 9155 -1 1 1 0 -1 1 1 0 -114 292 1192 99949 -6 6 14 1317 -2 4 14 8244 -1 1 1 0 -2 2 5 76 -2 2 14 334 -1 1 1 0 -40 53 154 15716 -1 1 1 0 -6 7 10 1995 -3 3 6 226 -1 1 4 77 -1 2 7 3499 -1 1 1 11 -1 1 1 19 -3 4 8 188 -25 31 138 4193 -5 5 9 35 -9 24 38 6644 -1 1 1 11 -2 2 5 361 -16 34 43 5946 -1 1 1 0 -5 6 8 66 -23 30 70 4786 -3 4 22 2926 -1 1 3 28 -6 8 9 160 -8 11 68 6594 -1 1 2 253 -9 9 16 907 -3 3 3 22 -3 3 4 78 -1 1 14 411 -1 1 1 11 -5 6 15 390 -2 2 2 0 -1 1 1 0 -3 8 8 22 -2 2 2 0 -1 1 8 3835 -1 1 1 0 -2 2 2 0 -4 5 17 300 -57 63 166 5264 -1 1 6 944 -1 2 2 0 -1 1 1 0 +1 1 +1 1 +1 1 +1 1 +6 8 +1 1 +1 1 +1 1 +1 3 +1 2 +4 4 +4 6 +1 1 +1 1 +1 1 +3 3 +5 5 +2 2 +5 8 +82 93 +12 12 +1 2 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +3 3 +1 1 +7 12 +1 1 +5 5 +1 1 +1 1 +1 1 +5 5 +3 3 +3 3 +1 1 +1 1 +1 1 +7 9 +1 1 +2 2 +7 7 +6 8 +3 4 +1 1 +1 1 +1 1 +1 1 +2 2 +3 3 +2 2 +7 7 +2 2 +2 2 +2 3 +1 1 +8 9 +2 2 +3 3 +1 1 +1 1 +4 4 +3 3 +2 2 +1 1 +1 1 +6 7 +6 13 +26 34 +27 29 +1 3 +1 1 +1 1 +1 2 +1 1 +1 2 +4 8 +1 1 +1 3 +1 1 +1 1 +1 1 +5 9 +1 1 +1 1 +2 2 +1 1 +1 1 +1 1 +2 2 +1 1 +2 2 +2 2 +2 2 +12 12 +1 1 +1 1 +2 2 +13 48 +1 1 +14 16 +1 2 +4 5 +1 1 +164 212 +5 6 +1 1 +58 77 +1 1 +7 7 +1 1 +1 1 +12 15 +2 2 +1 2 +1 1 +1 1 +2 5 +2 2 +14 34 +1 1 +1 1 +1 1 +7 7 +1 1 +28 31 +3 5 +6 10 +5 5 +4 4 +5 5 +1 1 +1 1 +24 37 +1 1 +1 1 +2 3 +3 3 +1 1 +1 1 +19 25 +7 7 +1 1 +1 1 +1 1 +1 1 +2 3 +1 1 +1 4 +1 1 +9 24 +2 2 +3 3 +2 3 +2 2 +2 3 +21 28 +4 4 +2 2 +125 148 +689 2520 +6 6 +8 8 +1 1 +2 2 +5 7 +1 1 +1 3 +346 684 +11 14 +1 1 +2 2 +1 1 +2 2 +3 4 +21 26 +1 1 +1 1 +1 1 +1 1 +1 1 +2 3 +3 3 +1 3 +6 6 +1 1 +4 7 +2 2 +1 1 +2 2 +1 1 +1 1 +121 272 +2 2 +1 1 +1 1 +1 1 +2 2 +1 1 +3 22 +1 1 +3 3 +1 2 +1 1 +9 15 +1 1 +1 1 +16 17 +1 1 +1 1 +8 20 +1 1 +1 1 +1 14 +11 11 +1 1 +1 1 +17 23 +1 5 +1 1 +17 26 +1 1 +1 2 +1 1 +4 4 +2 2 +2 2 +14 37 +4 5 +3 3 +1 1 +1 2 +2 2 +39 48 +1 17 +1 1 +4 35 +1 1 +1 1 +1 1 +1 1 +1 1 +1 27 +1 1 +1 1 +1 1 +4 5 +1 1 +2 4 +6 6 +1 1 +3 5 +1 1 +1 1 +2 2 +1 1 +5 5 +1 1 +1 1 +1 1 +4 4 +1 1 +1 1 +2 2 +8 8 +3 4 +1 1 +1 2 +1 1 +1 4 +1 1 +1 1 +2 2 +2 2 +2 2 +4 6 +2 2 +2 2 +1 1 +1 1 +31 83 +30 38 +2 2 +1 1 +1 1 +1 4 +32 37 +1 1 +4 4 +1 1 +38 44 +5 14 +5 7 +5 5 +1 1 +3 4 +24 34 +5 7 +2 2 +10 10 +4 4 +1 1 +64 77 +28 42 +3 4 +1 1 +3 6 +1 1 +2 2 +6 6 +1 1 +6 6 +1 1 +21 31 +1 1 +1 1 +1 1 +1 1 +1 2 +19 21 +53 57 +1 1 +29 33 +2 2 +115 128 +1 1 +5 7 +1 1 +3 3 +1 1 +1 1 +30 48 +1 1 +1 2 +38 58 +3 3 +1 1 +2 2 +16 17 +3 3 +2 2 +13 20 +2 2 +3 5 +2 2 +4 5 +213 425 +13 15 +2 2 +3 3 +1 1 +1 1 +1 1 +5 7 +1 1 +5 5 +1 1 +1 2 +1 2 +16 17 +3 3 +1 1 +1 1 +1 1 +3 3 +1 1 +38 50 +3 6 +1 1 +6 9 +45 57 +2 2 +3 3 +1 1 +9 13 +3 3 +9 10 +1 1 +6 6 +1 1 +1 2 +1 3 +1 1 +1 3 +1 1 +1 1 +1 1 +1 2 +1 1 +2 2 +3 3 +1 1 +1 1 +15 16 +1 1 +3 6 +1 1 +1 1 +1 2 +1 2 +6 8 +1 1 +12 24 +3 5 +1 1 +2 2 +2 5 +2 2 +1 1 +3 3 +1 1 +3 3 +2 2 +81 87 +1 1 +1 1 +1 1 +3 3 +1 1 +1 1 +1 27 +1 1 +1 1 +1 1 +20 25 +1 1 +6 7 +1 1 +35 35 +5 8 +1 2 +5 5 +3 3 +1 1 +1 1 +1 1 +1 1 +1 1 +3 3 +1 1 +5 5 +1 1 +5 5 +1 1 +1 1 +3 4 +4 4 +1 1 +3 3 +19 20 +1 1 +1 1 +1 3 +1 1 +1 1 +3 3 +1 1 +1 1 +41 99 +1 2 +3 5 +1 1 +2 4 +1 1 +12 17 +2 2 +1 1 +17 22 +1 1 +2 2 +3 4 +4 4 +1 1 +1 1 +1 1 +1 1 +1 1 +2 2 +1 1 +1 1 +2 2 +2 3 +33 39 +1 1 +17 18 +1 1 +5 5 +1 1 +1 1 +1 1 +4 4 +5 5 +1 1 +1 1 +3 4 +1 1 +1 1 +3 3 +1 1 +1 1 +5 5 +2 4 +2 2 +6 7 +1 1 +1 2 +1 1 +1 1 +1 1 +1 1 +6 6 +2 3 +3 3 +4 4 +55 56 +2 2 +1 2 +1 1 +1 1 +1 1 +3 3 +1 1 +1 2 +2 2 +11 11 +13 32 +1 1 +2 2 +14 21 +1 2 +18 22 +14 15 +1 1 +1 2 +1 1 +1 1 +1 1 +3 3 +1 1 +1 1 +5 20 +7 8 +16 18 +3 3 +1 2 +1 1 +1 1 +3 19 +1 1 +1 1 +17 25 +1 1 +1 1 +1 2 +1 1 +4 5 +97 100 +3 5 +3 4 +1 1 +2 2 +1 1 +1 1 +58 65 +4 5 +1 1 +1 2 +1 1 +1 1 +4 4 +1 1 +5 5 +1 1 +1 1 +1 1 +1 1 +2 2 +1 4 +1 1 +1 1 +1 1 +6 7 +2 2 +1 1 +2 2 +3 6 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 2 +1 1 +1 1 +1 1 +1 1 +2 2 +1 3 +2 3 +2 2 +1 3 +1 1 +4 4 +1 4 +1 1 +1 1 +4 4 +1 1 +65 81 +161 261 +1 1 +1 7 +3 3 +2 5 +1 1 +1 1 +1 1 +35 36 +1 1 +1 1 +6 6 +3 3 +2 4 +1 7 +2 3 +3 3 +1 1 +1 2 +9 12 +13 22 +32 33 +1 1 +21 41 +1 1 +1 1 +1 1 +1 1 +1 1 +4 4 +8 10 +4 4 +2 4 +2 2 +1 1 +1 1 +1 1 +1 1 +2 2 +2 3 +1 1 +1 1 +21 25 +12 12 +40 82 +1 1 +2 2 +149 159 +2 2 +1 1 +2 11 +1 1 +1 1 +1 1 +1 1 +1 1 +3 3 +1 2 +3 3 +1 1 +8 9 +58 59 +5 10 +1 1 +1 2 +2 2 +1 1 +1 1 +3 3 +1 1 +12 18 +1 1 +4 4 +2 3 +1 3 +1 1 +1 1 +18 21 +114 122 +2 2 +1 1 +15 18 +1 1 +1 13 +8 9 +20 22 +1 1 +5 7 +1 1 +1 1 +1 1 +4 5 +2 2 +5 5 +2 2 +1 1 +1 6 +2 2 +3 4 +1 1 +3 3 +4 4 +2 2 +26 30 +2 2 +3 3 +3 3 +1 1 +1 1 +1 1 +3 4 +1 1 +3 3 +37 45 +1 1 +1 1 +1 1 +1 1 +3 3 +1 1 +1 1 +1 1 +31 39 +1 1 +1 1 +1 1 +6 6 +4 4 +3 3 +1 1 +2 2 +1 1 +1 18 +1 1 +2 2 +2 2 +1 1 +1 1 +1 2 +2 2 +1 1 +1 1 +1 1 +1 1 +1 1 +6 16 +1 1 +1 1 +1 1 +1 1 +4 4 +24 101 +3 3 +1 1 +1 1 +2 2 +1 1 +3 3 +2 6 +2 2 +1 1 +1 1 +1 1 +1 1 +10 14 +3 4 +6 10 +7 8 +1 1 +1 1 +4 5 +1 1 +4 4 +1 1 +2 2 +1 4 +1 2 +1 1 +1 1 +1 2 +1 1 +2 3 +2 2 +1 1 +1 1 +1 1 +2 3 +1 1 +5 7 +57 75 +1 1 +488 786 +1 1 +65 117 +1 1 +131 136 +7 9 +1 1 +2 2 +3 3 +1 1 +1 1 +1 1 +1 1 +2 2 +1 1 +4 4 +2 2 +10 10 +1 1 +3 3 +1 1 +1 1 +2 2 +1 1 +3 3 +2 3 +2 2 +5 10 +1 1 +1 1 +7 10 +3 3 +1 2 +1 1 +1 1 +13 18 +4 4 +1 1 +3 3 +1 1 +12 14 +3 3 +1 1 +14 17 +2 2 +3 3 +1 1 +6 8 +1 1 +9 9 +1 1 +1 1 +1 1 +2 2 +5 8 +1 1 +4 5 +7 8 +1 1 +6 12 +2 2 +5 5 +1 1 +1 9 +2 3 +12 12 +3 3 +1 1 +29 33 +1 3 +1 1 +1 1 +1 1 +3 3 +1 1 +1 1 +2 2 +1 1 +2 2 +1 1 +1 1 +1 1 +20 21 +4 4 +1 1 +1 1 +1 1 +1 1 +5 5 +3 3 +1 1 +2 3 +2 2 +1 1 +1 2 +3 3 +1 1 +3 3 +50 62 +1 1 +2 2 +2 2 +43 55 +10 11 +1 1 +5 5 +1 1 +3 3 +2 2 +35 121 +2 9 +1 1 +1 1 +1 1 +1 3 +1 1 +1 1 +1 1 +3 4 +2 2 +1 1 +5 6 +2 2 +1 2 +2 3 +2 2 +1 2 +2 3 +1 1 +1 1 +19 37 +1 1 +2 2 +1 1 +2 2 +3 3 +5 5 +2 2 +1 1 +1 1 +1 1 +34 46 +6 6 +3 4 +1 3 +1 1 +1 1 +114 292 +6 6 +2 4 +1 1 +2 2 +2 2 +1 1 +40 53 +1 1 +6 7 +3 3 +1 1 +1 2 +1 1 +1 1 +3 4 +25 31 +5 5 +9 24 +1 1 +2 2 +16 34 +1 1 +5 6 +23 30 +3 4 +1 1 +6 8 +8 11 +1 1 +9 9 +3 3 +3 3 +1 1 +1 1 +5 6 +2 2 +1 1 +3 8 +2 2 +1 1 +1 1 +2 2 +4 5 +57 63 +1 1 +1 2 +1 1 diff --git a/dbms/tests/queries/1_stateful/00024_random_counters.sql b/dbms/tests/queries/1_stateful/00024_random_counters.sql index 12ae7d1b25c..99ba9cc653b 100644 --- a/dbms/tests/queries/1_stateful/00024_random_counters.sql +++ b/dbms/tests/queries/1_stateful/00024_random_counters.sql @@ -1,1000 +1,1000 @@ -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32152608; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9627212; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25152951; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22202319; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13848191; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27855803; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27944638; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16513894; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4314057; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11878090; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23005927; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17205778; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21296650; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12068702; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8446208; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8439835; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30344780; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2881921; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1828473; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27620040; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14960013; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 103918; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9626742; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18370244; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 813903; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22176733; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17175454; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31608140; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11802602; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12577104; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 153437; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32240558; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27444870; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 79306; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15222279; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11782937; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1677; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9527330; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23580782; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33027895; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 199609; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29139484; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1700065; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30212873; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6773723; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21842879; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9460479; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16451704; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 51267; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30489182; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11947625; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18776987; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25762358; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 74905; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 877422; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3465045; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2084559; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13828281; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30299683; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 132115; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10919775; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12329250; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11525543; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32395537; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24537202; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2270964; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8518291; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11897183; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23805647; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22652078; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19363661; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32339088; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11394550; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1988179; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2135273; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14500371; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10463153; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18838936; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24492652; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26848923; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12495799; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12028938; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8934725; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18602951; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32404741; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19171705; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9831187; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20047182; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26690858; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 126413; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31244775; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15690176; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28374997; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12717244; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9152092; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5397339; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12452068; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13626118; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 46783; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11484344; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21453219; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7692388; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30879805; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27784549; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 665663; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30535786; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11685143; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13652647; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9880318; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30148588; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32745436; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27390924; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17470663; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 196859; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22123478; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 87021; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25264218; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24125574; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26099981; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1141558; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 220829; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15651875; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 182483; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28430678; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31384642; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1008241; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10462834; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26829659; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29130002; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17891770; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26531140; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15014338; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15375411; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7952204; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 41859; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21651593; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9527676; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 107394; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23409492; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31407407; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29312961; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9705505; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29848510; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10187274; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 112606; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15639744; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4375349; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1423039; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13933371; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20430236; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30679961; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 37094; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23197674; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 994587; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 437496; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3904733; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19200606; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 84668; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28581029; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11074306; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2470089; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12251899; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16996077; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12426411; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1034934; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4721601; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22026000; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21031300; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 559124; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15492463; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21419604; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25632271; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14446476; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12684903; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23292922; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26976782; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20269131; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18309978; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5305320; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30926629; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14816057; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19523905; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18775058; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32507411; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25535479; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24858652; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32420158; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4805894; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8157258; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5759745; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12626987; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5342591; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10951832; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9729032; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27999107; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7302193; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30447727; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15764416; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15727130; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15116605; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 527313; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16687935; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28304381; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17699739; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17339596; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29348067; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20861945; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12922065; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27019489; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18299445; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 108465; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 233447; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13042904; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31481509; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2267268; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26140306; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19094364; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25000943; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6860549; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30714288; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16289139; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1419182; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33436573; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30062358; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18167743; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27846382; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30148240; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32332238; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25129158; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14066924; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19832770; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29018190; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 852275; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11328399; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28179212; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20155907; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30685297; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32783957; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1552720; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28110991; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4814424; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20171153; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14920591; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 65690; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14357916; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26533001; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17014738; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11977336; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30142464; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14082365; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18851419; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27638649; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8798932; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 717825; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6912378; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26898048; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5992218; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13422462; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21204372; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17845298; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6933004; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21627605; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3395439; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22315068; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24973444; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27751340; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6022884; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32417601; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18087198; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21940806; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23809389; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9510424; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30651933; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17818815; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9038457; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9153497; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29938964; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10471118; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12913162; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14933629; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7173707; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28680585; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1279785; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33276693; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 573557; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27753414; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22968595; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25211823; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32687774; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6062762; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18866703; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5164840; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6462629; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25039797; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10789598; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33076990; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28960547; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32723171; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17888313; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29810654; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21760643; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16678170; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 368520; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12506284; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9802670; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18488016; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 227003; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15254606; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32580177; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30313645; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20879524; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27222776; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11266528; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17018146; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19902143; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19469853; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22823497; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 56768; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 136798; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16554922; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20627728; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6551053; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 124145; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10881152; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17271030; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28213281; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15665842; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28264219; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29277533; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22926441; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31057728; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8027311; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14229492; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14782220; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29099258; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 99953; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9334015; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16156945; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 124031; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1670442; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21036594; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22954047; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16054043; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 121765; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1482385; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25977258; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24596247; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 550092; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1579438; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1205; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 126296; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 177248; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27523607; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15873699; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11971473; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18965085; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19035683; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29640643; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11929806; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9352219; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18492653; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7967264; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11391453; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4289; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3567; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13575826; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2566437; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21042675; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26498330; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23764459; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32664413; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10116935; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24572551; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26788657; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12830859; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 530033; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6764575; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25219472; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10721285; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26254035; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15486693; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10323514; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23578364; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25449880; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13428298; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17679279; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27610140; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15346859; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 535736; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 513828; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20411888; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13595045; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33221835; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 97601; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12819274; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18047205; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19900235; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27830172; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20839743; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29980468; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27417156; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17908689; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24471592; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32147490; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22966030; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21060870; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 238185; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10152551; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11255139; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 982334; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15199978; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15678357; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18206303; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10902608; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22494906; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22204221; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13097211; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30998656; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26656294; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 922545; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9428510; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15137339; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15578624; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31695129; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15791360; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29571338; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5371768; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15163979; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16312681; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6126176; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16061128; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8528634; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 136544; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3093873; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3994698; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8302978; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16115563; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21804036; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9785708; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10847072; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30692218; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15582824; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19802155; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20835290; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 204284; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25636491; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30446517; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16761451; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 456303; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17301839; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27472581; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24078399; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26345482; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 451381; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8576994; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19418898; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10068353; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3767138; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 758020; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13521375; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25968099; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26805240; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13051011; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 901894; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33097016; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12545080; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29944288; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8250825; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12499373; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22535728; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11929724; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3615273; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24172869; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 116132; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12002817; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23681158; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3938; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8468701; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1295067; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27469232; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32708119; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 122578; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12139400; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12219626; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9262336; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9269892; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 122701; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19589931; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29539889; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31115640; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6283044; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30642040; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18065262; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26714391; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15351586; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13090710; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16201652; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31960256; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2658509; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 467277; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1274110; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23640128; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16197014; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28228612; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11659509; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24981440; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 52285; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30583892; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31467341; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25512316; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2908472; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 422752; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32718035; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14213540; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14951444; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6819113; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9532880; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4102488; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19537427; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7078160; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29521616; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5045377; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23131467; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22383622; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22079706; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29466380; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12045654; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30178011; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20821588; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21966434; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29390311; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19370159; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24857158; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31982180; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11990254; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3841725; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13993951; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31252290; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26398773; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 891512; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27087947; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2097095; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26252354; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13928858; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4331960; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30552074; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27905732; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30049284; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2118697; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20849218; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11338538; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3348692; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17693905; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23502543; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8905975; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18343399; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15235863; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20356153; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10552704; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28875831; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1488561; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15012941; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25726446; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2601050; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27426912; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11269650; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14880200; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 362337; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23533327; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26381021; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17522450; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31868526; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18276314; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1841289; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22234319; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11463222; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15251006; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24841412; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28755796; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9087442; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13734462; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9285105; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13289061; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29890926; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30509694; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17698850; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 46229; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16541087; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11305551; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 429238; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7583796; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8604476; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29759280; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1388922; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10884907; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18220244; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 122157; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18069840; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6707469; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26818794; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14770800; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16652737; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25497243; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14747538; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21371935; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1681601; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5343898; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22040058; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 752596; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9377867; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1848946; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1449313; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31332002; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10829982; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22431161; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29172033; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7631750; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 898844; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21460344; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25387068; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30980374; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13021547; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27715925; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30292547; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18666245; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18954194; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29070192; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 914290; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14807517; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23062682; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5132969; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15094854; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 622095; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1244323; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14804701; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11656845; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17167258; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8959523; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23121135; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4339624; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22679035; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13127067; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18362622; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4189114; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18776826; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26792263; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13409810; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22183039; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16132723; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3925258; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14248840; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18135589; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11234961; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11179577; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 178965; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10138078; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21048048; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8001235; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32833016; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32275374; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1430786; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12969140; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25529912; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18395861; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27380554; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16653574; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16372034; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28050494; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6886254; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7472729; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12646802; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6589761; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19556032; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10261903; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4389; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2415202; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20007939; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17957094; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9920354; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24840314; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5077718; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11650674; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19766470; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7854638; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9169290; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22873394; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30838169; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 79894; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25792494; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25326672; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33123311; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33237554; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15130284; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18811870; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25418177; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17202302; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31836505; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28671820; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25643858; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16338596; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27288074; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9458517; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25163573; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15680967; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20413991; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19332304; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23159444; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24708786; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 250297; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29944728; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14582542; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 512441; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31273184; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30255145; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 89813; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14959234; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26621829; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 279206; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13041403; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33392742; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10895948; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20804625; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10129067; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13855355; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31007051; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4109301; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29492024; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28963180; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11530154; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31889101; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1713672; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16069992; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9075873; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14512529; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8632591; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33056094; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28349520; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26806792; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11496875; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11797321; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25795940; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33196708; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13243216; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25096876; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26974949; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27061789; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29686454; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5045092; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2893170; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21528033; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16980819; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30854698; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1041468; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 215125; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 91347; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22706469; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33038294; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1446406; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 183702; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10246325; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13754526; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6854006; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26686232; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29345198; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15956574; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8558022; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14066782; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31710428; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6750831; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14832055; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29613113; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15159107; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6309003; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4311581; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28180829; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15131841; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20458889; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26250664; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31737265; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 802571; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25064649; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21183784; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3218637; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3375471; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1690000; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18602620; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29918973; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8555235; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32152623; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19670163; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25856874; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6142197; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27822106; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8944163; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7596672; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 129436; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33541084; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5199217; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10337246; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12718765; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10729131; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28049397; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1410155; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24924437; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16706889; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 54647; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29407271; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1575071; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6861225; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30114382; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 129970; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21103497; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3433579; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14174715; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8450741; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30033987; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11474175; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9601520; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 7377941; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15646334; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18305797; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2057218; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17121933; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6870927; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19743903; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9019159; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21251610; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 239704; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16170940; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31857931; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25174672; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31546315; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 811438; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33135020; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28325470; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1196502; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 117339; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19198214; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28046111; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27663162; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3651; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8443242; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6773651; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28957858; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15586212; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 155469; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 731800; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13198917; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2080118; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17987407; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1832110; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32960999; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13858070; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2800568; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 381151; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26724412; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 238149; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20458616; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16847984; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14870120; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 4729620; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12886810; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 109350; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17512881; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5250020; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 184094; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3071553; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18940958; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16166873; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13648378; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32750584; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31167464; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21597707; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21992900; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16695153; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12272303; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18958518; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11827733; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12495926; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21022681; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32262727; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12082756; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15636497; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20081370; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26349655; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32832383; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18190567; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 61749; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28596915; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28835938; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32924951; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15835912; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22905942; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12295903; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12461093; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27568271; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33525856; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10351138; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16804486; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24506501; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1336365; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19178381; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17921720; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25396786; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22031463; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19624501; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 28665905; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14851585; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27554706; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14188052; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33301471; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32896955; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1134828; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27050219; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23641604; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22935857; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29805516; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25890338; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20710225; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3925036; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31404180; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25888177; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11074293; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30922753; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11403908; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25615656; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17652214; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 16155802; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5565120; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5508217; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33281735; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 11619273; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 67148; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22687534; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17887682; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18506413; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1443226; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13761576; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30941622; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17681363; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 187532; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 95405; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31073741; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 9706801; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12504322; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31779591; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18781661; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 18284607; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 10633383; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3249127; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17567300; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8789986; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 30073024; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26477401; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32222832; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 23098807; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 50708; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25067039; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29132588; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22947337; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27778601; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25325678; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12822401; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 8876685; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31096269; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17466070; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26058342; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 1468384; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 22665021; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24895973; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15423066; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26091197; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 12103346; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15917190; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 31527060; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 3944; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 24572480; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 229185; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 17038391; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 27368675; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 26899897; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 13257515; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19531252; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 21048946; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 33104049; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20824535; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15014380; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 25235392; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29560548; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 2599836; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 32842358; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 5795232; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29588193; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 19019850; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 29580949; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15335748; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 15094099; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 6308405; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 20762370; -SELECT uniq(UserID), sum(Sign), sum(Sign * PageViews), sum(Sign * Duration) FROM test.visits WHERE CounterID = 14121177; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32152608; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9627212; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25152951; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22202319; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13848191; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27855803; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27944638; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16513894; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 4314057; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11878090; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 23005927; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17205778; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21296650; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12068702; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8446208; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8439835; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30344780; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 2881921; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1828473; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27620040; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14960013; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 103918; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9626742; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18370244; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 813903; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22176733; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17175454; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31608140; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11802602; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12577104; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 153437; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32240558; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27444870; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 79306; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15222279; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11782937; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1677; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9527330; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 23580782; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 33027895; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 199609; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29139484; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1700065; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30212873; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6773723; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21842879; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9460479; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16451704; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 51267; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30489182; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11947625; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18776987; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25762358; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 74905; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 877422; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3465045; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 2084559; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13828281; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30299683; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 132115; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10919775; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12329250; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11525543; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32395537; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 24537202; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 2270964; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8518291; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11897183; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 23805647; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22652078; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19363661; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32339088; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11394550; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1988179; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 2135273; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14500371; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10463153; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18838936; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 24492652; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26848923; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12495799; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12028938; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8934725; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18602951; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32404741; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19171705; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9831187; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20047182; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26690858; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 126413; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31244775; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15690176; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28374997; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12717244; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9152092; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 5397339; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12452068; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13626118; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 46783; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11484344; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21453219; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 7692388; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30879805; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27784549; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 665663; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30535786; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11685143; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13652647; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9880318; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30148588; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32745436; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27390924; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17470663; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 196859; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22123478; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 87021; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25264218; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 24125574; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26099981; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1141558; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 220829; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15651875; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 182483; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28430678; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31384642; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1008241; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10462834; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26829659; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29130002; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17891770; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26531140; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15014338; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15375411; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 7952204; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 41859; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21651593; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9527676; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 107394; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 23409492; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31407407; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29312961; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9705505; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29848510; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10187274; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 112606; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15639744; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 4375349; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1423039; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13933371; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20430236; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30679961; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 37094; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 23197674; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 994587; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 437496; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3904733; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19200606; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 84668; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28581029; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11074306; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 2470089; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12251899; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16996077; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12426411; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1034934; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 4721601; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22026000; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21031300; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 559124; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15492463; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21419604; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25632271; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14446476; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12684903; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 23292922; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26976782; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20269131; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18309978; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 5305320; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30926629; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14816057; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19523905; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18775058; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32507411; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25535479; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 24858652; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32420158; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 4805894; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8157258; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 5759745; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12626987; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 5342591; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10951832; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9729032; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27999107; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 7302193; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30447727; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15764416; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15727130; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15116605; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 527313; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16687935; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28304381; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17699739; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17339596; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29348067; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20861945; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12922065; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27019489; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18299445; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 108465; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 233447; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13042904; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31481509; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 2267268; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26140306; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19094364; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25000943; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6860549; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30714288; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16289139; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1419182; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 33436573; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30062358; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18167743; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27846382; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30148240; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32332238; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25129158; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14066924; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19832770; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29018190; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 852275; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11328399; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28179212; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20155907; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30685297; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32783957; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1552720; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28110991; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 4814424; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20171153; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14920591; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 65690; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14357916; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26533001; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17014738; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11977336; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30142464; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14082365; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18851419; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27638649; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8798932; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 717825; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6912378; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26898048; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 5992218; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13422462; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21204372; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17845298; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6933004; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21627605; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3395439; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22315068; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 24973444; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27751340; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6022884; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32417601; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18087198; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21940806; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 23809389; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9510424; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30651933; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17818815; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9038457; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9153497; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29938964; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10471118; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12913162; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14933629; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 7173707; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28680585; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1279785; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 33276693; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 573557; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27753414; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22968595; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25211823; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32687774; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6062762; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18866703; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 5164840; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6462629; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25039797; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10789598; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 33076990; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28960547; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32723171; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17888313; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29810654; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21760643; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16678170; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 368520; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12506284; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9802670; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18488016; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 227003; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15254606; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32580177; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30313645; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20879524; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27222776; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11266528; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17018146; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19902143; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19469853; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22823497; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 56768; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 136798; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16554922; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20627728; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6551053; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 124145; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10881152; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17271030; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28213281; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15665842; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28264219; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29277533; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22926441; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31057728; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8027311; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14229492; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14782220; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29099258; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 99953; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9334015; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16156945; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 124031; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1670442; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21036594; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22954047; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16054043; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 121765; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1482385; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25977258; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 24596247; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 550092; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1579438; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1205; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 126296; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 177248; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27523607; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15873699; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11971473; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18965085; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19035683; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29640643; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11929806; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9352219; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18492653; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 7967264; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11391453; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 4289; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3567; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13575826; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 2566437; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21042675; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26498330; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 23764459; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32664413; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10116935; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 24572551; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26788657; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12830859; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 530033; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6764575; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25219472; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10721285; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26254035; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15486693; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10323514; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 23578364; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25449880; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13428298; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17679279; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27610140; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15346859; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 535736; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 513828; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20411888; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13595045; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 33221835; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 97601; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12819274; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18047205; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19900235; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27830172; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20839743; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29980468; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27417156; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17908689; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 24471592; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32147490; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22966030; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21060870; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 238185; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10152551; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11255139; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 982334; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15199978; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15678357; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18206303; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10902608; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22494906; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22204221; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13097211; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30998656; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26656294; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 922545; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9428510; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15137339; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15578624; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31695129; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15791360; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29571338; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 5371768; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15163979; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16312681; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6126176; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16061128; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8528634; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 136544; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3093873; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3994698; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8302978; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16115563; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21804036; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9785708; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10847072; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30692218; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15582824; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19802155; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20835290; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 204284; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25636491; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30446517; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16761451; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 456303; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17301839; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27472581; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 24078399; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26345482; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 451381; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8576994; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19418898; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10068353; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3767138; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 758020; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13521375; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25968099; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26805240; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13051011; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 901894; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 33097016; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12545080; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29944288; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8250825; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12499373; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22535728; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11929724; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3615273; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 24172869; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 116132; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12002817; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 23681158; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3938; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8468701; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1295067; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27469232; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32708119; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 122578; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12139400; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12219626; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9262336; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9269892; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 122701; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19589931; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29539889; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31115640; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6283044; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30642040; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18065262; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26714391; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15351586; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13090710; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16201652; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31960256; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 2658509; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 467277; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1274110; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 23640128; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16197014; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28228612; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11659509; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 24981440; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 52285; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30583892; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31467341; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25512316; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 2908472; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 422752; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32718035; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14213540; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14951444; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6819113; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9532880; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 4102488; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19537427; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 7078160; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29521616; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 5045377; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 23131467; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22383622; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22079706; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29466380; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12045654; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30178011; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20821588; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21966434; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29390311; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19370159; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 24857158; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31982180; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11990254; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3841725; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13993951; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31252290; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26398773; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 891512; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27087947; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 2097095; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26252354; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13928858; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 4331960; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30552074; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27905732; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30049284; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 2118697; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20849218; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11338538; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3348692; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17693905; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 23502543; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8905975; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18343399; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15235863; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20356153; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10552704; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28875831; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1488561; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15012941; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25726446; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 2601050; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27426912; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11269650; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14880200; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 362337; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 23533327; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26381021; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17522450; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31868526; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18276314; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1841289; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22234319; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11463222; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15251006; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 24841412; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28755796; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9087442; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13734462; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9285105; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13289061; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29890926; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30509694; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17698850; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 46229; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16541087; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11305551; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 429238; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 7583796; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8604476; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29759280; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1388922; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10884907; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18220244; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 122157; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18069840; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6707469; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26818794; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14770800; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16652737; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25497243; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14747538; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21371935; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1681601; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 5343898; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22040058; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 752596; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9377867; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1848946; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1449313; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31332002; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10829982; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22431161; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29172033; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 7631750; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 898844; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21460344; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25387068; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30980374; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13021547; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27715925; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30292547; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18666245; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18954194; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29070192; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 914290; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14807517; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 23062682; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 5132969; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15094854; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 622095; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1244323; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14804701; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11656845; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17167258; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8959523; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 23121135; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 4339624; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22679035; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13127067; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18362622; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 4189114; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18776826; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26792263; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13409810; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22183039; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16132723; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3925258; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14248840; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18135589; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11234961; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11179577; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 178965; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10138078; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21048048; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8001235; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32833016; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32275374; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1430786; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12969140; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25529912; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18395861; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27380554; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16653574; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16372034; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28050494; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6886254; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 7472729; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12646802; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6589761; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19556032; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10261903; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 4389; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 2415202; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20007939; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17957094; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9920354; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 24840314; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 5077718; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11650674; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19766470; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 7854638; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9169290; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22873394; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30838169; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 79894; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25792494; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25326672; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 33123311; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 33237554; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15130284; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18811870; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25418177; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17202302; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31836505; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28671820; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25643858; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16338596; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27288074; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9458517; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25163573; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15680967; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20413991; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19332304; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 23159444; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 24708786; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 250297; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29944728; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14582542; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 512441; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31273184; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30255145; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 89813; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14959234; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26621829; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 279206; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13041403; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 33392742; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10895948; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20804625; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10129067; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13855355; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31007051; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 4109301; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29492024; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28963180; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11530154; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31889101; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1713672; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16069992; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9075873; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14512529; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8632591; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 33056094; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28349520; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26806792; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11496875; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11797321; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25795940; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 33196708; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13243216; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25096876; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26974949; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27061789; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29686454; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 5045092; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 2893170; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21528033; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16980819; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30854698; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1041468; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 215125; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 91347; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22706469; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 33038294; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1446406; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 183702; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10246325; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13754526; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6854006; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26686232; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29345198; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15956574; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8558022; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14066782; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31710428; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6750831; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14832055; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29613113; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15159107; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6309003; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 4311581; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28180829; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15131841; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20458889; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26250664; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31737265; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 802571; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25064649; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21183784; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3218637; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3375471; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1690000; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18602620; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29918973; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8555235; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32152623; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19670163; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25856874; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6142197; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27822106; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8944163; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 7596672; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 129436; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 33541084; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 5199217; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10337246; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12718765; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10729131; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28049397; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1410155; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 24924437; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16706889; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 54647; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29407271; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1575071; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6861225; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30114382; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 129970; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21103497; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3433579; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14174715; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8450741; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30033987; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11474175; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9601520; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 7377941; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15646334; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18305797; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 2057218; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17121933; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6870927; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19743903; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9019159; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21251610; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 239704; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16170940; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31857931; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25174672; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31546315; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 811438; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 33135020; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28325470; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1196502; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 117339; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19198214; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28046111; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27663162; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3651; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8443242; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6773651; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28957858; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15586212; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 155469; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 731800; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13198917; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 2080118; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17987407; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1832110; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32960999; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13858070; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 2800568; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 381151; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26724412; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 238149; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20458616; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16847984; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14870120; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 4729620; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12886810; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 109350; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17512881; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 5250020; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 184094; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3071553; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18940958; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16166873; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13648378; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32750584; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31167464; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21597707; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21992900; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16695153; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12272303; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18958518; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11827733; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12495926; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21022681; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32262727; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12082756; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15636497; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20081370; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26349655; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32832383; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18190567; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 61749; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28596915; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28835938; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32924951; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15835912; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22905942; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12295903; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12461093; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27568271; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 33525856; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10351138; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16804486; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 24506501; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1336365; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19178381; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17921720; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25396786; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22031463; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19624501; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 28665905; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14851585; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27554706; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14188052; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 33301471; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32896955; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1134828; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27050219; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 23641604; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22935857; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29805516; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25890338; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20710225; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3925036; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31404180; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25888177; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11074293; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30922753; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11403908; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25615656; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17652214; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 16155802; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 5565120; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 5508217; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 33281735; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 11619273; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 67148; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22687534; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17887682; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18506413; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1443226; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13761576; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30941622; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17681363; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 187532; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 95405; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31073741; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 9706801; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12504322; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31779591; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18781661; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 18284607; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 10633383; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3249127; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17567300; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8789986; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 30073024; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26477401; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32222832; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 23098807; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 50708; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25067039; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29132588; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22947337; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27778601; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25325678; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12822401; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 8876685; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31096269; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17466070; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26058342; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 1468384; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 22665021; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 24895973; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15423066; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26091197; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 12103346; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15917190; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 31527060; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 3944; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 24572480; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 229185; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 17038391; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 27368675; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 26899897; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 13257515; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19531252; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 21048946; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 33104049; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20824535; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15014380; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 25235392; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29560548; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 2599836; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 32842358; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 5795232; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29588193; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 19019850; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 29580949; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15335748; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 15094099; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 6308405; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 20762370; +SELECT uniq(UserID), sum(Sign) FROM test.visits WHERE CounterID = 14121177; From 17335ab1f1e4671e39e45c463d2b720eee6c01ee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Jan 2019 01:52:33 +0300 Subject: [PATCH 43/71] Fixed some warnings found by PVS-Studio --- dbms/programs/server/Server.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index f0fd80669c0..c64b8358612 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -565,7 +565,7 @@ int Server::main(const std::vector & /*args*/) auto address = socket_bind_listen(socket, listen_host, config().getInt("http_port")); socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); - servers.emplace_back(new Poco::Net::HTTPServer( + servers.emplace_back(std::make_unique( new HTTPHandlerFactory(*this, "HTTPHandler-factory"), server_pool, socket, @@ -582,7 +582,7 @@ int Server::main(const std::vector & /*args*/) auto address = socket_bind_listen(socket, listen_host, config().getInt("https_port"), /* secure = */ true); socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); - servers.emplace_back(new Poco::Net::HTTPServer( + servers.emplace_back(std::make_unique( new HTTPHandlerFactory(*this, "HTTPSHandler-factory"), server_pool, socket, @@ -602,7 +602,7 @@ int Server::main(const std::vector & /*args*/) auto address = socket_bind_listen(socket, listen_host, config().getInt("tcp_port")); socket.setReceiveTimeout(settings.receive_timeout); socket.setSendTimeout(settings.send_timeout); - servers.emplace_back(new Poco::Net::TCPServer( + servers.emplace_back(std::make_unique( new TCPHandlerFactory(*this), server_pool, socket, @@ -619,7 +619,7 @@ int Server::main(const std::vector & /*args*/) auto address = socket_bind_listen(socket, listen_host, config().getInt("tcp_port_secure"), /* secure = */ true); socket.setReceiveTimeout(settings.receive_timeout); socket.setSendTimeout(settings.send_timeout); - servers.emplace_back(new Poco::Net::TCPServer( + servers.emplace_back(std::make_unique( new TCPHandlerFactory(*this, /* secure= */ true), server_pool, socket, @@ -642,7 +642,7 @@ int Server::main(const std::vector & /*args*/) auto address = socket_bind_listen(socket, listen_host, config().getInt("interserver_http_port")); socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); - servers.emplace_back(new Poco::Net::HTTPServer( + servers.emplace_back(std::make_unique( new InterserverIOHTTPHandlerFactory(*this, "InterserverIOHTTPHandler-factory"), server_pool, socket, @@ -658,7 +658,7 @@ int Server::main(const std::vector & /*args*/) auto address = socket_bind_listen(socket, listen_host, config().getInt("interserver_https_port"), /* secure = */ true); socket.setReceiveTimeout(settings.http_receive_timeout); socket.setSendTimeout(settings.http_send_timeout); - servers.emplace_back(new Poco::Net::HTTPServer( + servers.emplace_back(std::make_unique( new InterserverIOHTTPHandlerFactory(*this, "InterserverIOHTTPHandler-factory"), server_pool, socket, From c7b95b5175b9b47451ca1bd02af2b93034df0caf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Jan 2019 02:27:52 +0300 Subject: [PATCH 44/71] Miscellaneous changes for PVS-Studio --- dbms/programs/client/ConnectionParameters.h | 6 +++--- dbms/programs/client/TestHint.h | 7 +++---- dbms/src/AggregateFunctions/AggregateFunctionRetention.h | 2 +- dbms/src/Dictionaries/CacheDictionary.cpp | 5 ----- dbms/src/Dictionaries/CacheDictionary.h | 7 ++++--- dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp | 4 ---- dbms/src/Dictionaries/ComplexKeyCacheDictionary.h | 7 ++++--- dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp | 6 ------ dbms/src/Dictionaries/ComplexKeyHashedDictionary.h | 7 ++++--- dbms/src/Dictionaries/FlatDictionary.cpp | 6 ------ dbms/src/Dictionaries/FlatDictionary.h | 7 ++++--- dbms/src/Dictionaries/HashedDictionary.cpp | 6 ------ dbms/src/Dictionaries/HashedDictionary.h | 7 ++++--- dbms/src/Dictionaries/IDictionarySource.h | 1 + dbms/src/Dictionaries/RangeHashedDictionary.cpp | 6 ------ dbms/src/Dictionaries/RangeHashedDictionary.h | 7 ++++--- dbms/src/Dictionaries/TrieDictionary.cpp | 5 ----- dbms/src/Dictionaries/TrieDictionary.h | 7 ++++--- dbms/src/Functions/if.cpp | 2 +- dbms/src/Interpreters/IExternalLoadable.h | 3 ++- dbms/src/Parsers/parseQuery.cpp | 2 -- 21 files changed, 39 insertions(+), 71 deletions(-) diff --git a/dbms/programs/client/ConnectionParameters.h b/dbms/programs/client/ConnectionParameters.h index 68bc3728349..557929a9331 100644 --- a/dbms/programs/client/ConnectionParameters.h +++ b/dbms/programs/client/ConnectionParameters.h @@ -25,12 +25,12 @@ namespace ErrorCodes struct ConnectionParameters { String host; - UInt16 port; + UInt16 port{}; String default_database; String user; String password; - Protocol::Secure security; - Protocol::Compression compression; + Protocol::Secure security = Protocol::Secure::Disable; + Protocol::Compression compression = Protocol::Compression::Enable; ConnectionTimeouts timeouts; ConnectionParameters() {} diff --git a/dbms/programs/client/TestHint.h b/dbms/programs/client/TestHint.h index c1ac913e7a8..66e86b5e750 100644 --- a/dbms/programs/client/TestHint.h +++ b/dbms/programs/client/TestHint.h @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include @@ -91,14 +93,11 @@ private: void parse(const String & hint) { - std::stringstream ss; - ss << hint; + ReadBufferFromString ss(hint); while (!ss.eof()) { String item; ss >> item; - if (item.empty()) - break; if (item == "serverError") ss >> server_error; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionRetention.h b/dbms/src/AggregateFunctions/AggregateFunctionRetention.h index 8fac497bdfe..688f7f1404c 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionRetention.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionRetention.h @@ -86,7 +86,7 @@ public: ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } - events_size = arguments.size(); + events_size = static_cast(arguments.size()); } diff --git a/dbms/src/Dictionaries/CacheDictionary.cpp b/dbms/src/Dictionaries/CacheDictionary.cpp index cd61d862c20..752752fae59 100644 --- a/dbms/src/Dictionaries/CacheDictionary.cpp +++ b/dbms/src/Dictionaries/CacheDictionary.cpp @@ -81,11 +81,6 @@ CacheDictionary::CacheDictionary( createAttributes(); } -CacheDictionary::CacheDictionary(const CacheDictionary & other) - : CacheDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.size} -{ -} - void CacheDictionary::toParent(const PaddedPODArray & ids, PaddedPODArray & out) const { diff --git a/dbms/src/Dictionaries/CacheDictionary.h b/dbms/src/Dictionaries/CacheDictionary.h index 9122b0d82bc..7a08ad5c3ab 100644 --- a/dbms/src/Dictionaries/CacheDictionary.h +++ b/dbms/src/Dictionaries/CacheDictionary.h @@ -30,8 +30,6 @@ public: const DictionaryLifetime dict_lifetime, const size_t size); - CacheDictionary(const CacheDictionary & other); - std::exception_ptr getCreationException() const override { return {}; } std::string getName() const override { return name; } @@ -53,7 +51,10 @@ public: bool isCached() const override { return true; } - std::unique_ptr clone() const override { return std::make_unique(*this); } + std::unique_ptr clone() const override + { + return std::make_unique(name, dict_struct, source_ptr->clone(), dict_lifetime, size); + } const IDictionarySource * getSource() const override { return source_ptr.get(); } diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp index 84ab8c569a4..42aa0c943c7 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp @@ -70,10 +70,6 @@ ComplexKeyCacheDictionary::ComplexKeyCacheDictionary( createAttributes(); } -ComplexKeyCacheDictionary::ComplexKeyCacheDictionary(const ComplexKeyCacheDictionary & other) - : ComplexKeyCacheDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.size} -{ -} void ComplexKeyCacheDictionary::getString( const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ColumnString * out) const diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h index 017a638d776..92666158015 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h @@ -47,8 +47,6 @@ public: const DictionaryLifetime dict_lifetime, const size_t size); - ComplexKeyCacheDictionary(const ComplexKeyCacheDictionary & other); - std::string getKeyDescription() const { return key_description; } std::exception_ptr getCreationException() const override { return {}; } @@ -76,7 +74,10 @@ public: bool isCached() const override { return true; } - std::unique_ptr clone() const override { return std::make_unique(*this); } + std::unique_ptr clone() const override + { + return std::make_unique(name, dict_struct, source_ptr->clone(), dict_lifetime, size); + } const IDictionarySource * getSource() const override { return source_ptr.get(); } diff --git a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp index 8422fb2a68f..a36b225680d 100644 --- a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp +++ b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp @@ -43,12 +43,6 @@ ComplexKeyHashedDictionary::ComplexKeyHashedDictionary( creation_time = std::chrono::system_clock::now(); } -ComplexKeyHashedDictionary::ComplexKeyHashedDictionary(const ComplexKeyHashedDictionary & other) - : ComplexKeyHashedDictionary{ - other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty, other.saved_block} -{ -} - #define DECLARE(TYPE) \ void ComplexKeyHashedDictionary::get##TYPE( \ const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType & out) const \ diff --git a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h index 8362d1f6f4a..fc1fc709388 100644 --- a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h +++ b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h @@ -29,8 +29,6 @@ public: bool require_nonempty, BlockPtr saved_block = nullptr); - ComplexKeyHashedDictionary(const ComplexKeyHashedDictionary & other); - std::string getKeyDescription() const { return key_description; } std::exception_ptr getCreationException() const override { return creation_exception; } @@ -51,7 +49,10 @@ public: bool isCached() const override { return false; } - std::unique_ptr clone() const override { return std::make_unique(*this); } + std::unique_ptr clone() const override + { + return std::make_unique(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block); + } const IDictionarySource * getSource() const override { return source_ptr.get(); } diff --git a/dbms/src/Dictionaries/FlatDictionary.cpp b/dbms/src/Dictionaries/FlatDictionary.cpp index 32fa5085f63..26640972014 100644 --- a/dbms/src/Dictionaries/FlatDictionary.cpp +++ b/dbms/src/Dictionaries/FlatDictionary.cpp @@ -50,12 +50,6 @@ FlatDictionary::FlatDictionary( creation_time = std::chrono::system_clock::now(); } -FlatDictionary::FlatDictionary(const FlatDictionary & other) - : FlatDictionary{ - other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty, other.saved_block} -{ -} - void FlatDictionary::toParent(const PaddedPODArray & ids, PaddedPODArray & out) const { diff --git a/dbms/src/Dictionaries/FlatDictionary.h b/dbms/src/Dictionaries/FlatDictionary.h index a5f1fe4b4fc..ccdf522162e 100644 --- a/dbms/src/Dictionaries/FlatDictionary.h +++ b/dbms/src/Dictionaries/FlatDictionary.h @@ -28,8 +28,6 @@ public: bool require_nonempty, BlockPtr saved_block = nullptr); - FlatDictionary(const FlatDictionary & other); - std::exception_ptr getCreationException() const override { return creation_exception; } std::string getName() const override { return name; } @@ -48,7 +46,10 @@ public: bool isCached() const override { return false; } - std::unique_ptr clone() const override { return std::make_unique(*this); } + std::unique_ptr clone() const override + { + return std::make_unique(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block); + } const IDictionarySource * getSource() const override { return source_ptr.get(); } diff --git a/dbms/src/Dictionaries/HashedDictionary.cpp b/dbms/src/Dictionaries/HashedDictionary.cpp index 47817af25f5..6154b9243ff 100644 --- a/dbms/src/Dictionaries/HashedDictionary.cpp +++ b/dbms/src/Dictionaries/HashedDictionary.cpp @@ -44,12 +44,6 @@ HashedDictionary::HashedDictionary( creation_time = std::chrono::system_clock::now(); } -HashedDictionary::HashedDictionary(const HashedDictionary & other) - : HashedDictionary{ - other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty, other.saved_block} -{ -} - void HashedDictionary::toParent(const PaddedPODArray & ids, PaddedPODArray & out) const { diff --git a/dbms/src/Dictionaries/HashedDictionary.h b/dbms/src/Dictionaries/HashedDictionary.h index 053af0eac0c..0ced0726b99 100644 --- a/dbms/src/Dictionaries/HashedDictionary.h +++ b/dbms/src/Dictionaries/HashedDictionary.h @@ -27,8 +27,6 @@ public: bool require_nonempty, BlockPtr saved_block = nullptr); - HashedDictionary(const HashedDictionary & other); - std::exception_ptr getCreationException() const override { return creation_exception; } std::string getName() const override { return name; } @@ -47,7 +45,10 @@ public: bool isCached() const override { return false; } - std::unique_ptr clone() const override { return std::make_unique(*this); } + std::unique_ptr clone() const override + { + return std::make_unique(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block); + } const IDictionarySource * getSource() const override { return source_ptr.get(); } diff --git a/dbms/src/Dictionaries/IDictionarySource.h b/dbms/src/Dictionaries/IDictionarySource.h index 23f0a436e19..bc2df7c3c56 100644 --- a/dbms/src/Dictionaries/IDictionarySource.h +++ b/dbms/src/Dictionaries/IDictionarySource.h @@ -3,6 +3,7 @@ #include #include + namespace DB { class IDictionarySource; diff --git a/dbms/src/Dictionaries/RangeHashedDictionary.cpp b/dbms/src/Dictionaries/RangeHashedDictionary.cpp index 1ee3b806287..48c884fa773 100644 --- a/dbms/src/Dictionaries/RangeHashedDictionary.cpp +++ b/dbms/src/Dictionaries/RangeHashedDictionary.cpp @@ -94,12 +94,6 @@ RangeHashedDictionary::RangeHashedDictionary( creation_time = std::chrono::system_clock::now(); } -RangeHashedDictionary::RangeHashedDictionary(const RangeHashedDictionary & other) - : RangeHashedDictionary{ - other.dictionary_name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty} -{ -} - #define DECLARE_MULTIPLE_GETTER(TYPE) \ void RangeHashedDictionary::get##TYPE( \ diff --git a/dbms/src/Dictionaries/RangeHashedDictionary.h b/dbms/src/Dictionaries/RangeHashedDictionary.h index a5043d40d5d..53e48ffe7eb 100644 --- a/dbms/src/Dictionaries/RangeHashedDictionary.h +++ b/dbms/src/Dictionaries/RangeHashedDictionary.h @@ -24,8 +24,6 @@ public: const DictionaryLifetime dict_lifetime, bool require_nonempty); - RangeHashedDictionary(const RangeHashedDictionary & other); - std::exception_ptr getCreationException() const override { return creation_exception; } std::string getName() const override { return dictionary_name; } @@ -44,7 +42,10 @@ public: bool isCached() const override { return false; } - std::unique_ptr clone() const override { return std::make_unique(*this); } + std::unique_ptr clone() const override + { + return std::make_unique(dictionary_name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty); + } const IDictionarySource * getSource() const override { return source_ptr.get(); } diff --git a/dbms/src/Dictionaries/TrieDictionary.cpp b/dbms/src/Dictionaries/TrieDictionary.cpp index 00bafdf62e3..9dc5fc0891d 100644 --- a/dbms/src/Dictionaries/TrieDictionary.cpp +++ b/dbms/src/Dictionaries/TrieDictionary.cpp @@ -63,11 +63,6 @@ TrieDictionary::TrieDictionary( creation_time = std::chrono::system_clock::now(); } -TrieDictionary::TrieDictionary(const TrieDictionary & other) - : TrieDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty} -{ -} - TrieDictionary::~TrieDictionary() { btrie_destroy(trie); diff --git a/dbms/src/Dictionaries/TrieDictionary.h b/dbms/src/Dictionaries/TrieDictionary.h index 1d370c8080c..3191916c094 100644 --- a/dbms/src/Dictionaries/TrieDictionary.h +++ b/dbms/src/Dictionaries/TrieDictionary.h @@ -29,8 +29,6 @@ public: const DictionaryLifetime dict_lifetime, bool require_nonempty); - TrieDictionary(const TrieDictionary & other); - ~TrieDictionary() override; std::string getKeyDescription() const { return key_description; } @@ -53,7 +51,10 @@ public: bool isCached() const override { return false; } - std::unique_ptr clone() const override { return std::make_unique(*this); } + std::unique_ptr clone() const override + { + return std::make_unique(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty); + } const IDictionarySource * getSource() const override { return source_ptr.get(); } diff --git a/dbms/src/Functions/if.cpp b/dbms/src/Functions/if.cpp index 41a5277401c..64fe301291c 100644 --- a/dbms/src/Functions/if.cpp +++ b/dbms/src/Functions/if.cpp @@ -904,7 +904,7 @@ public: using T0 = typename Types::LeftType; using T1 = typename Types::RightType; - if constexpr ((IsDecimalNumber && IsDecimalNumber) || (!IsDecimalNumber && !IsDecimalNumber)) + if constexpr (IsDecimalNumber == IsDecimalNumber) return executeTyped(cond_col, block, arguments, result, input_rows_count); else throw Exception("Conditional function with Decimal and non Decimal", ErrorCodes::NOT_IMPLEMENTED); diff --git a/dbms/src/Interpreters/IExternalLoadable.h b/dbms/src/Interpreters/IExternalLoadable.h index 2353ac288e2..c94d8d97a49 100644 --- a/dbms/src/Interpreters/IExternalLoadable.h +++ b/dbms/src/Interpreters/IExternalLoadable.h @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -26,7 +27,7 @@ struct ExternalLoadableLifetime final /// Basic interface for external loadable objects. Is used in ExternalLoader. -class IExternalLoadable : public std::enable_shared_from_this +class IExternalLoadable : public std::enable_shared_from_this, private boost::noncopyable { public: virtual ~IExternalLoadable() = default; diff --git a/dbms/src/Parsers/parseQuery.cpp b/dbms/src/Parsers/parseQuery.cpp index f40d156f17d..4f8ab83b7fd 100644 --- a/dbms/src/Parsers/parseQuery.cpp +++ b/dbms/src/Parsers/parseQuery.cpp @@ -354,8 +354,6 @@ std::pair splitMultipartQuery(const std::string & queries, s begin = pos; ast = parseQueryAndMovePosition(parser, pos, end, "", true, 0); - if (!ast) - break; ASTInsertQuery * insert = typeid_cast(ast.get()); From 9a22185bd6a128ac400babf4598d2f49b5257426 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 20 Jan 2019 02:51:03 +0300 Subject: [PATCH 45/71] Update ReadBufferFromHDFS.cpp --- dbms/src/IO/ReadBufferFromHDFS.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/IO/ReadBufferFromHDFS.cpp b/dbms/src/IO/ReadBufferFromHDFS.cpp index 39f7d1fcc7e..73edbc5fab9 100644 --- a/dbms/src/IO/ReadBufferFromHDFS.cpp +++ b/dbms/src/IO/ReadBufferFromHDFS.cpp @@ -19,7 +19,7 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl hdfsFile fin; HDFSBuilderPtr builder; HDFSFSPtr fs; - + ReadBufferFromHDFSImpl(const std::string & hdfs_name_) : hdfs_uri(hdfs_name_) , builder(createHDFSBuilder(hdfs_uri)) From a436915ed68afe7df979e0b3c10b631979d276b6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Jan 2019 03:10:27 +0300 Subject: [PATCH 46/71] Fixed test #3960 --- .../00814_replicated_minimalistic_part_header_zookeeper.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql b/dbms/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql index 53905a72ab4..bd45ede270f 100644 --- a/dbms/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql @@ -36,10 +36,10 @@ SELECT sleep(2) FORMAT Null; SELECT '*** Test part removal ***'; SELECT '*** replica 1 ***'; -SELECT name FROM system.parts WHERE database = 'test' AND table = 'part_header_r1'; +SELECT name FROM system.parts WHERE active AND database = 'test' AND table = 'part_header_r1'; SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test/part_header/replicas/1/parts'; SELECT '*** replica 2 ***'; -SELECT name FROM system.parts WHERE database = 'test' AND table = 'part_header_r2'; +SELECT name FROM system.parts WHERE active AND database = 'test' AND table = 'part_header_r2'; SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test/part_header/replicas/1/parts'; SELECT '*** Test ALTER ***'; From 05e713e724447c1108f10c3c10b0c9bf3626d9d8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Jan 2019 03:10:27 +0300 Subject: [PATCH 47/71] Fixed test #3960 --- .../00814_replicated_minimalistic_part_header_zookeeper.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql b/dbms/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql index 53905a72ab4..bd45ede270f 100644 --- a/dbms/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql @@ -36,10 +36,10 @@ SELECT sleep(2) FORMAT Null; SELECT '*** Test part removal ***'; SELECT '*** replica 1 ***'; -SELECT name FROM system.parts WHERE database = 'test' AND table = 'part_header_r1'; +SELECT name FROM system.parts WHERE active AND database = 'test' AND table = 'part_header_r1'; SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test/part_header/replicas/1/parts'; SELECT '*** replica 2 ***'; -SELECT name FROM system.parts WHERE database = 'test' AND table = 'part_header_r2'; +SELECT name FROM system.parts WHERE active AND database = 'test' AND table = 'part_header_r2'; SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test/part_header/replicas/1/parts'; SELECT '*** Test ALTER ***'; From afa9e8d4ea4cb3fd5f57af3b75132112b3ca1591 Mon Sep 17 00:00:00 2001 From: Alexander GQ Gerasiov Date: Mon, 21 Jan 2019 01:26:50 +0300 Subject: [PATCH 48/71] .gitignore: Move debian/ specific entries to debian/.gitignore Signed-off-by: Alexander GQ Gerasiov --- .gitignore | 11 ----------- debian/.gitignore | 12 ++++++++++++ 2 files changed, 12 insertions(+), 11 deletions(-) create mode 100644 debian/.gitignore diff --git a/.gitignore b/.gitignore index 3fa4a095a1b..a21df2e8e0f 100644 --- a/.gitignore +++ b/.gitignore @@ -180,18 +180,10 @@ utils/zookeeper-create-entry-to-download-part/zookeeper-create-entry-to-download utils/zookeeper-dump-tree/zookeeper-dump-tree utils/zookeeper-remove-by-list/zookeeper-remove-by-list dbms/src/Storages/tests/remove_symlink_directory -debian/control -debian/copyright -debian/tmp/ libs/libcommon/src/tests/json_test utils/compressor/zstd_test utils/wikistat-loader/wikistat-loader dbms/src/Common/tests/pod_array -debian/clickhouse-benchmark/ -debian/clickhouse-client/ -debian/clickhouse-server-base/ -debian/clickhouse-server-common/ -debian/files dbms/src/Server/data/* dbms/src/Server/metadata/* @@ -210,9 +202,6 @@ vgcore* *.changes build-stamp configure-stamp -debian/*.debhelper.log -debian/*.debhelper -debian/*.substvars *.bin *.mrk diff --git a/debian/.gitignore b/debian/.gitignore new file mode 100644 index 00000000000..3b10df86d93 --- /dev/null +++ b/debian/.gitignore @@ -0,0 +1,12 @@ +control +copyright +tmp/ +clickhouse-benchmark/ +clickhouse-client/ +clickhouse-server-base/ +clickhouse-server-common/ +files +*.debhelper.log +*.debhelper +*.substvars + From a0a0e1d3e579c2a29745b899dcbccdbd269ded6d Mon Sep 17 00:00:00 2001 From: Alexander GQ Gerasiov Date: Mon, 21 Jan 2019 01:35:57 +0300 Subject: [PATCH 49/71] Remove .travis.yml.bak --- .travis.yml.bak | 50 ------------------------------------------------- 1 file changed, 50 deletions(-) delete mode 100644 .travis.yml.bak diff --git a/.travis.yml.bak b/.travis.yml.bak deleted file mode 100644 index 0aed0a24427..00000000000 --- a/.travis.yml.bak +++ /dev/null @@ -1,50 +0,0 @@ -language: generic - -matrix: - fast_finish: true - include: - # We need to have gcc7 headers to compile c++17 code on clang -# - os: linux -# -# cache: -# ccache: true -# timeout: 1000 -# directories: -# - /home/travis/.ccache -# -# addons: -# apt: -# update: true -# sources: -# - ubuntu-toolchain-r-test -# - llvm-toolchain-trusty-5.0 -# packages: [ ninja-build, g++-7, clang-5.0, lld-5.0, libicu-dev, libreadline-dev, libmysqlclient-dev, unixodbc-dev, libltdl-dev, libssl-dev, libboost-dev, zlib1g-dev, libdouble-conversion-dev, libsparsehash-dev, librdkafka-dev, libcapnp-dev, libsparsehash-dev, libgoogle-perftools-dev, bash, expect, python, python-lxml, python-termcolor, curl, perl, sudo, openssl] -# -# env: -# - MATRIX_EVAL="export CC=clang-5.0 CXX=clang++-5.0" -# -# script: -# - utils/travis/normal.sh - - - os: linux - - sudo: required - - cache: - timeout: 1000 - directories: - - /var/cache/pbuilder/ccache - - addons: - apt: - update: true - packages: [ pbuilder, fakeroot, debhelper ] - - script: - - utils/travis/pbuilder.sh - - allow_failures: - - os: osx - -before_script: - - eval "${MATRIX_EVAL}" From 0728519b2ddd94f3c59fa79fd9ac97ffda4e605d Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 21 Jan 2019 11:46:28 +0300 Subject: [PATCH 50/71] Add image for stateful tests --- docker/test/stateful/Dockerfile | 29 ++++++++++ docker/test/stateful/listen.xml | 1 + docker/test/stateful/log_queries.xml | 7 +++ docker/test/stateful/part_log.xml | 8 +++ docker/test/stateful/s3downloader | 86 ++++++++++++++++++++++++++++ docker/test/stateful/zookeeper.xml | 16 ++++++ 6 files changed, 147 insertions(+) create mode 100644 docker/test/stateful/Dockerfile create mode 100644 docker/test/stateful/listen.xml create mode 100644 docker/test/stateful/log_queries.xml create mode 100644 docker/test/stateful/part_log.xml create mode 100755 docker/test/stateful/s3downloader create mode 100644 docker/test/stateful/zookeeper.xml diff --git a/docker/test/stateful/Dockerfile b/docker/test/stateful/Dockerfile new file mode 100644 index 00000000000..52275e2b43c --- /dev/null +++ b/docker/test/stateful/Dockerfile @@ -0,0 +1,29 @@ +FROM yandex/clickhouse-stateless-test + +RUN apt-get update -y \ + && env DEBIAN_FRONTEND=noninteractive \ + apt-get install --yes --no-install-recommends \ + python-requests + +COPY s3downloader /s3downloader + +ENV DATASETS="hits visits" + +CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ + dpkg -i package_folder/clickhouse-server_*.deb; \ + dpkg -i package_folder/clickhouse-client_*.deb; \ + dpkg -i package_folder/clickhouse-test_*.deb; \ + service zookeeper start; sleep 5; \ + service clickhouse-server start && sleep 5 \ + && /s3downloader --dataset-names $DATASETS \ + && chmod 777 -R /var/lib/clickhouse \ + && clickhouse-client --query "SHOW DATABASES" \ + && clickhouse-client --query "CREATE DATABASE datasets" \ + && clickhouse-client --query "CREATE DATABASE test" \ + && service clickhouse-server restart && sleep 5 \ + && clickhouse-client --query "SHOW TABLES FROM datasets" \ + && clickhouse-client --query "SHOW TABLES FROM test" \ + && clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" \ + && clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" \ + && clickhouse-client --query "SHOW TABLES FROM test" \ + && clickhouse-test --shard --zookeeper --no-stateless $SKIP_TESTS_OPTION 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt diff --git a/docker/test/stateful/listen.xml b/docker/test/stateful/listen.xml new file mode 100644 index 00000000000..675225f9575 --- /dev/null +++ b/docker/test/stateful/listen.xml @@ -0,0 +1 @@ +:: diff --git a/docker/test/stateful/log_queries.xml b/docker/test/stateful/log_queries.xml new file mode 100644 index 00000000000..25261072ade --- /dev/null +++ b/docker/test/stateful/log_queries.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/docker/test/stateful/part_log.xml b/docker/test/stateful/part_log.xml new file mode 100644 index 00000000000..6c6fc9c6982 --- /dev/null +++ b/docker/test/stateful/part_log.xml @@ -0,0 +1,8 @@ + + + system + part_log
+ + 7500 +
+
diff --git a/docker/test/stateful/s3downloader b/docker/test/stateful/s3downloader new file mode 100755 index 00000000000..f8e2bf3cbe4 --- /dev/null +++ b/docker/test/stateful/s3downloader @@ -0,0 +1,86 @@ +#!/usr/bin/env python +# -*- coding: utf-8 -*- +import os +import sys +import tarfile +import logging +import argparse +import requests +import tempfile + + +DEFAULT_URL = 'https://clickhouse-datasets.s3.yandex.net' + +AVAILABLE_DATASETS = { + 'hits': 'hits_v1.tar', + 'visits': 'visits_v1.tar', +} + +def _get_temp_file_name(): + return os.path.join(tempfile._get_default_tempdir(), next(tempfile._get_candidate_names())) + +def build_url(base_url, dataset): + return os.path.join(base_url, dataset, 'partitions', AVAILABLE_DATASETS[dataset]) + +def dowload_with_progress(url, path): + logging.info("Downloading from %s to temp path %s", url, path) + with open(path, 'w') 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) + sys.stdout.write("\r[{}{}] {}%".format('=' * done, ' ' * (50-done), percent)) + sys.stdout.flush() + sys.stdout.write("\n") + logging.info("Downloading finished") + +def unpack_to_clickhouse_directory(tar_path, clickhouse_path): + logging.info("Will unpack data from temp path %s to clickhouse db %s", tar_path, clickhouse_path) + with tarfile.open(tar_path, 'r') as comp_file: + comp_file.extractall(path=clickhouse_path) + logging.info("Unpack finished") + + +if __name__ == "__main__": + logging.basicConfig(level=logging.INFO) + + parser = argparse.ArgumentParser( + description="Simple tool for dowloading datasets for clickhouse from S3") + + parser.add_argument('--dataset-names', required=True, nargs='+', choices=AVAILABLE_DATASETS.keys()) + parser.add_argument('--url-prefix', default=DEFAULT_URL) + parser.add_argument('--clickhouse-data-path', default='/var/lib/clickhouse/') + + args = parser.parse_args() + datasets = args.dataset_names + logging.info("Will fetch following datasets: %s", ', '.join(datasets)) + for dataset in datasets: + logging.info("Processing %s", dataset) + temp_archive_path = _get_temp_file_name() + try: + download_url_for_dataset = build_url(args.url_prefix, dataset) + dowload_with_progress(download_url_for_dataset, temp_archive_path) + unpack_to_clickhouse_directory(temp_archive_path, args.clickhouse_data_path) + except Exception as ex: + logging.info("Some exception occured %s", str(ex)) + raise + finally: + logging.info("Will remove dowloaded file %s from filesystem if it exists", temp_archive_path) + if os.path.exists(temp_archive_path): + os.remove(temp_archive_path) + logging.info("Processing of %s finished", dataset) + logging.info("Fetch finished, enjoy your tables!") + + diff --git a/docker/test/stateful/zookeeper.xml b/docker/test/stateful/zookeeper.xml new file mode 100644 index 00000000000..095f4be78c1 --- /dev/null +++ b/docker/test/stateful/zookeeper.xml @@ -0,0 +1,16 @@ + + + + localhost + 2181 + + + yandex.ru + 2181 + + + 111.0.1.2 + 2181 + + + From dfe0a734cd9e1d96921bf9603008596d97afefc9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 21 Jan 2019 12:48:35 +0300 Subject: [PATCH 51/71] Auto version update to [19.1.2] [54413] --- dbms/cmake/version.cmake | 8 ++++---- debian/changelog | 4 ++-- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index 34edab61470..aa1fe079457 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -2,10 +2,10 @@ set(VERSION_REVISION 54413) set(VERSION_MAJOR 19) set(VERSION_MINOR 1) -set(VERSION_PATCH 1) -set(VERSION_GITHASH 4e7747117123f5a1b027a64865844b4faa10447d) -set(VERSION_DESCRIBE v19.1.1-testing) -set(VERSION_STRING 19.1.1) +set(VERSION_PATCH 2) +set(VERSION_GITHASH 0728519b2ddd94f3c59fa79fd9ac97ffda4e605d) +set(VERSION_DESCRIBE v19.1.2-testing) +set(VERSION_STRING 19.1.2) # end of autochange set(VERSION_EXTRA "" CACHE STRING "") diff --git a/debian/changelog b/debian/changelog index 08dd7ff59a8..bc2013e8b1f 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (19.1.1) unstable; urgency=low +clickhouse (19.1.2) unstable; urgency=low * Modified source code - -- Wed, 16 Jan 2019 14:04:37 +0300 + -- Mon, 21 Jan 2019 12:48:35 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index 37d7ca0a7d1..ef2f40cf0eb 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.1.1 +ARG version=19.1.2 RUN apt-get update \ && apt-get install --yes --no-install-recommends \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index 4f2c8ae839d..eb80f2323b9 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.1.1 +ARG version=19.1.2 ARG gosu_ver=1.10 RUN apt-get update \ diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index 6bd520a151b..78af590bcbf 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.1.1 +ARG version=19.1.2 RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ From ac0060079ab221278338db343ca9eaf006fc4ee1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 21 Jan 2019 16:12:44 +0300 Subject: [PATCH 52/71] Fixed warnings in SpecializedAggregator --- dbms/src/Interpreters/Compiler.cpp | 4 ++-- dbms/src/Interpreters/SpecializedAggregator.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Interpreters/Compiler.cpp b/dbms/src/Interpreters/Compiler.cpp index 8a60b24a24b..0d8fc36f840 100644 --- a/dbms/src/Interpreters/Compiler.cpp +++ b/dbms/src/Interpreters/Compiler.cpp @@ -186,10 +186,10 @@ SharedLibraryPtr Compiler::getOrCount( static void addCodeToAssertHeadersMatch(WriteBuffer & out) { out << - "#define STRING2(x) #x\n" - "#define STRING(x) STRING2(x)\n" "#include \n" "#if VERSION_REVISION != " << ClickHouseRevision::get() << "\n" + "#define STRING2(x) #x\n" + "#define STRING(x) STRING2(x)\n" "#pragma message \"ClickHouse headers revision = \" STRING(VERSION_REVISION) \n" "#error \"ClickHouse headers revision doesn't match runtime revision of the server (" << ClickHouseRevision::get() << ").\"\n" "#endif\n\n"; diff --git a/dbms/src/Interpreters/SpecializedAggregator.h b/dbms/src/Interpreters/SpecializedAggregator.h index 615911a7224..de7ea6fd921 100644 --- a/dbms/src/Interpreters/SpecializedAggregator.h +++ b/dbms/src/Interpreters/SpecializedAggregator.h @@ -136,7 +136,7 @@ void NO_INLINE Aggregator::executeSpecializedCase( AggregateDataPtr overflow_row) const { /// For all rows. - typename Method::Key prev_key; + typename Method::Key prev_key{}; AggregateDataPtr value = nullptr; for (size_t i = 0; i < rows; ++i) { From 436cf294db169b4ff24ef2825d46421b96271c0d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 21 Jan 2019 16:26:13 +0300 Subject: [PATCH 53/71] Auto version update to [19.1.3] [54413] --- dbms/cmake/version.cmake | 8 ++++---- debian/changelog | 4 ++-- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index aa1fe079457..ccb379a4a69 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -2,10 +2,10 @@ set(VERSION_REVISION 54413) set(VERSION_MAJOR 19) set(VERSION_MINOR 1) -set(VERSION_PATCH 2) -set(VERSION_GITHASH 0728519b2ddd94f3c59fa79fd9ac97ffda4e605d) -set(VERSION_DESCRIBE v19.1.2-testing) -set(VERSION_STRING 19.1.2) +set(VERSION_PATCH 3) +set(VERSION_GITHASH ac0060079ab221278338db343ca9eaf006fc4ee1) +set(VERSION_DESCRIBE v19.1.3-testing) +set(VERSION_STRING 19.1.3) # end of autochange set(VERSION_EXTRA "" CACHE STRING "") diff --git a/debian/changelog b/debian/changelog index bc2013e8b1f..0b436d08615 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (19.1.2) unstable; urgency=low +clickhouse (19.1.3) unstable; urgency=low * Modified source code - -- Mon, 21 Jan 2019 12:48:35 +0300 + -- Mon, 21 Jan 2019 16:26:13 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index ef2f40cf0eb..c607bbb2f46 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.1.2 +ARG version=19.1.3 RUN apt-get update \ && apt-get install --yes --no-install-recommends \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index eb80f2323b9..4d40cbe7714 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.1.2 +ARG version=19.1.3 ARG gosu_ver=1.10 RUN apt-get update \ diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index 78af590bcbf..c7b325e4d58 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.1.2 +ARG version=19.1.3 RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ From 34fb1c89f1c4c5e57e249657fedd4359e45aa9b6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 21 Jan 2019 17:00:06 +0300 Subject: [PATCH 54/71] Add ability to choose codecs for storage log and tiny log --- dbms/src/Storages/ColumnsDescription.cpp | 6 + dbms/src/Storages/ColumnsDescription.h | 2 + dbms/src/Storages/StorageLog.cpp | 11 +- dbms/src/Storages/StorageTinyLog.cpp | 7 +- ...m_compression_codes_log_storages.reference | 26 ++++ ..._custom_compression_codes_log_storages.sql | 118 ++++++++++++++++++ 6 files changed, 165 insertions(+), 5 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.reference create mode 100644 dbms/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql diff --git a/dbms/src/Storages/ColumnsDescription.cpp b/dbms/src/Storages/ColumnsDescription.cpp index c64f78b330d..783d2d3397d 100644 --- a/dbms/src/Storages/ColumnsDescription.cpp +++ b/dbms/src/Storages/ColumnsDescription.cpp @@ -240,6 +240,12 @@ CompressionCodecPtr ColumnsDescription::getCodecOrDefault(const String & column_ return codec->second; } + +CompressionCodecPtr ColumnsDescription::getCodecOrDefault(const String & column_name) const +{ + return getCodecOrDefault(column_name, CompressionCodecFactory::instance().getDefaultCodec()); +} + ColumnsDescription ColumnsDescription::parse(const String & str) { ReadBufferFromString buf{str}; diff --git a/dbms/src/Storages/ColumnsDescription.h b/dbms/src/Storages/ColumnsDescription.h index edeabf63dfb..e54dddde1de 100644 --- a/dbms/src/Storages/ColumnsDescription.h +++ b/dbms/src/Storages/ColumnsDescription.h @@ -69,6 +69,8 @@ struct ColumnsDescription CompressionCodecPtr getCodecOrDefault(const String & column_name, CompressionCodecPtr default_codec) const; + CompressionCodecPtr getCodecOrDefault(const String & column_name) const; + static ColumnsDescription parse(const String & str); static const ColumnsDescription * loadFromContext(const Context & context, const String & db, const String & table); diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index e04d02e7de9..5ee8726dd52 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -144,9 +144,9 @@ private: struct Stream { - Stream(const std::string & data_path, size_t max_compress_block_size) : + Stream(const std::string & data_path, CompressionCodecPtr codec, size_t max_compress_block_size) : plain(data_path, max_compress_block_size, O_APPEND | O_CREAT | O_WRONLY), - compressed(plain, CompressionCodecFactory::instance().getDefaultCodec(), max_compress_block_size) + compressed(plain, codec, max_compress_block_size) { plain_offset = Poco::File(data_path).getSize(); } @@ -355,7 +355,12 @@ void LogBlockOutputStream::writeData(const String & name, const IDataType & type if (written_streams.count(stream_name)) return; - streams.try_emplace(stream_name, storage.files[stream_name].data_file.path(), storage.max_compress_block_size); + const auto & columns = storage.getColumns(); + streams.try_emplace( + stream_name, + storage.files[stream_name].data_file.path(), + columns.getCodecOrDefault(name), + storage.max_compress_block_size); }, settings.path); settings.getter = createStreamGetter(name, written_streams); diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index d3259ff859c..9f87fb172fc 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -135,9 +135,9 @@ private: struct Stream { - Stream(const std::string & data_path, size_t max_compress_block_size) : + Stream(const std::string & data_path, CompressionCodecPtr codec, size_t max_compress_block_size) : plain(data_path, max_compress_block_size, O_APPEND | O_CREAT | O_WRONLY), - compressed(plain, CompressionCodecFactory::instance().getDefaultCodec(), max_compress_block_size) + compressed(plain, codec, max_compress_block_size) { } @@ -237,6 +237,7 @@ void TinyLogBlockInputStream::readData(const String & name, const IDataType & ty IDataType::OutputStreamGetter TinyLogBlockOutputStream::createStreamGetter(const String & name, WrittenStreams & written_streams) { + return [&] (const IDataType::SubstreamPath & path) -> WriteBuffer * { String stream_name = IDataType::getFileNameForStream(name, path); @@ -244,8 +245,10 @@ IDataType::OutputStreamGetter TinyLogBlockOutputStream::createStreamGetter(const if (!written_streams.insert(stream_name).second) return nullptr; + const auto & columns = storage.getColumns(); if (!streams.count(stream_name)) streams[stream_name] = std::make_unique(storage.files[stream_name].data_file.path(), + columns.getCodecOrDefault(name), storage.max_compress_block_size); return &streams[stream_name]->compressed; diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.reference b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.reference new file mode 100644 index 00000000000..8104bd3831f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.reference @@ -0,0 +1,26 @@ +CREATE TABLE test.compression_codec_log ( id UInt64 CODEC(LZ4), data String CODEC(ZSTD(1)), ddd Date CODEC(NONE), somenum Float64 CODEC(ZSTD(2)), somestr FixedString(3) CODEC(LZ4HC(7)), othernum Int64 CODEC(Delta(8))) ENGINE = Log() +1 hello 2018-12-14 1.1 aaa 5 +2 world 2018-12-15 2.2 bbb 6 +3 ! 2018-12-16 3.3 ccc 7 +2 +CREATE TABLE test.compression_codec_multiple_log ( id UInt64 CODEC(LZ4, ZSTD(1), NONE, LZ4HC(0), Delta(4)), data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC(0), LZ4, LZ4, Delta(8)), ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD(1), LZ4HC(0), LZ4HC(0)), somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD(1))) ENGINE = Log() +1 world 2018-10-05 1.1 +2 hello 2018-10-01 2.2 +3 buy 2018-10-11 3.3 +10003 +10003 +274972506.6 +9175437371954010821 +CREATE TABLE test.compression_codec_tiny_log ( id UInt64 CODEC(LZ4), data String CODEC(ZSTD(1)), ddd Date CODEC(NONE), somenum Float64 CODEC(ZSTD(2)), somestr FixedString(3) CODEC(LZ4HC(7)), othernum Int64 CODEC(Delta(8))) ENGINE = TinyLog() +1 hello 2018-12-14 1.1 aaa 5 +2 world 2018-12-15 2.2 bbb 6 +3 ! 2018-12-16 3.3 ccc 7 +2 +CREATE TABLE test.compression_codec_multiple_tiny_log ( id UInt64 CODEC(LZ4, ZSTD(1), NONE, LZ4HC(0), Delta(4)), data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC(0), LZ4, LZ4, Delta(8)), ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD(1), LZ4HC(0), LZ4HC(0)), somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD(1))) ENGINE = TinyLog() +1 world 2018-10-05 1.1 +2 hello 2018-10-01 2.2 +3 buy 2018-10-11 3.3 +10003 +10003 +274972506.6 +9175437371954010821 diff --git a/dbms/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql new file mode 100644 index 00000000000..7798cf7223e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00804_test_custom_compression_codes_log_storages.sql @@ -0,0 +1,118 @@ +SET send_logs_level = 'none'; + +-- copy-paste for storage log + +DROP TABLE IF EXISTS test.compression_codec_log; + +CREATE TABLE test.compression_codec_log( + id UInt64 CODEC(LZ4), + data String CODEC(ZSTD), + ddd Date CODEC(NONE), + somenum Float64 CODEC(ZSTD(2)), + somestr FixedString(3) CODEC(LZ4HC(7)), + othernum Int64 CODEC(Delta) +) ENGINE = Log(); + +SHOW CREATE TABLE test.compression_codec_log; + +INSERT INTO test.compression_codec_log VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5); +INSERT INTO test.compression_codec_log VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6); +INSERT INTO test.compression_codec_log VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7); + +SELECT * FROM test.compression_codec_log ORDER BY id; + +INSERT INTO test.compression_codec_log VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8); + +DETACH TABLE test.compression_codec_log; +ATTACH TABLE test.compression_codec_log; + +SELECT count(*) FROM test.compression_codec_log WHERE id = 2 GROUP BY id; + +DROP TABLE IF EXISTS test.compression_codec_log; + +DROP TABLE IF EXISTS test.compression_codec_multiple_log; + +CREATE TABLE test.compression_codec_multiple_log ( + id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4)), + data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8)), + ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC), + somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD) +) ENGINE = Log(); + +SHOW CREATE TABLE test.compression_codec_multiple_log; + +INSERT INTO test.compression_codec_multiple_log VALUES (1, 'world', toDate('2018-10-05'), 1.1), (2, 'hello', toDate('2018-10-01'), 2.2), (3, 'buy', toDate('2018-10-11'), 3.3); + +SELECT * FROM test.compression_codec_multiple_log ORDER BY id; + +INSERT INTO test.compression_codec_multiple_log select modulo(number, 100), toString(number), toDate('2018-12-01'), 5.5 * number FROM system.numbers limit 10000; + +SELECT count(*) FROM test.compression_codec_multiple_log; + +SELECT count(distinct data) FROM test.compression_codec_multiple_log; + +SELECT floor(sum(somenum), 1) FROM test.compression_codec_multiple_log; + +TRUNCATE TABLE test.compression_codec_multiple_log; + +INSERT INTO test.compression_codec_multiple_log select modulo(number, 100), toString(number), toDate('2018-12-01'), 5.5 * number FROM system.numbers limit 10000; + +SELECT sum(cityHash64(*)) FROM test.compression_codec_multiple_log; + +-- copy-paste for storage tiny log +DROP TABLE IF EXISTS test.compression_codec_tiny_log; + +CREATE TABLE test.compression_codec_tiny_log( + id UInt64 CODEC(LZ4), + data String CODEC(ZSTD), + ddd Date CODEC(NONE), + somenum Float64 CODEC(ZSTD(2)), + somestr FixedString(3) CODEC(LZ4HC(7)), + othernum Int64 CODEC(Delta) +) ENGINE = TinyLog(); + +SHOW CREATE TABLE test.compression_codec_tiny_log; + +INSERT INTO test.compression_codec_tiny_log VALUES(1, 'hello', toDate('2018-12-14'), 1.1, 'aaa', 5); +INSERT INTO test.compression_codec_tiny_log VALUES(2, 'world', toDate('2018-12-15'), 2.2, 'bbb', 6); +INSERT INTO test.compression_codec_tiny_log VALUES(3, '!', toDate('2018-12-16'), 3.3, 'ccc', 7); + +SELECT * FROM test.compression_codec_tiny_log ORDER BY id; + +INSERT INTO test.compression_codec_tiny_log VALUES(2, '', toDate('2018-12-13'), 4.4, 'ddd', 8); + +DETACH TABLE test.compression_codec_tiny_log; +ATTACH TABLE test.compression_codec_tiny_log; + +SELECT count(*) FROM test.compression_codec_tiny_log WHERE id = 2 GROUP BY id; + +DROP TABLE IF EXISTS test.compression_codec_tiny_log; + +DROP TABLE IF EXISTS test.compression_codec_multiple_tiny_log; + +CREATE TABLE test.compression_codec_multiple_tiny_log ( + id UInt64 CODEC(LZ4, ZSTD, NONE, LZ4HC, Delta(4)), + data String CODEC(ZSTD(2), NONE, Delta(2), LZ4HC, LZ4, LZ4, Delta(8)), + ddd Date CODEC(NONE, NONE, NONE, Delta(1), LZ4, ZSTD, LZ4HC, LZ4HC), + somenum Float64 CODEC(Delta(4), LZ4, LZ4, ZSTD(2), LZ4HC(5), ZSTD(3), ZSTD) +) ENGINE = TinyLog(); + +SHOW CREATE TABLE test.compression_codec_multiple_tiny_log; + +INSERT INTO test.compression_codec_multiple_tiny_log VALUES (1, 'world', toDate('2018-10-05'), 1.1), (2, 'hello', toDate('2018-10-01'), 2.2), (3, 'buy', toDate('2018-10-11'), 3.3); + +SELECT * FROM test.compression_codec_multiple_tiny_log ORDER BY id; + +INSERT INTO test.compression_codec_multiple_tiny_log select modulo(number, 100), toString(number), toDate('2018-12-01'), 5.5 * number FROM system.numbers limit 10000; + +SELECT count(*) FROM test.compression_codec_multiple_tiny_log; + +SELECT count(distinct data) FROM test.compression_codec_multiple_tiny_log; + +SELECT floor(sum(somenum), 1) FROM test.compression_codec_multiple_tiny_log; + +TRUNCATE TABLE test.compression_codec_multiple_tiny_log; + +INSERT INTO test.compression_codec_multiple_tiny_log select modulo(number, 100), toString(number), toDate('2018-12-01'), 5.5 * number FROM system.numbers limit 10000; + +SELECT sum(cityHash64(*)) FROM test.compression_codec_multiple_tiny_log; From 88c413fbe2565cb91ec232d38504a472118e9464 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 21 Jan 2019 17:03:34 +0300 Subject: [PATCH 55/71] Remove accidental change --- dbms/src/Storages/StorageTinyLog.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index 9f87fb172fc..2d6b3983af4 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -237,7 +237,6 @@ void TinyLogBlockInputStream::readData(const String & name, const IDataType & ty IDataType::OutputStreamGetter TinyLogBlockOutputStream::createStreamGetter(const String & name, WrittenStreams & written_streams) { - return [&] (const IDataType::SubstreamPath & path) -> WriteBuffer * { String stream_name = IDataType::getFileNameForStream(name, path); From 2eb7b152734a1e2b03e71a65cc674a99d8b9b9ee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 21 Jan 2019 19:14:26 +0300 Subject: [PATCH 56/71] Addition to prev. revision #4103 --- dbms/programs/client/TestHint.h | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/dbms/programs/client/TestHint.h b/dbms/programs/client/TestHint.h index 66e86b5e750..b260bead0ba 100644 --- a/dbms/programs/client/TestHint.h +++ b/dbms/programs/client/TestHint.h @@ -5,8 +5,6 @@ #include #include #include -#include -#include #include @@ -93,11 +91,15 @@ private: void parse(const String & hint) { - ReadBufferFromString ss(hint); + std::stringstream ss; + ss << hint; + String item; + while (!ss.eof()) { - String item; ss >> item; + if (ss.eof()) + break; if (item == "serverError") ss >> server_error; From f7ecfc776b3131209b05007e54f33aeb72993975 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 21 Jan 2019 21:04:08 +0300 Subject: [PATCH 57/71] Miscellaneous #4111 --- dbms/src/Compression/CompressedWriteBuffer.cpp | 2 +- dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp | 2 +- dbms/src/Storages/StorageLog.cpp | 2 +- dbms/src/Storages/StorageTinyLog.cpp | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Compression/CompressedWriteBuffer.cpp b/dbms/src/Compression/CompressedWriteBuffer.cpp index 7fc8d5ab5f9..1285949c863 100644 --- a/dbms/src/Compression/CompressedWriteBuffer.cpp +++ b/dbms/src/Compression/CompressedWriteBuffer.cpp @@ -43,7 +43,7 @@ CompressedWriteBuffer::CompressedWriteBuffer( WriteBuffer & out_, CompressionCodecPtr codec_, size_t buf_size) - : BufferWithOwnMemory(buf_size), out(out_), codec(codec_) + : BufferWithOwnMemory(buf_size), out(out_), codec(std::move(codec_)) { } diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index ff47b97fd60..b64543124b8 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -32,7 +32,7 @@ IMergedBlockOutputStream::IMergedBlockOutputStream( min_compress_block_size(min_compress_block_size_), max_compress_block_size(max_compress_block_size_), aio_threshold(aio_threshold_), - codec(codec_) + codec(std::move(codec_)) { } diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index 5ee8726dd52..1f2fa87da53 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -146,7 +146,7 @@ private: { Stream(const std::string & data_path, CompressionCodecPtr codec, size_t max_compress_block_size) : plain(data_path, max_compress_block_size, O_APPEND | O_CREAT | O_WRONLY), - compressed(plain, codec, max_compress_block_size) + compressed(plain, std::move(codec), max_compress_block_size) { plain_offset = Poco::File(data_path).getSize(); } diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index 2d6b3983af4..c7546742202 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -137,7 +137,7 @@ private: { Stream(const std::string & data_path, CompressionCodecPtr codec, size_t max_compress_block_size) : plain(data_path, max_compress_block_size, O_APPEND | O_CREAT | O_WRONLY), - compressed(plain, codec, max_compress_block_size) + compressed(plain, std::move(codec), max_compress_block_size) { } From c70e8cc5f0b250b4c472c238d37d4418e9823bcf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 21 Jan 2019 22:45:26 +0300 Subject: [PATCH 58/71] Miscellaneous #3726 --- dbms/src/Interpreters/Cluster.cpp | 20 +++++++------------ dbms/src/Interpreters/Cluster.h | 10 +++++----- dbms/src/Interpreters/DDLWorker.cpp | 6 ++---- .../Storages/Distributed/DirectoryMonitor.cpp | 3 +-- .../DistributedBlockOutputStream.cpp | 2 +- 5 files changed, 16 insertions(+), 25 deletions(-) diff --git a/dbms/src/Interpreters/Cluster.cpp b/dbms/src/Interpreters/Cluster.cpp index a74286dd1d0..afeed3ba577 100644 --- a/dbms/src/Interpreters/Cluster.cpp +++ b/dbms/src/Interpreters/Cluster.cpp @@ -104,18 +104,17 @@ String Cluster::Address::readableString() const return res; } -void Cluster::Address::fromString(const String & host_port_string, String & host_name, UInt16 & port) +std::pair Cluster::Address::fromString(const String & host_port_string) { auto pos = host_port_string.find_last_of(':'); if (pos == std::string::npos) throw Exception("Incorrect : format " + host_port_string, ErrorCodes::SYNTAX_ERROR); - host_name = unescapeForFileName(host_port_string.substr(0, pos)); - port = parse(host_port_string.substr(pos + 1)); + return {unescapeForFileName(host_port_string.substr(0, pos)), parse(host_port_string.substr(pos + 1))}; } -String Cluster::Address::toStringFull() const +String Cluster::Address::toFullString() const { return escapeForFileName(user) + @@ -126,7 +125,7 @@ String Cluster::Address::toStringFull() const + ((secure == Protocol::Secure::Enable) ? "+secure" : ""); } -void Cluster::Address::fromFullString(const String & full_string, Cluster::Address & address) +Cluster::Address Cluster::Address::fromFullString(const String & full_string) { const char * address_begin = full_string.data(); const char * address_end = address_begin + full_string.size(); @@ -152,19 +151,14 @@ void Cluster::Address::fromFullString(const String & full_string, Cluster::Addre const char * has_db = strchr(full_string.data(), '#'); const char * port_end = has_db ? has_db : address_end; + Address address; address.secure = secure; address.port = parse(host_end + 1, port_end - (host_end + 1)); address.host_name = unescapeForFileName(std::string(user_pw_end + 1, host_end)); address.user = unescapeForFileName(std::string(address_begin, has_pw ? colon : user_pw_end)); address.password = has_pw ? unescapeForFileName(std::string(colon + 1, user_pw_end)) : std::string(); address.default_database = has_db ? unescapeForFileName(std::string(has_db + 1, address_end)) : std::string(); -} - -bool Cluster::Address::operator==(const Cluster::Address & other) const -{ - return other.host_name == host_name && other.port == port - && other.secure == secure && other.user == user - && other.password == password && other.default_database == default_database; + return address; } @@ -303,7 +297,7 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, const Setting { if (internal_replication) { - auto dir_name = replica_addresses.back().toStringFull(); + auto dir_name = replica_addresses.back().toFullString(); if (first) dir_name_for_internal_replication = dir_name; else diff --git a/dbms/src/Interpreters/Cluster.h b/dbms/src/Interpreters/Cluster.h index 999f90043e9..0d2d83f7bf7 100644 --- a/dbms/src/Interpreters/Cluster.h +++ b/dbms/src/Interpreters/Cluster.h @@ -78,12 +78,11 @@ public: static String toString(const String & host_name, UInt16 port); - static void fromString(const String & host_port_string, String & host_name, UInt16 & port); + static std::pair fromString(const String & host_port_string); /// Retrurns escaped user:password@resolved_host_address:resolved_host_port#default_database - String toStringFull() const; - - static void fromFullString(const String & address_full_string, Address & address); + String toFullString() const; + static Address fromFullString(const String & address_full_string); /// Returns initially resolved address Poco::Net::SocketAddress getResolvedAddress() const @@ -91,7 +90,8 @@ public: return initially_resolved_address; } - bool operator==(const Address & other) const; + auto tuple() const { return std::tie(host_name, port, secure, user, password, default_database); } + bool operator==(const Address & other) const { return tuple() == other.tuple(); } private: Poco::Net::SocketAddress initially_resolved_address; diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 54fcffbea2a..7671fcb1b9b 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -70,7 +70,7 @@ struct HostID static HostID fromString(const String & host_port_str) { HostID res; - Cluster::Address::fromString(host_port_str, res.host_name, res.port); + std::tie(res.host_name, res.port) = Cluster::Address::fromString(host_port_str); return res; } @@ -1076,9 +1076,7 @@ public: status.tryDeserializeText(status_data); } - String host; - UInt16 port; - Cluster::Address::fromString(host_id, host, port); + auto [host, port] = Cluster::Address::fromString(host_id); if (status.code != 0 && first_exception == nullptr) first_exception = std::make_unique("There was an error on [" + host + ":" + toString(port) + "]: " + status.message, status.code); diff --git a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp index 29a77557c6b..3cd51d0bf98 100644 --- a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp @@ -48,8 +48,7 @@ namespace for (auto it = boost::make_split_iterator(name, boost::first_finder(",")); it != decltype(it){}; ++it) { - Cluster::Address address; - Cluster::Address::fromFullString(boost::copy_range(*it), address); + Cluster::Address address = Cluster::Address::fromFullString(boost::copy_range(*it)); pools.emplace_back(factory(address)); } diff --git a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 19ba8e48653..afbc7855c8f 100644 --- a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -494,7 +494,7 @@ void DistributedBlockOutputStream::writeAsyncImpl(const Block & block, const siz std::vector dir_names; for (const auto & address : cluster->getShardsAddresses()[shard_id]) if (!address.is_local) - dir_names.push_back(address.toStringFull()); + dir_names.push_back(address.toFullString()); if (!dir_names.empty()) writeToShard(block, dir_names); From 1d0bee7deeccf59f59179708e2508d49adc8de6f Mon Sep 17 00:00:00 2001 From: chertus Date: Mon, 21 Jan 2019 22:56:11 +0300 Subject: [PATCH 59/71] fix collecting required columns in lambda --- dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp | 2 +- .../queries/0_stateless/00819_ast_refactoring_bugs.reference | 0 .../tests/queries/0_stateless/00819_ast_refactoring_bugs.sql | 5 +++++ 3 files changed, 6 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.reference create mode 100644 dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.sql diff --git a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp index 7d93da95e8d..5dc479fee5e 100644 --- a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -157,7 +157,7 @@ void RequiredSourceColumnsMatcher::visit(const ASTFunction & node, const ASTPtr local_aliases.push_back(name); /// visit child with masked local aliases - visit(node.arguments->children[1], data); + RequiredSourceColumnsVisitor(data).visit(node.arguments->children[1]); for (const auto & name : local_aliases) data.private_aliases.erase(name); diff --git a/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.reference b/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.sql b/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.sql new file mode 100644 index 00000000000..98ae13d2b9e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.sql @@ -0,0 +1,5 @@ +CREATE TABLE IF NOT EXISTS test.sign (Sign Int8, Arr Array(Int8)) ENGINE = Memory; + +SELECT arrayMap(x -> x * Sign, Arr) FROM test.sign; + +DROP TABLE test.sign; From 24fc3ad544d6f29dfd505411c3a8a4eeeb21ced8 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 22 Jan 2019 02:29:09 +0300 Subject: [PATCH 60/71] Auto version update to [19.1.4] [54413] --- dbms/cmake/version.cmake | 8 ++++---- debian/changelog | 4 ++-- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index ccb379a4a69..d1656c9f90b 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -2,10 +2,10 @@ set(VERSION_REVISION 54413) set(VERSION_MAJOR 19) set(VERSION_MINOR 1) -set(VERSION_PATCH 3) -set(VERSION_GITHASH ac0060079ab221278338db343ca9eaf006fc4ee1) -set(VERSION_DESCRIBE v19.1.3-testing) -set(VERSION_STRING 19.1.3) +set(VERSION_PATCH 4) +set(VERSION_GITHASH be762f58e1d0286c54609301cabc1934f49257fc) +set(VERSION_DESCRIBE v19.1.4-testing) +set(VERSION_STRING 19.1.4) # end of autochange set(VERSION_EXTRA "" CACHE STRING "") diff --git a/debian/changelog b/debian/changelog index 0b436d08615..36903d88d05 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (19.1.3) unstable; urgency=low +clickhouse (19.1.4) unstable; urgency=low * Modified source code - -- Mon, 21 Jan 2019 16:26:13 +0300 + -- Tue, 22 Jan 2019 02:29:09 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index c607bbb2f46..7cf376c6ed1 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.1.3 +ARG version=19.1.4 RUN apt-get update \ && apt-get install --yes --no-install-recommends \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index 4d40cbe7714..a45221e060e 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.1.3 +ARG version=19.1.4 ARG gosu_ver=1.10 RUN apt-get update \ diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index c7b325e4d58..2a21d932422 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.1.3 +ARG version=19.1.4 RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ From a9054e05f05ccf58416dc091002313db3c814d2a Mon Sep 17 00:00:00 2001 From: ogorbacheva Date: Tue, 22 Jan 2019 16:21:20 +0300 Subject: [PATCH 61/71] Doc fix: edit info about escaping (#4118) * Doc fix: delete VerticalRaw format and remove escaping from Pretty and Vertical formats * Doc fix: editing text about escaping * Doc fix: fix example in pretty format * Doc fix: fix example in pretty format * Doc fix: last fixes * Doc fix: last fixes * Doc fix --- docs/en/interfaces/formats.md | 48 ++++++++++++++++------------------- 1 file changed, 22 insertions(+), 26 deletions(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index c47e75142df..eddefaa9394 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -14,7 +14,6 @@ The table below lists supported formats and how they can be used in `INSERT` and | [CSVWithNames](#csvwithnames) | ✔ | ✔ | | [Values](#values) | ✔ | ✔ | | [Vertical](#vertical) | ✗ | ✔ | -| [VerticalRaw](#verticalraw) | ✗ | ✔ | | [JSON](#json) | ✗ | ✔ | | [JSONCompact](#jsoncompact) | ✗ | ✔ | | [JSONEachRow](#jsoneachrow) | ✔ | ✔ | @@ -345,6 +344,8 @@ Each result block is output as a separate table. This is necessary so that block [NULL](../query_language/syntax.md) is output as `ᴺᵁᴸᴸ`. +Example (shown for the [PrettyCompact](#prettycompact) format): + ``` sql SELECT * FROM t_null ``` @@ -355,10 +356,22 @@ SELECT * FROM t_null └───┴──────┘ ``` +Rows are not escaped in Pretty* formats. Example is shown for the [PrettyCompact](#prettycompact) format: + +``` sql +SELECT 'String with \'quotes\' and \t character' AS Escaping_test +``` + +``` +┌─Escaping_test────────────────────────┐ +│ String with 'quotes' and character │ +└──────────────────────────────────────┘ +``` + To avoid dumping too much data to the terminal, only the first 10,000 rows are printed. If the number of rows is greater than or equal to 10,000, the message "Showed first 10 000" is printed. This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). -The Pretty format supports outputting total values (when using WITH TOTALS) and extremes (when 'extremes' is set to 1). In these cases, total values and extreme values are output after the main data, in separate tables. Example (shown for the PrettyCompact format): +The Pretty format supports outputting total values (when using WITH TOTALS) and extremes (when 'extremes' is set to 1). In these cases, total values and extreme values are output after the main data, in separate tables. Example (shown for the [PrettyCompact](#prettycompact) format): ``` sql SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORDER BY EventDate FORMAT PrettyCompact @@ -389,7 +402,7 @@ Extremes: ## PrettyCompact {#prettycompact} -Differs from `Pretty` in that the grid is drawn between rows and the result is more compact. +Differs from [Pretty](#pretty) in that the grid is drawn between rows and the result is more compact. This format is used by default in the command-line client in interactive mode. ## PrettyCompactMonoBlock {#prettycompactmonoblock} @@ -461,37 +474,20 @@ Row 1: x: 1 y: ᴺᵁᴸᴸ ``` +Rows are not escaped in Vertical format: -This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). - -## VerticalRaw {#verticalraw} - -Differs from `Vertical` format in that the rows are not escaped. -This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). - -Examples: +``` sql +SELECT 'string with \'quotes\' and \t with some special \n characters' AS test FORMAT Vertical +``` ``` -:) SHOW CREATE TABLE geonames FORMAT VerticalRaw; Row 1: ────── -statement: CREATE TABLE default.geonames ( geonameid UInt32, date Date DEFAULT CAST('2017-12-08' AS Date)) ENGINE = MergeTree(date, geonameid, 8192) - -:) SELECT 'string with \'quotes\' and \t with some special \n characters' AS test FORMAT VerticalRaw; -Row 1: -────── -test: string with 'quotes' and with some special +test: string with 'quotes' and with some special characters ``` -Compare with the Vertical format: - -``` -:) SELECT 'string with \'quotes\' and \t with some special \n characters' AS test FORMAT Vertical; -Row 1: -────── -test: string with \'quotes\' and \t with some special \n characters -``` +This format is only appropriate for outputting a query result, but not for parsing (retrieving data to insert in a table). ## XML {#xml} From e0267594438b627e9f112547e5fe2bd580da33fd Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 22 Jan 2019 17:37:28 +0300 Subject: [PATCH 62/71] Fix fds close logic --- libs/libdaemon/src/BaseDaemon.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index 838730185ad..5795f33ca01 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -788,10 +788,12 @@ void BaseDaemon::closeFDs() #endif if (proc_path.isDirectory()) /// Hooray, proc exists { - Poco::DirectoryIterator itr(proc_path), end; - for (; itr != end; ++itr) + std::vector fds; + /// in /proc/self/fd directory filenames are numeric file descriptors + proc_path.list(fds); + for (const auto & fd_str : fds) { - long fd = DB::parse(itr.name()); + long fd = DB::parse(fd_str); if (fd > 2 && fd != signal_pipe.read_fd && fd != signal_pipe.write_fd) ::close(fd); } From 3b0c67b6cd90f772942b2436458f47ca60bef0cd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 22 Jan 2019 19:21:55 +0300 Subject: [PATCH 63/71] Added test with all joins to bugs. --- dbms/tests/queries/bugs/all_join.sql | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) create mode 100644 dbms/tests/queries/bugs/all_join.sql diff --git a/dbms/tests/queries/bugs/all_join.sql b/dbms/tests/queries/bugs/all_join.sql new file mode 100644 index 00000000000..8b63adf36f0 --- /dev/null +++ b/dbms/tests/queries/bugs/all_join.sql @@ -0,0 +1,24 @@ +drop table if exists test.persons; +drop table if exists test.children; + +create table test.persons ( + id String, + name String +) engine MergeTree order by id; + +create table test.children ( + id String, + childName String +) engine MergeTree order by id; + +insert into test.persons (id, name) values ('1', 'John'), ('2', 'Jack'), ('3', 'Daniel'), ('4', 'James'), ('5', 'Amanda'); +insert into test.children (id, childName) values ('1', 'Robert'), ('1', 'Susan'), ('3', 'Sarah'), ('4', 'David'), ('4', 'Joseph'), ('5', 'Robert'); + + +select * from test.persons all inner join test.children using id; + +select * from test.persons all inner join (select * from test.children) as j using id; + +select * from (select * from test.persons) as s all inner join (select * from test.children) as j using id; + + From 6053d1f235db5714e0845d8f82d45de09c2b568d Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 22 Jan 2019 19:28:05 +0300 Subject: [PATCH 64/71] Fix type --- libs/libdaemon/src/BaseDaemon.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index 5795f33ca01..7e0c9583b7e 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -793,20 +793,20 @@ void BaseDaemon::closeFDs() proc_path.list(fds); for (const auto & fd_str : fds) { - long fd = DB::parse(fd_str); + int fd = DB::parse(fd_str); if (fd > 2 && fd != signal_pipe.read_fd && fd != signal_pipe.write_fd) ::close(fd); } } else { - long max_fd = -1; + int max_fd = -1; #ifdef _SC_OPEN_MAX max_fd = sysconf(_SC_OPEN_MAX); if (max_fd == -1) #endif max_fd = 256; /// bad fallback - for (long fd = 3; fd < max_fd; ++fd) + for (int fd = 3; fd < max_fd; ++fd) if (fd != signal_pipe.read_fd && fd != signal_pipe.write_fd) ::close(fd); } From 97199b9712c58d43fcfca3cb14edd3a07b88ff13 Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 22 Jan 2019 20:36:08 +0300 Subject: [PATCH 65/71] fix ArrayJoin columns calculation --- .../Interpreters/ArrayJoinedColumnsVisitor.h | 45 +++++++++++++++++-- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 34 +++----------- dbms/src/Interpreters/SyntaxAnalyzer.h | 1 - .../00819_ast_refactoring_bugs.sql | 23 ++++++++-- 4 files changed, 67 insertions(+), 36 deletions(-) diff --git a/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h b/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h index 15985eb7ba2..a5b85bcd318 100644 --- a/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h +++ b/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h @@ -15,12 +15,22 @@ namespace DB { +namespace ErrorCodes +{ + extern const int ALIAS_REQUIRED; + extern const int MULTIPLE_EXPRESSIONS_FOR_ALIAS; + extern const int LOGICAL_ERROR; +} + /// Fills the array_join_result_to_source: on which columns-arrays to replicate, and how to call them after that. class ArrayJoinedColumnsMatcher { public: struct Data { + using Aliases = std::unordered_map; + + const Aliases & aliases; NameToNameMap & array_join_name_to_alias; NameToNameMap & array_join_alias_to_name; NameToNameMap & array_join_result_to_source; @@ -30,10 +40,6 @@ public: static bool needChildVisit(ASTPtr & node, const ASTPtr & child) { - /// Processed - if (typeid_cast(node.get())) - return false; - if (typeid_cast(node.get())) return false; @@ -48,10 +54,41 @@ public: { if (auto * t = typeid_cast(ast.get())) visit(*t, ast, data); + if (auto * t = typeid_cast(ast.get())) + return visit(*t, ast, data); return {}; } private: + static std::vector visit(const ASTSelectQuery & node, ASTPtr &, Data & data) + { + ASTPtr array_join_expression_list = node.array_join_expression_list(); + if (!array_join_expression_list) + throw Exception("Logical error: no ARRAY JOIN", ErrorCodes::LOGICAL_ERROR); + + std::vector out; + out.reserve(array_join_expression_list->children.size()); + + for (ASTPtr & ast : array_join_expression_list->children) + { + const String nested_table_name = ast->getColumnName(); + const String nested_table_alias = ast->getAliasOrColumnName(); + + if (nested_table_alias == nested_table_name && !isIdentifier(ast)) + throw Exception("No alias for non-trivial value in ARRAY JOIN: " + nested_table_name, ErrorCodes::ALIAS_REQUIRED); + + if (data.array_join_alias_to_name.count(nested_table_alias) || data.aliases.count(nested_table_alias)) + throw Exception("Duplicate alias in ARRAY JOIN: " + nested_table_alias, ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS); + + data.array_join_alias_to_name[nested_table_alias] = nested_table_name; + data.array_join_name_to_alias[nested_table_name] = nested_table_alias; + + out.emplace_back(&ast); + } + + return out; + } + static void visit(const ASTIdentifier & node, ASTPtr &, Data & data) { NameToNameMap & array_join_name_to_alias = data.array_join_name_to_alias; diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 81bcca0bbc6..9a00396d41f 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -35,8 +35,6 @@ namespace DB namespace ErrorCodes { - extern const int ALIAS_REQUIRED; - extern const int MULTIPLE_EXPRESSIONS_FOR_ALIAS; extern const int EMPTY_NESTED_TABLE; extern const int LOGICAL_ERROR; extern const int INVALID_JOIN_ON_EXPRESSION; @@ -434,33 +432,13 @@ void optimizeUsing(const ASTSelectQuery * select_query) void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query, const Names & source_columns, const NameSet & source_columns_set) { - ASTPtr array_join_expression_list = select_query->array_join_expression_list(); - if (array_join_expression_list) + if (ASTPtr array_join_expression_list = select_query->array_join_expression_list()) { - ASTs & array_join_asts = array_join_expression_list->children; - for (const auto & ast : array_join_asts) - { - const String nested_table_name = ast->getColumnName(); - const String nested_table_alias = ast->getAliasOrColumnName(); - - if (nested_table_alias == nested_table_name && !isIdentifier(ast)) - throw Exception("No alias for non-trivial value in ARRAY JOIN: " + nested_table_name, - ErrorCodes::ALIAS_REQUIRED); - - if (result.array_join_alias_to_name.count(nested_table_alias) || result.aliases.count(nested_table_alias)) - throw Exception("Duplicate alias in ARRAY JOIN: " + nested_table_alias, - ErrorCodes::MULTIPLE_EXPRESSIONS_FOR_ALIAS); - - result.array_join_alias_to_name[nested_table_alias] = nested_table_name; - result.array_join_name_to_alias[nested_table_name] = nested_table_alias; - } - - { - ArrayJoinedColumnsVisitor::Data visitor_data{result.array_join_name_to_alias, - result.array_join_alias_to_name, - result.array_join_result_to_source}; - ArrayJoinedColumnsVisitor(visitor_data).visit(query); - } + ArrayJoinedColumnsVisitor::Data visitor_data{result.aliases, + result.array_join_name_to_alias, + result.array_join_alias_to_name, + result.array_join_result_to_source}; + ArrayJoinedColumnsVisitor(visitor_data).visit(query); /// If the result of ARRAY JOIN is not used, it is necessary to ARRAY-JOIN any column, /// to get the correct number of rows. diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.h b/dbms/src/Interpreters/SyntaxAnalyzer.h index 54ca4dfcf2b..d34b16c1a40 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.h +++ b/dbms/src/Interpreters/SyntaxAnalyzer.h @@ -16,7 +16,6 @@ struct SyntaxAnalyzerResult NamesAndTypesList source_columns; - /// Note: used only in tests. using Aliases = std::unordered_map; Aliases aliases; diff --git a/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.sql b/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.sql index 98ae13d2b9e..27171f2e0de 100644 --- a/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.sql +++ b/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.sql @@ -1,5 +1,22 @@ -CREATE TABLE IF NOT EXISTS test.sign (Sign Int8, Arr Array(Int8)) ENGINE = Memory; +DROP TABLE IF EXISTS test.visits; +CREATE TABLE test.visits +( + Sign Int8, + Arr Array(Int8), + `ParsedParams.Key1` Array(String), + `ParsedParams.Key2` Array(String), + CounterID UInt32 +) ENGINE = Memory; -SELECT arrayMap(x -> x * Sign, Arr) FROM test.sign; +SELECT arrayMap(x -> x * Sign, Arr) FROM test.visits; -DROP TABLE test.sign; +SELECT PP.Key2 AS `ym:s:pl2` +FROM test.visits +ARRAY JOIN + `ParsedParams.Key2` AS `PP.Key2`, + `ParsedParams.Key1` AS `PP.Key1`, + arrayEnumerateUniq(`ParsedParams.Key2`, arrayMap(x_0 -> 1, `ParsedParams.Key1`)) AS `upp_==_yes_`, + arrayEnumerateUniq(`ParsedParams.Key2`) AS _uniq_ParsedParams +WHERE CounterID = 100500; + +DROP TABLE test.visits; From f54ed4a2e3e2b18bd0fa9fdfe455a52c406e0cd7 Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 22 Jan 2019 21:02:53 +0300 Subject: [PATCH 66/71] do not revisit ArrayJoin identifiers --- dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h b/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h index a5b85bcd318..d345ee42dbe 100644 --- a/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h +++ b/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h @@ -83,7 +83,8 @@ private: data.array_join_alias_to_name[nested_table_alias] = nested_table_name; data.array_join_name_to_alias[nested_table_name] = nested_table_alias; - out.emplace_back(&ast); + for (ASTPtr & child2 : ast->children) + out.emplace_back(&child2); } return out; From 9481d01ca81eddfecec0d85acb94c5dc06f68b72 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 22 Jan 2019 22:18:30 +0300 Subject: [PATCH 67/71] Auto version update to [19.1.5] [54413] --- dbms/cmake/version.cmake | 8 ++++---- debian/changelog | 4 ++-- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index d1656c9f90b..c684c9a4f7c 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -2,10 +2,10 @@ set(VERSION_REVISION 54413) set(VERSION_MAJOR 19) set(VERSION_MINOR 1) -set(VERSION_PATCH 4) -set(VERSION_GITHASH be762f58e1d0286c54609301cabc1934f49257fc) -set(VERSION_DESCRIBE v19.1.4-testing) -set(VERSION_STRING 19.1.4) +set(VERSION_PATCH 5) +set(VERSION_GITHASH 2a7e7364c139b3c97f54f38ca6ea76ab4fa61e4b) +set(VERSION_DESCRIBE v19.1.5-testing) +set(VERSION_STRING 19.1.5) # end of autochange set(VERSION_EXTRA "" CACHE STRING "") diff --git a/debian/changelog b/debian/changelog index 36903d88d05..33331e2f495 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (19.1.4) unstable; urgency=low +clickhouse (19.1.5) unstable; urgency=low * Modified source code - -- Tue, 22 Jan 2019 02:29:09 +0300 + -- Tue, 22 Jan 2019 22:18:30 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index 7cf376c6ed1..688c52c372b 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.1.4 +ARG version=19.1.5 RUN apt-get update \ && apt-get install --yes --no-install-recommends \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index a45221e060e..ef272d1447e 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.1.4 +ARG version=19.1.5 ARG gosu_ver=1.10 RUN apt-get update \ diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index 2a21d932422..5965bc4e9a6 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.1.4 +ARG version=19.1.5 RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ From 6c83beafdd4323e890869f3ac7c59f41206d89fa Mon Sep 17 00:00:00 2001 From: maiha Date: Wed, 23 Jan 2019 04:56:53 +0900 Subject: [PATCH 68/71] dbms: Fixed misspells in comments --- dbms/programs/client/Client.cpp | 6 +++--- dbms/programs/local/LocalServer.cpp | 2 +- dbms/programs/obfuscator/Obfuscator.cpp | 4 ++-- .../odbc-bridge/validateODBCConnectionString.cpp | 2 +- dbms/programs/server/users.xml | 4 ++-- .../AggregateFunctionGroupUniqArray.h | 2 +- dbms/src/AggregateFunctions/AggregateFunctionTopK.h | 2 +- dbms/src/AggregateFunctions/QuantileExactWeighted.h | 2 +- dbms/src/Columns/IColumn.h | 8 ++++---- dbms/src/Common/ArrayCache.h | 2 +- dbms/src/Common/Config/ConfigProcessor.cpp | 2 +- dbms/src/Common/HashTable/Hash.h | 4 ++-- dbms/src/Common/OptimizedRegularExpression.cpp | 2 +- dbms/src/Common/SpaceSaving.h | 2 +- dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 2 +- .../ZooKeeper/tests/zk_many_watches_reconnect.cpp | 2 +- dbms/src/DataStreams/ColumnGathererStream.h | 2 +- dbms/src/DataStreams/FinishSortingBlockInputStream.cpp | 4 ++-- .../DataStreams/GraphiteRollupSortedBlockInputStream.h | 2 +- dbms/src/DataTypes/DataTypeString.cpp | 2 +- dbms/src/DataTypes/DataTypesNumber.cpp | 2 +- dbms/src/DataTypes/NumberTraits.h | 2 +- dbms/src/Functions/FunctionMathBinaryFloat64.h | 2 +- dbms/src/Functions/FunctionMathUnaryFloat64.h | 2 +- dbms/src/Functions/FunctionsRandom.h | 2 +- dbms/src/Functions/FunctionsStringSearch.h | 2 +- dbms/src/Functions/FunctionsURL.h | 2 +- dbms/src/Functions/IFunction.h | 2 +- dbms/src/Functions/formatDateTime.cpp | 2 +- dbms/src/Functions/minus.cpp | 2 +- dbms/src/Functions/multiply.cpp | 2 +- dbms/src/Functions/plus.cpp | 2 +- dbms/src/IO/ReadHelpers.cpp | 2 +- dbms/src/IO/ReadHelpers.h | 2 +- dbms/src/Interpreters/Context.cpp | 2 +- dbms/src/Interpreters/DDLWorker.cpp | 4 ++-- dbms/src/Interpreters/InterpreterRenameQuery.cpp | 2 +- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 2 +- dbms/src/Interpreters/InterpreterSystemQuery.cpp | 2 +- dbms/src/Interpreters/Settings.h | 2 +- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 2 +- .../Interpreters/TranslateQualifiedNamesVisitor.cpp | 2 +- dbms/src/Interpreters/executeQuery.cpp | 2 +- dbms/src/Parsers/ASTTablesInSelectQuery.h | 2 +- dbms/src/Parsers/Lexer.cpp | 4 ++-- dbms/src/Parsers/parseQuery.h | 2 +- dbms/src/Storages/Kafka/StorageKafka.cpp | 2 +- dbms/src/Storages/MergeTree/AllMergeSelector.h | 2 +- dbms/src/Storages/MergeTree/KeyCondition.h | 2 +- .../src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp | 4 ++-- .../Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- dbms/src/Storages/MergeTree/SimpleMergeSelector.h | 2 +- dbms/src/Storages/StorageFile.cpp | 2 +- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++----- .../queries/0_stateless/00113_shard_group_array.sql | 2 +- 55 files changed, 71 insertions(+), 71 deletions(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 0edffbcf4de..dd3fb2b84b6 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -860,7 +860,7 @@ private: } - /// Process the query that doesn't require transfering data blocks to the server. + /// Process the query that doesn't require transferring data blocks to the server. void processOrdinaryQuery() { connection->sendQuery(query, query_id, QueryProcessingStage::Complete, &context.getSettingsRef(), nullptr, true); @@ -869,7 +869,7 @@ private: } - /// Process the query that requires transfering data blocks to the server. + /// Process the query that requires transferring data blocks to the server. void processInsertQuery() { /// Send part of query without data, because data will be sent separately. @@ -1136,7 +1136,7 @@ private: } - /// Process Log packets, exit when recieve Exception or EndOfStream + /// Process Log packets, exit when receive Exception or EndOfStream bool receiveEndOfQuery() { while (true) diff --git a/dbms/programs/local/LocalServer.cpp b/dbms/programs/local/LocalServer.cpp index 1e3bc19f360..0acdae801ac 100644 --- a/dbms/programs/local/LocalServer.cpp +++ b/dbms/programs/local/LocalServer.cpp @@ -137,7 +137,7 @@ try static KillingErrorHandler error_handler; Poco::ErrorHandler::set(&error_handler); - /// Don't initilaize DateLUT + /// Don't initialize DateLUT registerFunctions(); registerAggregateFunctions(); diff --git a/dbms/programs/obfuscator/Obfuscator.cpp b/dbms/programs/obfuscator/Obfuscator.cpp index 6edb0de82b3..48b6d482703 100644 --- a/dbms/programs/obfuscator/Obfuscator.cpp +++ b/dbms/programs/obfuscator/Obfuscator.cpp @@ -51,7 +51,7 @@ It is designed to retain the following properties of data: - probability distributions of length of strings; - probability of zero values of numbers; empty strings and arrays, NULLs; - data compression ratio when compressed with LZ77 and entropy family of codecs; -- continuouty (magnitude of difference) of time values across table; continuouty of floating point values. +- continuity (magnitude of difference) of time values across table; continuity of floating point values. - date component of DateTime values; - UTF-8 validity of string values; - string values continue to look somewhat natural. @@ -246,7 +246,7 @@ Float transformFloatMantissa(Float x, UInt64 seed) /// Transform difference from previous number by applying pseudorandom permutation to mantissa part of it. -/// It allows to retain some continuouty property of source data. +/// It allows to retain some continuity property of source data. template class FloatModel : public IModel { diff --git a/dbms/programs/odbc-bridge/validateODBCConnectionString.cpp b/dbms/programs/odbc-bridge/validateODBCConnectionString.cpp index a64d7bb0b7c..e564cebdeee 100644 --- a/dbms/programs/odbc-bridge/validateODBCConnectionString.cpp +++ b/dbms/programs/odbc-bridge/validateODBCConnectionString.cpp @@ -22,7 +22,7 @@ std::string validateODBCConnectionString(const std::string & connection_string) /// Connection string is a list of name, value pairs. /// name and value are separated by '='. /// names are case insensitive. - /// name=value pairs are sepated by ';'. + /// name=value pairs are separated by ';'. /// ASCII whitespace characters are skipped before and after delimiters. /// value may be optionally enclosed by {} /// in enclosed value, } is escaped as }}. diff --git a/dbms/programs/server/users.xml b/dbms/programs/server/users.xml index 979b2d3fc17..3b08368eae5 100644 --- a/dbms/programs/server/users.xml +++ b/dbms/programs/server/users.xml @@ -13,9 +13,9 @@ random diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h index 9dc3882cca7..b638996f553 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h @@ -109,7 +109,7 @@ struct AggreagteFunctionGroupUniqArrayGenericData }; /** Template parameter with true value should be used for columns that store their elements in memory continuously. - * For such columns groupUniqArray() can be implemented more efficently (especially for small numeric arrays). + * For such columns groupUniqArray() can be implemented more efficiently (especially for small numeric arrays). */ template class AggreagteFunctionGroupUniqArrayGeneric diff --git a/dbms/src/AggregateFunctions/AggregateFunctionTopK.h b/dbms/src/AggregateFunctions/AggregateFunctionTopK.h index 4d1238a12cc..68de4dd349e 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionTopK.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionTopK.h @@ -123,7 +123,7 @@ struct AggregateFunctionTopKGenericData }; /** Template parameter with true value should be used for columns that store their elements in memory continuously. - * For such columns topK() can be implemented more efficently (especially for small numeric arrays). + * For such columns topK() can be implemented more efficiently (especially for small numeric arrays). */ template class AggregateFunctionTopKGeneric : public IAggregateFunctionDataHelper> diff --git a/dbms/src/AggregateFunctions/QuantileExactWeighted.h b/dbms/src/AggregateFunctions/QuantileExactWeighted.h index eca89f30aa9..1614633740c 100644 --- a/dbms/src/AggregateFunctions/QuantileExactWeighted.h +++ b/dbms/src/AggregateFunctions/QuantileExactWeighted.h @@ -12,7 +12,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -/** Calculates quantile by counting number of occurences for each value in a hash map. +/** Calculates quantile by counting number of occurrences for each value in a hash map. * * It use O(distinct(N)) memory. Can be naturally applied for values with weight. * In case of many identical values, it can be more efficient than QuantileExact even when weight is not used. diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index 954715e4ff8..2560b9639ad 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -43,7 +43,7 @@ public: virtual const char * getFamilyName() const = 0; /** If column isn't constant, returns nullptr (or itself). - * If column is constant, transforms constant to full column (if column type allows such tranform) and return it. + * If column is constant, transforms constant to full column (if column type allows such transform) and return it. */ virtual Ptr convertToFullColumnIfConst() const { return getPtr(); } @@ -149,7 +149,7 @@ public: virtual void insertDefault() = 0; /** Removes last n elements. - * Is used to support exeption-safety of several operations. + * Is used to support exception-safety of several operations. * For example, sometimes insertion should be reverted if we catch an exception during operation processing. * If column has less than n elements or n == 0 - undefined behavior. */ @@ -234,8 +234,8 @@ public: virtual void gather(ColumnGathererStream & gatherer_stream) = 0; /** Computes minimum and maximum element of the column. - * In addition to numeric types, the funtion is completely implemented for Date and DateTime. - * For strings and arrays function should retrurn default value. + * In addition to numeric types, the function is completely implemented for Date and DateTime. + * For strings and arrays function should return default value. * (except for constant columns; they should return value of the constant). * If column is empty function should return default value. */ diff --git a/dbms/src/Common/ArrayCache.h b/dbms/src/Common/ArrayCache.h index 1521b753996..304ab8e03c3 100644 --- a/dbms/src/Common/ArrayCache.h +++ b/dbms/src/Common/ArrayCache.h @@ -64,7 +64,7 @@ namespace DB * During insertion, each key is locked - to avoid parallel initialization of regions for same key. * * On insertion, we search for free region of at least requested size. - * If nothing was found, we evict oldest unused region; if not enogh size, we evict it neighbours; and try again. + * If nothing was found, we evict oldest unused region; if not enough size, we evict it neighbours; and try again. * * Metadata is allocated by usual allocator and its memory usage is not accounted. * diff --git a/dbms/src/Common/Config/ConfigProcessor.cpp b/dbms/src/Common/Config/ConfigProcessor.cpp index 33b03dff38f..73028e79f23 100644 --- a/dbms/src/Common/Config/ConfigProcessor.cpp +++ b/dbms/src/Common/Config/ConfigProcessor.cpp @@ -23,7 +23,7 @@ using namespace Poco::XML; namespace DB { -/// For cutting prerpocessed path to this base +/// For cutting preprocessed path to this base static std::string main_config_path; /// Extracts from a string the first encountered number consisting of at least two digits. diff --git a/dbms/src/Common/HashTable/Hash.h b/dbms/src/Common/HashTable/Hash.h index 57fa060cb32..6e786e1ddbf 100644 --- a/dbms/src/Common/HashTable/Hash.h +++ b/dbms/src/Common/HashTable/Hash.h @@ -12,7 +12,7 @@ * - in typical implementation of standard library, hash function for integers is trivial and just use lower bits; * - traffic is non-uniformly distributed across a day; * - we are using open-addressing linear probing hash tables that are most critical to hash function quality, - * and trivial hash function gives disasterous results. + * and trivial hash function gives disastrous results. */ /** Taken from MurmurHash. This is Murmur finalizer. @@ -160,7 +160,7 @@ struct TrivialHash * NOTE Salting is far from perfect, because it commutes with first steps of calculation. * * NOTE As mentioned, this function is slower than intHash64. - * But occasionaly, it is faster, when written in a loop and loop is vectorized. + * But occasionally, it is faster, when written in a loop and loop is vectorized. */ template inline DB::UInt32 intHash32(DB::UInt64 key) diff --git a/dbms/src/Common/OptimizedRegularExpression.cpp b/dbms/src/Common/OptimizedRegularExpression.cpp index 16251ae1ee0..0b5f5c42b14 100644 --- a/dbms/src/Common/OptimizedRegularExpression.cpp +++ b/dbms/src/Common/OptimizedRegularExpression.cpp @@ -165,7 +165,7 @@ void OptimizedRegularExpressionImpl::analyze( ++pos; break; - /// Quantifiers that allow a zero number of occurences. + /// Quantifiers that allow a zero number of occurrences. case '{': in_curly_braces = true; [[fallthrough]]; diff --git a/dbms/src/Common/SpaceSaving.h b/dbms/src/Common/SpaceSaving.h index 27477e4f9a2..feba4414a54 100644 --- a/dbms/src/Common/SpaceSaving.h +++ b/dbms/src/Common/SpaceSaving.h @@ -40,7 +40,7 @@ struct SpaceSavingArena /* * Specialized storage for StringRef with a freelist arena. - * Keys of this type that are retained on insertion must be serialised into local storage, + * Keys of this type that are retained on insertion must be serialized into local storage, * otherwise the reference would be invalid after the processed block is released. */ template <> diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h index c93f13b9351..010abd22ef7 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -41,7 +41,7 @@ * - extremely creepy code for implementation of "chroot" feature. * * As of 2018, there are no active maintainers of libzookeeper: - * - bugs in JIRA are fixed only occasionaly with ad-hoc patches by library users. + * - bugs in JIRA are fixed only occasionally with ad-hoc patches by library users. * * libzookeeper is a classical example of bad code written in C. * diff --git a/dbms/src/Common/ZooKeeper/tests/zk_many_watches_reconnect.cpp b/dbms/src/Common/ZooKeeper/tests/zk_many_watches_reconnect.cpp index cae37319c74..fa4fea55580 100644 --- a/dbms/src/Common/ZooKeeper/tests/zk_many_watches_reconnect.cpp +++ b/dbms/src/Common/ZooKeeper/tests/zk_many_watches_reconnect.cpp @@ -5,7 +5,7 @@ /// A tool for reproducing https://issues.apache.org/jira/browse/ZOOKEEPER-706 /// Original libzookeeper can't reconnect the session if the length of SET_WATCHES message -/// exceeeds jute.maxbuffer (0xfffff by default). +/// exceeds jute.maxbuffer (0xfffff by default). /// This happens when the number of watches exceeds ~29000. /// /// Session reconnect can be caused by forbidding packets to the current zookeeper server, e.g. diff --git a/dbms/src/DataStreams/ColumnGathererStream.h b/dbms/src/DataStreams/ColumnGathererStream.h index f52e75d2b30..10b71ca3321 100644 --- a/dbms/src/DataStreams/ColumnGathererStream.h +++ b/dbms/src/DataStreams/ColumnGathererStream.h @@ -51,7 +51,7 @@ using MergedRowSources = PODArray; /** Gather single stream from multiple streams according to streams mask. * Stream mask maps row number to index of source stream. - * Streams should conatin exactly one column. + * Streams should contain exactly one column. */ class ColumnGathererStream : public IProfilingBlockInputStream { diff --git a/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp b/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp index 015ea8bf903..ed3552f7f00 100644 --- a/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp +++ b/dbms/src/DataStreams/FinishSortingBlockInputStream.cpp @@ -85,7 +85,7 @@ Block FinishSortingBlockInputStream::readImpl() { Block block = children.back()->read(); - /// End of input stream, but we can`t return immediatly, we need to merge already read blocks. + /// End of input stream, but we can`t return immediately, we need to merge already read blocks. /// Check it later, when get end of stream from impl. if (!block) { @@ -102,7 +102,7 @@ Block FinishSortingBlockInputStream::readImpl() if (size == 0) continue; - /// We need to sort each block separatly before merging. + /// We need to sort each block separately before merging. sortBlock(block, description_to_sort); removeConstantsFromBlock(block); diff --git a/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.h b/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.h index aab8735c602..e18522d6d25 100644 --- a/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.h +++ b/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.h @@ -17,7 +17,7 @@ namespace DB /** Intended for implementation of "rollup" - aggregation (rounding) of older data * for a table with Graphite data (Graphite is the system for time series monitoring). * - * Table with graphite data has at least the folowing columns (accurate to the name): + * Table with graphite data has at least the following columns (accurate to the name): * Path, Time, Value, Version * * Path - name of metric (sensor); diff --git a/dbms/src/DataTypes/DataTypeString.cpp b/dbms/src/DataTypes/DataTypeString.cpp index 55a4ac920b6..0563a2e01d1 100644 --- a/dbms/src/DataTypes/DataTypeString.cpp +++ b/dbms/src/DataTypes/DataTypeString.cpp @@ -322,7 +322,7 @@ void registerDataTypeString(DataTypeFactory & factory) factory.registerSimpleDataType("String", creator); - /// These synonims are added for compatibility. + /// These synonyms are added for compatibility. factory.registerAlias("CHAR", "String", DataTypeFactory::CaseInsensitive); factory.registerAlias("VARCHAR", "String", DataTypeFactory::CaseInsensitive); diff --git a/dbms/src/DataTypes/DataTypesNumber.cpp b/dbms/src/DataTypes/DataTypesNumber.cpp index 254d6ba6852..5739a64d815 100644 --- a/dbms/src/DataTypes/DataTypesNumber.cpp +++ b/dbms/src/DataTypes/DataTypesNumber.cpp @@ -20,7 +20,7 @@ void registerDataTypeNumbers(DataTypeFactory & factory) factory.registerSimpleDataType("Float32", [] { return DataTypePtr(std::make_shared()); }); factory.registerSimpleDataType("Float64", [] { return DataTypePtr(std::make_shared()); }); - /// These synonims are added for compatibility. + /// These synonyms are added for compatibility. factory.registerAlias("TINYINT", "Int8", DataTypeFactory::CaseInsensitive); factory.registerAlias("SMALLINT", "Int16", DataTypeFactory::CaseInsensitive); diff --git a/dbms/src/DataTypes/NumberTraits.h b/dbms/src/DataTypes/NumberTraits.h index c3642f20c3b..00a0d297d72 100644 --- a/dbms/src/DataTypes/NumberTraits.h +++ b/dbms/src/DataTypes/NumberTraits.h @@ -188,7 +188,7 @@ template struct ToInteger // CLICKHOUSE-29. The same depth, different signs -// NOTE: This case is applied for 64-bit integers only (for backward compability), but could be used for any-bit integers +// NOTE: This case is applied for 64-bit integers only (for backward compatibility), but could be used for any-bit integers template constexpr bool LeastGreatestSpecialCase = std::is_integral_v && std::is_integral_v diff --git a/dbms/src/Functions/FunctionMathBinaryFloat64.h b/dbms/src/Functions/FunctionMathBinaryFloat64.h index 9dd462725fc..6431e204830 100644 --- a/dbms/src/Functions/FunctionMathBinaryFloat64.h +++ b/dbms/src/Functions/FunctionMathBinaryFloat64.h @@ -11,7 +11,7 @@ #include /** More efficient implementations of mathematical functions are possible when using a separate library. - * Disabled due to licence compatibility limitations. + * Disabled due to license compatibility limitations. * To enable: download http://www.agner.org/optimize/vectorclass.zip and unpack to contrib/vectorclass * Then rebuild with -DENABLE_VECTORCLASS=1 */ diff --git a/dbms/src/Functions/FunctionMathUnaryFloat64.h b/dbms/src/Functions/FunctionMathUnaryFloat64.h index f336847bee5..aad080531a2 100644 --- a/dbms/src/Functions/FunctionMathUnaryFloat64.h +++ b/dbms/src/Functions/FunctionMathUnaryFloat64.h @@ -10,7 +10,7 @@ #include /** More efficient implementations of mathematical functions are possible when using a separate library. - * Disabled due to licence compatibility limitations. + * Disabled due to license compatibility limitations. * To enable: download http://www.agner.org/optimize/vectorclass.zip and unpack to contrib/vectorclass * Then rebuild with -DENABLE_VECTORCLASS=1 */ diff --git a/dbms/src/Functions/FunctionsRandom.h b/dbms/src/Functions/FunctionsRandom.h index 47cb275ee89..069c0afa86b 100644 --- a/dbms/src/Functions/FunctionsRandom.h +++ b/dbms/src/Functions/FunctionsRandom.h @@ -24,7 +24,7 @@ namespace ErrorCodes * * Non-cryptographic generators: * - * rand - linear congruental generator 0 .. 2^32 - 1. + * rand - linear congruential generator 0 .. 2^32 - 1. * rand64 - combines several rand values to get values from the range 0 .. 2^64 - 1. * * randConstant - service function, produces a constant column with a random value. diff --git a/dbms/src/Functions/FunctionsStringSearch.h b/dbms/src/Functions/FunctionsStringSearch.h index 2949ba52626..19edeeae61e 100644 --- a/dbms/src/Functions/FunctionsStringSearch.h +++ b/dbms/src/Functions/FunctionsStringSearch.h @@ -39,7 +39,7 @@ namespace DB * replaceRegexpOne(haystack, pattern, replacement) - replaces the pattern with the specified regexp, only the first occurrence. * replaceRegexpAll(haystack, pattern, replacement) - replaces the pattern with the specified type, all occurrences. * - * multiPosition(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find first occurences (positions) of all the const patterns inside haystack + * multiPosition(haystack, [pattern_1, pattern_2, ..., pattern_n]) -- find first occurrences (positions) of all the const patterns inside haystack * multiPositionUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) * multiPositionCaseInsensitive(haystack, [pattern_1, pattern_2, ..., pattern_n]) * multiPositionCaseInsensitiveUTF8(haystack, [pattern_1, pattern_2, ..., pattern_n]) diff --git a/dbms/src/Functions/FunctionsURL.h b/dbms/src/Functions/FunctionsURL.h index 69f0b363112..2d77026f4c4 100644 --- a/dbms/src/Functions/FunctionsURL.h +++ b/dbms/src/Functions/FunctionsURL.h @@ -23,7 +23,7 @@ namespace DB * queryStringAndFragment * * Functions, removing parts from URL. - * If URL has nothing like, then it is retured unchanged. + * If URL has nothing like, then it is returned unchanged. * * cutWWW * cutFragment diff --git a/dbms/src/Functions/IFunction.h b/dbms/src/Functions/IFunction.h index 547229ecae1..0f945365efd 100644 --- a/dbms/src/Functions/IFunction.h +++ b/dbms/src/Functions/IFunction.h @@ -163,7 +163,7 @@ public: * Function could be injective with some arguments fixed to some constant values. * Examples: * plus(const, x); - * multiply(const, x) where x is an integer and constant is not divisable by two; + * multiply(const, x) where x is an integer and constant is not divisible by two; * concat(x, 'const'); * concat(x, 'const', y) where const contain at least one non-numeric character; * concat with FixedString diff --git a/dbms/src/Functions/formatDateTime.cpp b/dbms/src/Functions/formatDateTime.cpp index b9f75d1e6bd..ffbf391db6f 100644 --- a/dbms/src/Functions/formatDateTime.cpp +++ b/dbms/src/Functions/formatDateTime.cpp @@ -36,7 +36,7 @@ namespace ErrorCodes * * It is implemented in two steps. * At first step, it creates a pattern of zeros, literal characters, whitespaces, etc. - * and quickly fills resulting charater array (string column) with this pattern. + * and quickly fills resulting character array (string column) with this pattern. * At second step, it walks across the resulting character array and modifies/replaces specific charaters, * by calling some functions by pointers and shifting cursor by specified amount. * diff --git a/dbms/src/Functions/minus.cpp b/dbms/src/Functions/minus.cpp index dad7c14e4ae..df421aeb4c7 100644 --- a/dbms/src/Functions/minus.cpp +++ b/dbms/src/Functions/minus.cpp @@ -17,7 +17,7 @@ struct MinusImpl return static_cast(a) - b; } - /// Apply operation and check overflow. It's used for Deciamal operations. @returns true if overflowed, false othervise. + /// Apply operation and check overflow. It's used for Deciamal operations. @returns true if overflowed, false otherwise. template static inline bool apply(A a, B b, Result & c) { diff --git a/dbms/src/Functions/multiply.cpp b/dbms/src/Functions/multiply.cpp index 1a39445f09f..96f319f97e6 100644 --- a/dbms/src/Functions/multiply.cpp +++ b/dbms/src/Functions/multiply.cpp @@ -17,7 +17,7 @@ struct MultiplyImpl return static_cast(a) * b; } - /// Apply operation and check overflow. It's used for Deciamal operations. @returns true if overflowed, false othervise. + /// Apply operation and check overflow. It's used for Deciamal operations. @returns true if overflowed, false otherwise. template static inline bool apply(A a, B b, Result & c) { diff --git a/dbms/src/Functions/plus.cpp b/dbms/src/Functions/plus.cpp index de3f6d0c789..c260f557b9c 100644 --- a/dbms/src/Functions/plus.cpp +++ b/dbms/src/Functions/plus.cpp @@ -18,7 +18,7 @@ struct PlusImpl return static_cast(a) + b; } - /// Apply operation and check overflow. It's used for Deciamal operations. @returns true if overflowed, false othervise. + /// Apply operation and check overflow. It's used for Deciamal operations. @returns true if overflowed, false otherwise. template static inline bool apply(A a, B b, Result & c) { diff --git a/dbms/src/IO/ReadHelpers.cpp b/dbms/src/IO/ReadHelpers.cpp index ee6f163bbe6..46b8119052f 100644 --- a/dbms/src/IO/ReadHelpers.cpp +++ b/dbms/src/IO/ReadHelpers.cpp @@ -991,7 +991,7 @@ void skipToUnescapedNextLineOrEOF(ReadBuffer & buf) if (buf.eof()) return; - /// Skip escaped character. We do not consider escape sequences with more than one charater after backslash (\x01). + /// Skip escaped character. We do not consider escape sequences with more than one character after backslash (\x01). /// It's ok for the purpose of this function, because we are interested only in \n and \\. ++buf.position(); continue; diff --git a/dbms/src/IO/ReadHelpers.h b/dbms/src/IO/ReadHelpers.h index 2a91cf69309..f9f5a753750 100644 --- a/dbms/src/IO/ReadHelpers.h +++ b/dbms/src/IO/ReadHelpers.h @@ -581,7 +581,7 @@ inline ReturnType readDateTimeTextImpl(time_t & datetime, ReadBuffer & buf, cons { /** Read 10 characters, that could represent unix timestamp. * Only unix timestamp of 5-10 characters is supported. - * Then look at 5th charater. If it is a number - treat whole as unix timestamp. + * Then look at 5th character. If it is a number - treat whole as unix timestamp. * If it is not a number - then parse datetime in YYYY-MM-DD hh:mm:ss format. */ diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 4ded6b4d216..f5c99c140bc 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -124,7 +124,7 @@ struct ContextShared ConfigurationPtr config; /// Global configuration settings. Databases databases; /// List of databases and tables in them. - mutable std::shared_ptr embedded_dictionaries; /// Metrica's dictionaeis. Have lazy initialization. + mutable std::shared_ptr embedded_dictionaries; /// Metrica's dictionaries. Have lazy initialization. mutable std::shared_ptr external_dictionaries; mutable std::shared_ptr external_models; String default_profile_name; /// Default profile name used for default values. diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 7671fcb1b9b..c7e1fa3c178 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -276,7 +276,7 @@ bool DDLWorker::initAndCheckTask(const String & entry_name, String & out_reason) catch (...) { /// What should we do if we even cannot parse host name and therefore cannot properly submit execution status? - /// We can try to create fail node using FQDN if it equal to host name in cluster config attempt will be sucessfull. + /// We can try to create fail node using FQDN if it equal to host name in cluster config attempt will be successful. /// Otherwise, that node will be ignored by DDLQueryStatusInputSream. tryLogCurrentException(log, "Cannot parse DDL task " + entry_name + ", will try to send error status"); @@ -1153,7 +1153,7 @@ private: Strings current_active_hosts; /// Hosts that were in active state at the last check size_t num_hosts_finished = 0; - /// Save the first detected error and throw it at the end of excecution + /// Save the first detected error and throw it at the end of execution std::unique_ptr first_exception; Int64 timeout_seconds = 120; diff --git a/dbms/src/Interpreters/InterpreterRenameQuery.cpp b/dbms/src/Interpreters/InterpreterRenameQuery.cpp index 74dfe7adecf..58c830f3627 100644 --- a/dbms/src/Interpreters/InterpreterRenameQuery.cpp +++ b/dbms/src/Interpreters/InterpreterRenameQuery.cpp @@ -77,7 +77,7 @@ BlockIO InterpreterRenameQuery::execute() std::set unique_tables_from; - /// Don't allow to drop tables (that we are renaming); do't allow to create tables in places where tables will be renamed. + /// Don't allow to drop tables (that we are renaming); don't allow to create tables in places where tables will be renamed. std::map> table_guards; for (const auto & elem : rename.elements) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 754922d5188..c490f280cc3 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -586,7 +586,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt executePreLimit(pipeline); } - // If there is no global subqueries, we can run subqueries only when recieve them on server. + // If there is no global subqueries, we can run subqueries only when receive them on server. if (!query_analyzer->hasGlobalSubqueries() && !expressions.subqueries_for_sets.empty()) executeSubqueriesInSetsAndJoins(pipeline, expressions.subqueries_for_sets); } diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index fc472ad8a9e..98e84a092ec 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -53,7 +53,7 @@ ExecutionStatus getOverallExecutionStatusOfCommands() return ExecutionStatus(0); } -/// Consequently tries to execute all commands and genreates final exception message for failed commands +/// Consequently tries to execute all commands and generates final exception message for failed commands template ExecutionStatus getOverallExecutionStatusOfCommands(Callable && command, Callables && ... commands) { diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 646064edc86..65fd3f7e5e8 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -278,7 +278,7 @@ struct Settings M(SettingBool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.") \ M(SettingBool, log_query_settings, true, "Log query settings into the query_log.") \ M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.") \ - M(SettingLogsLevel, send_logs_level, "none", "Send server text logs with specified minumum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'none'") \ + M(SettingLogsLevel, send_logs_level, "none", "Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'none'") \ M(SettingBool, enable_optimize_predicate_expression, 0, "If it is set to true, optimize predicates to subqueries.") \ \ M(SettingUInt64, low_cardinality_max_dictionary_size, 8192, "Maximum size (in rows) of shared global dictionary for LowCardinality type.") \ diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 9a00396d41f..e09f1b3a95b 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -766,7 +766,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( /// Executing scalar subqueries - replacing them with constant values. executeScalarSubqueries(query, context, subquery_depth); - /// Optimize if with constant condition after constants was substituted instead of sclalar subqueries. + /// Optimize if with constant condition after constants was substituted instead of scalar subqueries. OptimizeIfWithConstantConditionVisitor(result.aliases).visit(query); if (select_query) diff --git a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index e8eac45a1ae..f884028dd1c 100644 --- a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -115,7 +115,7 @@ std::vector TranslateQualifiedNamesMatcher::visit(ASTTableJoin & join, std::vector TranslateQualifiedNamesMatcher::visit(ASTSelectQuery & select, const ASTPtr & , Data &) { - /// If the WHERE clause or HAVING consists of a single quailified column, the reference must be translated not only in children, + /// If the WHERE clause or HAVING consists of a single qualified column, the reference must be translated not only in children, /// but also in where_expression and having_expression. std::vector out; if (select.prewhere_expression) diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 3911e437fa6..0a2a3960ab7 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -330,7 +330,7 @@ static std::tuple executeQueryImpl( { if (auto counting_stream = dynamic_cast(stream_out)) { - /// NOTE: Redundancy. The same values coulld be extracted from process_list_elem->progress_out.query_settings = process_list_elem->progress_in + /// NOTE: Redundancy. The same values could be extracted from process_list_elem->progress_out.query_settings = process_list_elem->progress_in elem.result_rows = counting_stream->getProgress().rows; elem.result_bytes = counting_stream->getProgress().bytes; } diff --git a/dbms/src/Parsers/ASTTablesInSelectQuery.h b/dbms/src/Parsers/ASTTablesInSelectQuery.h index c94192b44d2..4d4d0471ca1 100644 --- a/dbms/src/Parsers/ASTTablesInSelectQuery.h +++ b/dbms/src/Parsers/ASTTablesInSelectQuery.h @@ -15,7 +15,7 @@ namespace DB * or * (subquery) * - * Optionally with alias (correllation name): + * Optionally with alias (correlation name): * [AS] alias * * Table may contain FINAL and SAMPLE modifiers: diff --git a/dbms/src/Parsers/Lexer.cpp b/dbms/src/Parsers/Lexer.cpp index 6b8afaa76b7..0494eacd490 100644 --- a/dbms/src/Parsers/Lexer.cpp +++ b/dbms/src/Parsers/Lexer.cpp @@ -132,7 +132,7 @@ Token Lexer::nextTokenImpl() ++pos; } - /// exponentation (base 10 or base 2) + /// exponentiation (base 10 or base 2) if (pos + 1 < end && (hex ? (*pos == 'p' || *pos == 'P') : (*pos == 'e' || *pos == 'E'))) { ++pos; @@ -195,7 +195,7 @@ Token Lexer::nextTokenImpl() while (pos < end && isNumericASCII(*pos)) ++pos; - /// exponentation + /// exponentiation if (pos + 1 < end && (*pos == 'e' || *pos == 'E')) { ++pos; diff --git a/dbms/src/Parsers/parseQuery.h b/dbms/src/Parsers/parseQuery.h index b526280975c..85851eedacf 100644 --- a/dbms/src/Parsers/parseQuery.h +++ b/dbms/src/Parsers/parseQuery.h @@ -49,7 +49,7 @@ ASTPtr parseQuery( /** Split queries separated by ; on to list of single queries - * Returns pointer to the end of last sucessfuly parsed query (first), and true if all queries are sucessfuly parsed (second) + * Returns pointer to the end of last successfully parsed query (first), and true if all queries are successfully parsed (second) * NOTE: INSERT's data should be placed in single line. */ std::pair splitMultipartQuery(const std::string & queries, std::vector & queries_list); diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 77e3ca7d795..94ed3497f25 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -155,7 +155,7 @@ public: return; // An error was thrown during the stream or it did not finish successfully - // The read offsets weren't comitted, so consumer must rejoin the group from the original starting point + // The read offsets weren't committed, so consumer must rejoin the group from the original starting point if (!finalized) { LOG_TRACE(storage.log, "KafkaBlockInputStream did not finish successfully, unsubscribing from assignments and rejoining"); diff --git a/dbms/src/Storages/MergeTree/AllMergeSelector.h b/dbms/src/Storages/MergeTree/AllMergeSelector.h index 50df9b601d4..eade7954144 100644 --- a/dbms/src/Storages/MergeTree/AllMergeSelector.h +++ b/dbms/src/Storages/MergeTree/AllMergeSelector.h @@ -6,7 +6,7 @@ namespace DB { -/// Select all parts within partition (having at least two parts) with minumum total size. +/// Select all parts within partition (having at least two parts) with minimum total size. class AllMergeSelector : public IMergeSelector { public: diff --git a/dbms/src/Storages/MergeTree/KeyCondition.h b/dbms/src/Storages/MergeTree/KeyCondition.h index 1d700ad80d9..39ee16a71c6 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.h +++ b/dbms/src/Storages/MergeTree/KeyCondition.h @@ -339,7 +339,7 @@ private: /** Is node the key column * or expression in which column of key is wrapped by chain of functions, - * that can be monotomic on certain ranges? + * that can be monotonic on certain ranges? * If these conditions are true, then returns number of column in key, type of resulting expression * and fills chain of possibly-monotonic functions. */ diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 0408c5b7765..8712afe3a39 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -84,7 +84,7 @@ MergeTreeBlockSizePredictor::MergeTreeBlockSizePredictor( : data_part(data_part_) { number_of_rows_in_part = data_part->rows_count; - /// Initialize with sample block untill update won't called. + /// Initialize with sample block until update won't called. initialize(sample_block, columns); } @@ -173,7 +173,7 @@ void MergeTreeBlockSizePredictor::update(const Block & block, double decay) block_size_rows = new_rows; /// Make recursive updates for each read row: v_{i+1} = (1 - decay) v_{i} + decay v_{target} - /// Use sum of gemetric sequence formula to update multiple rows: v{n} = (1 - decay)^n v_{0} + (1 - (1 - decay)^n) v_{target} + /// Use sum of geometric sequence formula to update multiple rows: v{n} = (1 - decay)^n v_{0} + (1 - (1 - decay)^n) v_{target} /// NOTE: DEFAULT and MATERIALIZED columns without data has inaccurate estimation of v_{target} double alpha = std::pow(1. - decay, diff_rows); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 895a068621f..e8932397e51 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -714,7 +714,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (disk_reservation && sum_input_rows_upper_bound) { /// The same progress from merge_entry could be used for both algorithms (it should be more accurate) - /// But now we are using inaccurate row-based estimation in Horizontal case for backward compability + /// But now we are using inaccurate row-based estimation in Horizontal case for backward compatibility Float64 progress = (merge_alg == MergeAlgorithm::Horizontal) ? std::min(1., 1. * rows_written / sum_input_rows_upper_bound) : std::min(1., merge_entry->progress.load(std::memory_order_relaxed)); diff --git a/dbms/src/Storages/MergeTree/SimpleMergeSelector.h b/dbms/src/Storages/MergeTree/SimpleMergeSelector.h index 2f5219baf3d..e699f605b8f 100644 --- a/dbms/src/Storages/MergeTree/SimpleMergeSelector.h +++ b/dbms/src/Storages/MergeTree/SimpleMergeSelector.h @@ -16,7 +16,7 @@ public: /** Minimum ratio of size of one part to all parts in set of parts to merge (for usual cases). * For example, if all parts have equal size, it means, that at least 'base' number of parts should be merged. - * If parts has non-uniform sizes, then minumum number of parts to merge is effectively increased. + * If parts has non-uniform sizes, then minimum number of parts to merge is effectively increased. * This behaviour balances merge-tree workload. * It called 'base', because merge-tree depth could be estimated as logarithm with that base. * diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index f24badfa82f..341160f69f1 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -131,7 +131,7 @@ public: if (storage.table_fd_init_offset < 0) throw Exception("File descriptor isn't seekable, inside " + storage.getName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE); - /// ReadBuffer's seek() doesn't make sence, since cache is empty + /// ReadBuffer's seek() doesn't make sense, since cache is empty if (lseek(storage.table_fd, storage.table_fd_init_offset, SEEK_SET) < 0) throwFromErrno("Cannot seek file descriptor, inside " + storage.getName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 1c904e3342c..bd692738e12 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -1592,7 +1592,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) MergeTreePartInfo new_part_info; String checksum_hex; - /// Part which will be comitted + /// Part which will be committed MergeTreeData::MutableDataPartPtr res_part; /// We could find a covering part @@ -1624,7 +1624,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) data.format_version)); } - /// What parts we should add? Or we have already added all required parts (we an replica-intializer) + /// What parts we should add? Or we have already added all required parts (we an replica-initializer) { auto data_parts_lock = data.lockParts(); @@ -3427,7 +3427,7 @@ bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const St --right; - /// Artificial high level is choosen, to make this part "covering" all parts inside. + /// Artificial high level is chosen, to make this part "covering" all parts inside. part_info = MergeTreePartInfo(partition_id, left, right, MergeTreePartInfo::MAX_LEVEL, mutation_version); return true; } @@ -3692,7 +3692,7 @@ std::optional StorageReplicatedMergeTree::allocateBlockNumber( const String & partition_id, zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path) { - /// Lets check for duplicates in advance, to avoid superflous block numbers allocation + /// Lets check for duplicates in advance, to avoid superfluous block numbers allocation Coordination::Requests deduplication_check_ops; if (!zookeeper_block_id_path.empty()) { @@ -4742,7 +4742,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ for (size_t i = 0; i < src_all_parts.size(); ++i) { /// We also make some kind of deduplication to avoid duplicated parts in case of ATTACH PARTITION - /// Assume that merges in the partiton are quite rare + /// Assume that merges in the partition are quite rare /// Save deduplication block ids with special prefix replace_partition auto & src_part = src_all_parts[i]; diff --git a/dbms/tests/queries/0_stateless/00113_shard_group_array.sql b/dbms/tests/queries/0_stateless/00113_shard_group_array.sql index b425891b735..3d86f59579e 100644 --- a/dbms/tests/queries/0_stateless/00113_shard_group_array.sql +++ b/dbms/tests/queries/0_stateless/00113_shard_group_array.sql @@ -33,5 +33,5 @@ SELECT roundToExp2(number) AS k, length(groupArray(1)([hex(number)] AS i)), leng DROP TABLE test.numbers_mt; --- Check binary compability: +-- Check binary compatibility: -- clickhouse-client -h old -q "SELECT arrayReduce('groupArrayState', [['1'], ['22'], ['333']]) FORMAT RowBinary" | clickhouse-local -s --input-format RowBinary --structure "d AggregateFunction(groupArray2, Array(String))" -q "SELECT groupArray2Merge(d) FROM table" From d8ce0059833672152b703deb1596993af4a8574c Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Wed, 23 Jan 2019 11:38:32 +0300 Subject: [PATCH 69/71] Docs for multi string search (#4123) --- .../functions/string_search_functions.md | 18 ++++++++++++++++++ .../functions/string_search_functions.md | 15 +++++++++++++++ 2 files changed, 33 insertions(+) diff --git a/docs/en/query_language/functions/string_search_functions.md b/docs/en/query_language/functions/string_search_functions.md index ea5b1ecf8ff..ced657da2ed 100644 --- a/docs/en/query_language/functions/string_search_functions.md +++ b/docs/en/query_language/functions/string_search_functions.md @@ -16,6 +16,24 @@ The same as `position`, but the position is returned in Unicode code points. Wor For a case-insensitive search, use the function `positionCaseInsensitiveUTF8`. +## multiPosition(haystack, [needle_1, needle_2, ..., needle_n]) + +The same as `position`, but returns `Array` of the `position`s for all `needle_i`. + +For a case-insensitive search or/and in UTF-8 format use functions `multiPositionCaseInsensitive, multiPositionUTF8, multiPositionCaseInsensitiveUTF8`. + +## firstMatch(haystack, [needle_1, needle_2, ..., needle_n]) + +Returns the index `i` (starting from 1) of the first found `needle_i` in the string `haystack` and 0 otherwise. + +For a case-insensitive search or/and in UTF-8 format use functions `firstMatchCaseInsensitive, firstMatchUTF8, firstMatchCaseInsensitiveUTF8`. + +## multiSearch(haystack, [needle_1, needle_2, ..., needle_n]) + +Returns 1, if at least one string `needle_i` matches the string `haystack` and 0 otherwise. + +For a case-insensitive search or/and in UTF-8 format use functions `multiSearchCaseInsensitive, multiSearchUTF8, multiSearchCaseInsensitiveUTF8`. + ## match(haystack, pattern) Checks whether the string matches the `pattern` regular expression. A `re2` regular expression. The [syntax](https://github.com/google/re2/wiki/Syntax) of the `re2` regular expressions is more limited than the syntax of the Perl regular expressions. diff --git a/docs/ru/query_language/functions/string_search_functions.md b/docs/ru/query_language/functions/string_search_functions.md index 031082c6659..3bd5e77fac2 100644 --- a/docs/ru/query_language/functions/string_search_functions.md +++ b/docs/ru/query_language/functions/string_search_functions.md @@ -14,6 +14,21 @@ Для поиска без учета регистра используйте функцию `positionCaseInsensitiveUTF8`. +## multiPosition(haystack, [needle_1, needle_2, ..., needle_n]) +Так же, как и `position`, только возвращает `Array` первых вхождений. + +Для поиска без учета регистра и/или в кодировке UTF-8 используйте функции `multiPositionCaseInsensitive, multiPositionUTF8, multiPositionCaseInsensitiveUTF8`. + +## firstMatch(haystack, [needle_1, needle_2, ..., needle_n]) +Возвращает индекс `i` (нумерация с единицы) первой найденной строки `needle_i` в строке `haystack` и 0 иначе. + +Для поиска без учета регистра и/или в кодировке UTF-8 используйте функции `firstMatchCaseInsensitive, firstMatchUTF8, firstMatchCaseInsensitiveUTF8`. + +## multiSearch(haystack, [needle_1, needle_2, ..., needle_n]) +Возвращает 1, если хотя бы одна подстрока `needle_i` нашлась в строке `haystack` и 0 иначе. + +Для поиска без учета регистра и/или в кодировке UTF-8 используйте функции `multiSearchCaseInsensitive, multiSearchUTF8, multiSearchCaseInsensitiveUTF8`. + ## match(haystack, pattern) Проверка строки на соответствие регулярному выражению pattern. Регулярное выражение **re2**. Синтаксис регулярных выражений **re2** является более ограниченным по сравнению с регулярными выражениями **Perl** ([подробнее](https://github.com/google/re2/wiki/Syntax)). Возвращает 0 (если не соответствует) или 1 (если соответствует). From f6225b9c321f37bff9f0a128efd7267c81750339 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 23 Jan 2019 11:57:42 +0300 Subject: [PATCH 70/71] Restore Upcoming Events README.md section [C++ ClickHouse and CatBoost Sprints](https://events.yandex.ru/events/ClickHouse/2-feb-2019/) in Moscow on February 2. --- README.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/README.md b/README.md index 0e9974f763f..f496e32b905 100644 --- a/README.md +++ b/README.md @@ -10,3 +10,7 @@ ClickHouse is an open-source column-oriented database management system that all * [Blog](https://clickhouse.yandex/blog/en/) contains various ClickHouse-related articles, as well as announces and reports about events. * [Contacts](https://clickhouse.yandex/#contacts) can help to get your questions answered if there are any. * You can also [fill this form](https://forms.yandex.com/surveys/meet-yandex-clickhouse-team/) to meet Yandex ClickHouse team in person. + +## Upcoming Events + +* [C++ ClickHouse and CatBoost Sprints](https://events.yandex.ru/events/ClickHouse/2-feb-2019/) in Moscow on February 2. From 0ed41dc4c011734b69f924d52c1cd520696019e4 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 23 Jan 2019 12:12:29 +0300 Subject: [PATCH 71/71] Add C++ ClickHouse and CatBoost Sprints link to website front page (#4124) --- website/index.html | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/website/index.html b/website/index.html index 8e2f2427057..9961d229320 100644 --- a/website/index.html +++ b/website/index.html @@ -92,6 +92,11 @@
+

ClickHouse. Just makes you think faster.