Merge branch 'master' into PF202202071031

This commit is contained in:
mergify[bot] 2022-02-07 09:17:26 +00:00 committed by GitHub
commit 84c2f5e464
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
70 changed files with 967 additions and 777 deletions

View File

@ -26,4 +26,6 @@ toc_title: Introduction
- [Replicated](../../engines/database-engines/replicated.md)
- [SQLite](../../engines/database-engines/sqlite.md)
[来源文章](https://clickhouse.com/docs/en/database_engines/) <!--hide-->

View File

@ -315,7 +315,7 @@ void LocalServer::cleanup()
std::string LocalServer::getInitialCreateTableQuery()
{
if (!config().has("table-structure") && !config().has("table-file"))
if (!config().has("table-structure") && !config().has("table-file") && !config().has("table-data-format"))
return {};
auto table_name = backQuoteIfNeed(config().getString("table-name", "table"));

View File

@ -1,8 +1,8 @@
#include <Access/AccessRights.h>
#include <base/logger_useful.h>
#include <base/sort.h>
#include <boost/container/small_vector.hpp>
#include <boost/range/adaptor/map.hpp>
#include <boost/range/algorithm/sort.hpp>
#include <unordered_map>
namespace DB
@ -101,7 +101,7 @@ namespace
AccessRightsElements getResult() const
{
ProtoElements sorted = *this;
boost::range::sort(sorted);
::sort(sorted.begin(), sorted.end());
AccessRightsElements res;
res.reserve(sorted.size());

View File

@ -7,8 +7,8 @@
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <boost/range/algorithm/set_algorithm.hpp>
#include <boost/range/algorithm/sort.hpp>
#include <boost/range/algorithm_ext/push_back.hpp>
#include <base/sort.h>
namespace DB
@ -132,7 +132,7 @@ std::shared_ptr<ASTRolesOrUsersSet> RolesOrUsersSet::toAST() const
ast->names.reserve(ids.size());
for (const UUID & id : ids)
ast->names.emplace_back(::DB::toString(id));
boost::range::sort(ast->names);
::sort(ast->names.begin(), ast->names.end());
}
if (!except_ids.empty())
@ -140,7 +140,7 @@ std::shared_ptr<ASTRolesOrUsersSet> RolesOrUsersSet::toAST() const
ast->except_names.reserve(except_ids.size());
for (const UUID & except_id : except_ids)
ast->except_names.emplace_back(::DB::toString(except_id));
boost::range::sort(ast->except_names);
::sort(ast->except_names.begin(), ast->except_names.end());
}
return ast;
@ -161,7 +161,7 @@ std::shared_ptr<ASTRolesOrUsersSet> RolesOrUsersSet::toASTWithNames(const Access
if (name)
ast->names.emplace_back(std::move(*name));
}
boost::range::sort(ast->names);
::sort(ast->names.begin(), ast->names.end());
}
if (!except_ids.empty())
@ -173,7 +173,7 @@ std::shared_ptr<ASTRolesOrUsersSet> RolesOrUsersSet::toASTWithNames(const Access
if (except_name)
ast->except_names.emplace_back(std::move(*except_name));
}
boost::range::sort(ast->except_names);
::sort(ast->except_names.begin(), ast->except_names.end());
}
return ast;

View File

@ -90,11 +90,11 @@ struct AggregateFunctionIntervalLengthSumData
void sort()
{
if (!sorted)
{
::sort(std::begin(segments), std::end(segments));
sorted = true;
}
if (sorted)
return;
::sort(std::begin(segments), std::end(segments));
sorted = true;
}
void serialize(WriteBuffer & buf) const

View File

@ -75,11 +75,11 @@ struct AggregateFunctionSequenceMatchData final
void sort()
{
if (!sorted)
{
::sort(std::begin(events_list), std::end(events_list), Comparator{});
sorted = true;
}
if (sorted)
return;
::sort(std::begin(events_list), std::end(events_list), Comparator{});
sorted = true;
}
void serialize(WriteBuffer & buf) const

View File

@ -260,7 +260,8 @@ private:
if (sorted)
return;
::sort(samples.begin(), samples.end(), [](const auto & lhs, const auto & rhs) { return lhs.first < rhs.first; });
/// In order to provide deterministic result we must sort by value and hash
::sort(samples.begin(), samples.end(), [](const auto & lhs, const auto & rhs) { return lhs < rhs; });
sorted = true;
}

View File

@ -1,6 +1,5 @@
#include <algorithm>
#include <vector>
#include <base/sort.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include "arrayScalarProduct.h"
@ -113,7 +112,8 @@ public:
sorted_labels[i].label = label;
}
::sort(sorted_labels.begin(), sorted_labels.end(), [](const auto & lhs, const auto & rhs) { return lhs.score > rhs.score; });
/// Stable sort is required for for labels to apply in same order if score is equal
std::stable_sort(sorted_labels.begin(), sorted_labels.end(), [](const auto & lhs, const auto & rhs) { return lhs.score > rhs.score; });
/// We will first calculate non-normalized area.

View File

@ -10,7 +10,7 @@
#include <Access/Common/AccessFlags.h>
#include <Access/AccessControl.h>
#include <base/range.h>
#include <boost/range/algorithm/sort.hpp>
#include <base/sort.h>
#include <boost/range/algorithm_ext/push_back.hpp>
@ -61,7 +61,7 @@ std::vector<AccessEntityPtr> InterpreterShowAccessQuery::getEntities() const
}
}
boost::range::sort(entities, IAccessEntity::LessByTypeAndName{});
::sort(entities.begin(), entities.end(), IAccessEntity::LessByTypeAndName{});
return entities;
}

View File

@ -27,7 +27,7 @@
#include <Interpreters/Context.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <base/range.h>
#include <boost/range/algorithm/sort.hpp>
#include <base/sort.h>
namespace DB
@ -341,7 +341,7 @@ std::vector<AccessEntityPtr> InterpreterShowCreateAccessEntityQuery::getEntities
entities.push_back(access_control.read(access_control.getID(show_query.type, name)));
}
boost::range::sort(entities, IAccessEntity::LessByName{});
::sort(entities.begin(), entities.end(), IAccessEntity::LessByName{});
return entities;
}

View File

@ -13,8 +13,8 @@
#include <DataTypes/DataTypeString.h>
#include <Interpreters/Context.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <boost/range/algorithm/sort.hpp>
#include <boost/range/algorithm_ext/push_back.hpp>
#include <base/sort.h>
namespace DB
@ -159,7 +159,7 @@ std::vector<AccessEntityPtr> InterpreterShowGrantsQuery::getEntities() const
entities.push_back(entity);
}
boost::range::sort(entities, IAccessEntity::LessByTypeAndName{});
::sort(entities.begin(), entities.end(), IAccessEntity::LessByTypeAndName{});
return entities;
}

View File

@ -285,6 +285,15 @@ namespace
return Poco::Net::SocketAddress{peer.substr(peer.find(':') + 1)};
}
std::optional<String> getClientHeader(const String & key) const
{
const auto & client_metadata = grpc_context.client_metadata();
auto it = client_metadata.find(key);
if (it != client_metadata.end())
return String{it->second.data(), it->second.size()};
return std::nullopt;
}
void setResultCompression(grpc_compression_algorithm algorithm, grpc_compression_level level)
{
grpc_context.set_compression_algorithm(algorithm);
@ -296,8 +305,6 @@ namespace
setResultCompression(convertCompressionAlgorithm(compression.algorithm()), convertCompressionLevel(compression.level()));
}
grpc::ServerContext grpc_context;
protected:
CompletionCallback * getCallbackPtr(const CompletionCallback & callback)
{
@ -320,6 +327,8 @@ namespace
return &callback_in_map;
}
grpc::ServerContext grpc_context;
private:
grpc::ServerAsyncReaderWriter<GRPCResult, GRPCQueryInfo> reader_writer{&grpc_context};
std::unordered_map<size_t, CompletionCallback> callbacks;
@ -752,33 +761,21 @@ namespace
session->authenticate(user, password, user_address);
session->getClientInfo().quota_key = quota_key;
// Parse the OpenTelemetry traceparent header.
ClientInfo client_info = session->getClientInfo();
const auto & client_metadata = responder->grpc_context.client_metadata();
auto traceparent = client_metadata.find("traceparent");
if (traceparent != client_metadata.end())
/// Parse the OpenTelemetry traceparent header.
auto traceparent = responder->getClientHeader("traceparent");
if (traceparent)
{
grpc::string_ref parent_ref = traceparent->second;
std::string opentelemetry_traceparent(parent_ref.data(), parent_ref.length());
std::string error;
if (!client_info.client_trace_context.parseTraceparentHeader(
opentelemetry_traceparent, error))
String error;
if (!client_info.client_trace_context.parseTraceparentHeader(traceparent.value(), error))
{
throw Exception(ErrorCodes::BAD_REQUEST_PARAMETER,
"Failed to parse OpenTelemetry traceparent header '{}': {}",
opentelemetry_traceparent, error);
}
auto tracestate = client_metadata.find("tracestate");
if (tracestate != client_metadata.end())
{
grpc::string_ref state_ref = tracestate->second;
client_info.client_trace_context.tracestate =
std::string(state_ref.data(), state_ref.length());
}
else
{
client_info.client_trace_context.tracestate = "";
traceparent.value(), error);
}
auto tracestate = responder->getClientHeader("tracestate");
client_info.client_trace_context.tracestate = tracestate.value_or("");
}
/// The user could specify session identifier and session timeout.

View File

@ -23,7 +23,7 @@ NamesAndTypesList StorageSystemGrants::getNamesAndTypes()
NamesAndTypesList names_and_types{
{"user_name", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
{"role_name", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
{"access_type", std::make_shared<DataTypeEnum8>(StorageSystemPrivileges::getAccessTypeEnumValues())},
{"access_type", std::make_shared<DataTypeEnum16>(StorageSystemPrivileges::getAccessTypeEnumValues())},
{"database", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
{"table", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
{"column", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeString>())},
@ -46,7 +46,7 @@ void StorageSystemGrants::fillData(MutableColumns & res_columns, ContextPtr cont
auto & column_user_name_null_map = assert_cast<ColumnNullable &>(*res_columns[column_index++]).getNullMapData();
auto & column_role_name = assert_cast<ColumnString &>(assert_cast<ColumnNullable &>(*res_columns[column_index]).getNestedColumn());
auto & column_role_name_null_map = assert_cast<ColumnNullable &>(*res_columns[column_index++]).getNullMapData();
auto & column_access_type = assert_cast<ColumnInt8 &>(*res_columns[column_index++]).getData();
auto & column_access_type = assert_cast<ColumnInt16 &>(*res_columns[column_index++]).getData();
auto & column_database = assert_cast<ColumnString &>(assert_cast<ColumnNullable &>(*res_columns[column_index]).getNestedColumn());
auto & column_database_null_map = assert_cast<ColumnNullable &>(*res_columns[column_index++]).getNullMapData();
auto & column_table = assert_cast<ColumnString &>(assert_cast<ColumnNullable &>(*res_columns[column_index]).getNestedColumn());
@ -82,7 +82,7 @@ void StorageSystemGrants::fillData(MutableColumns & res_columns, ContextPtr cont
else
assert(false);
column_access_type.push_back(static_cast<Int8>(access_type));
column_access_type.push_back(static_cast<Int16>(access_type));
if (database)
{

View File

@ -44,11 +44,11 @@ namespace
}
const std::vector<std::pair<String, Int8>> & StorageSystemPrivileges::getAccessTypeEnumValues()
const std::vector<std::pair<String, Int16>> & StorageSystemPrivileges::getAccessTypeEnumValues()
{
static const std::vector<std::pair<String, Int8>> values = []
static const std::vector<std::pair<String, Int16>> values = []
{
std::vector<std::pair<String, Int8>> res;
std::vector<std::pair<String, Int16>> res;
#define ADD_ACCESS_TYPE_ENUM_VALUE(name, aliases, node_type, parent_group_name) \
res.emplace_back(toString(AccessType::name), static_cast<size_t>(AccessType::name));
@ -65,10 +65,10 @@ const std::vector<std::pair<String, Int8>> & StorageSystemPrivileges::getAccessT
NamesAndTypesList StorageSystemPrivileges::getNamesAndTypes()
{
NamesAndTypesList names_and_types{
{"privilege", std::make_shared<DataTypeEnum8>(getAccessTypeEnumValues())},
{"privilege", std::make_shared<DataTypeEnum16>(getAccessTypeEnumValues())},
{"aliases", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
{"level", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeEnum8>(getLevelEnumValues()))},
{"parent_group", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeEnum8>(getAccessTypeEnumValues()))},
{"parent_group", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeEnum16>(getAccessTypeEnumValues()))},
};
return names_and_types;
}
@ -77,17 +77,17 @@ NamesAndTypesList StorageSystemPrivileges::getNamesAndTypes()
void StorageSystemPrivileges::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const
{
size_t column_index = 0;
auto & column_access_type = assert_cast<ColumnInt8 &>(*res_columns[column_index++]).getData();
auto & column_access_type = assert_cast<ColumnInt16 &>(*res_columns[column_index++]).getData();
auto & column_aliases = assert_cast<ColumnString &>(assert_cast<ColumnArray &>(*res_columns[column_index]).getData());
auto & column_aliases_offsets = assert_cast<ColumnArray &>(*res_columns[column_index++]).getOffsets();
auto & column_level = assert_cast<ColumnInt8 &>(assert_cast<ColumnNullable &>(*res_columns[column_index]).getNestedColumn()).getData();
auto & column_level_null_map = assert_cast<ColumnNullable &>(*res_columns[column_index++]).getNullMapData();
auto & column_parent_group = assert_cast<ColumnInt8 &>(assert_cast<ColumnNullable &>(*res_columns[column_index]).getNestedColumn()).getData();
auto & column_parent_group = assert_cast<ColumnInt16 &>(assert_cast<ColumnNullable &>(*res_columns[column_index]).getNestedColumn()).getData();
auto & column_parent_group_null_map = assert_cast<ColumnNullable &>(*res_columns[column_index++]).getNullMapData();
auto add_row = [&](AccessType access_type, const std::string_view & aliases, Level max_level, AccessType parent_group)
{
column_access_type.push_back(static_cast<Int8>(access_type));
column_access_type.push_back(static_cast<Int16>(access_type));
for (size_t pos = 0; pos < aliases.length();)
{
@ -121,7 +121,7 @@ void StorageSystemPrivileges::fillData(MutableColumns & res_columns, ContextPtr,
}
else
{
column_parent_group.push_back(static_cast<Int8>(parent_group));
column_parent_group.push_back(static_cast<Int16>(parent_group));
column_parent_group_null_map.push_back(false);
}
};

View File

@ -14,7 +14,7 @@ class StorageSystemPrivileges final : public shared_ptr_helper<StorageSystemPriv
public:
std::string getName() const override { return "SystemPrivileges"; }
static NamesAndTypesList getNamesAndTypes();
static const std::vector<std::pair<String, Int8>> & getAccessTypeEnumValues();
static const std::vector<std::pair<String, Int16>> & getAccessTypeEnumValues();
protected:
friend struct shared_ptr_helper<StorageSystemPrivileges>;

View File

@ -431,3 +431,18 @@ def test_compressed_external_table():
b"3\tCarl\n"\
b"4\tDaniel\n"\
b"5\tEthan\n"
def test_opentelemetry_context_propagation():
trace_id = "80c190b5-9dc1-4eae-82b9-6c261438c817"
parent_span_id = 123
trace_state = "some custom state"
trace_id_hex = trace_id.replace("-", "")
parent_span_id_hex = f'{parent_span_id:0>16X}'
metadata = [("traceparent", f"00-{trace_id_hex}-{parent_span_id_hex}-01"), ("tracestate", trace_state)]
stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel)
query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 1")
result = stub.ExecuteQuery(query_info, metadata=metadata)
assert result.output == b"1\n"
node.query("SYSTEM FLUSH LOGS")
assert node.query(f"SELECT attribute['db.statement'], attribute['clickhouse.tracestate'] FROM system.opentelemetry_span_log "
f"WHERE trace_id='{trace_id}' AND parent_span_id={parent_span_id}") == "SELECT 1\tsome custom state\n"

View File

@ -382,12 +382,18 @@ def test_round_robin(start_cluster, name, engine):
used_disk = get_used_disks_for_table(node1, name)
assert len(used_disk) == 1, 'More than one disk used for single insert'
# sleep is required because we order disks by their modification time, and if insert will be fast
# modification time of two disks will be equal, then sort will not provide deterministic results
time.sleep(5)
node1.query_with_retry("insert into {} select * from numbers(10000, 10000)".format(name))
used_disks = get_used_disks_for_table(node1, name)
assert len(used_disks) == 2, 'Two disks should be used for two parts'
assert used_disks[0] != used_disks[1], "Should write to different disks"
time.sleep(5)
node1.query_with_retry("insert into {} select * from numbers(20000, 10000)".format(name))
used_disks = get_used_disks_for_table(node1, name)

View File

@ -225,11 +225,11 @@ def test_introspection():
["R2", "local directory"]])
assert instance.query(
"SELECT * from system.grants WHERE user_name IN ('A', 'B') OR role_name IN ('R1', 'R2') ORDER BY user_name, role_name, access_type, grant_option") == \
"SELECT * from system.grants WHERE user_name IN ('A', 'B') OR role_name IN ('R1', 'R2') ORDER BY user_name, role_name, access_type, database, table, column, is_partial_revoke, grant_option") == \
TSV([["A", "\\N", "SELECT", "test", "table", "\\N", 0, 0],
["B", "\\N", "CREATE", "\\N", "\\N", "\\N", 0, 1],
["\\N", "R2", "SELECT", "test", "table", "\\N", 0, 0],
["\\N", "R2", "SELECT", "test", "table", "x", 1, 0]])
["\\N", "R2", "SELECT", "test", "table", "x", 1, 0],
["\\N", "R2", "SELECT", "test", "table", "\\N", 0, 0]])
assert instance.query(
"SELECT * from system.role_grants WHERE user_name IN ('A', 'B') OR role_name IN ('R1', 'R2') ORDER BY user_name, role_name, granted_role_name") == \

View File

@ -15,7 +15,7 @@ Hello "world", 789 ,2016-01-03
default,,
default-eof,,' | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=1 --input_format_csv_empty_as_default=1 --query="INSERT INTO csv FORMAT CSV";
$CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY d";
$CLICKHOUSE_CLIENT --query="SELECT * FROM csv ORDER BY d, s";
$CLICKHOUSE_CLIENT --query="DROP TABLE csv";
$CLICKHOUSE_CLIENT --query="CREATE TABLE csv (t DateTime('Europe/Moscow'), s String) ENGINE = Memory";

View File

@ -1,8 +1,8 @@
2000-01-01 1 [1,2] [3,4] [10,11] [0,1,2] ['3','4','5'] [-1,-2,-3] [1,10,100]
2000-01-01 1 [2,1] [4,3] [20,22] [2,2,1] ['5','5','0'] [-3,-3,-33] [10,100,1000]
2000-01-01 2 [1,2] [3,4] [10,11] [0,1,2] ['3','4','5'] [-1,-2,-3] [1,10,100]
2000-01-01 2 [2,1,1] [4,3,3] [20,22,33] [2,2] ['5','5'] [-3,-3] [10,100]
2000-01-01 2 [1,2] [3,4] [10,11] [0,1,2] ['3','4','5'] [-1,-2,-3] [1,10,100]
2000-01-01 2 [2,1,1] [4,3,3] [20,22,33] [2,2] ['5','5'] [-3,-3] [10,100]
2000-01-01 1 1 3 10
2000-01-01 1 1 3 22
2000-01-01 1 2 4 11

View File

@ -5,11 +5,11 @@ CREATE TABLE summing_composite_key (d Date, k UInt64, FirstMap Nested(k1 UInt32,
INSERT INTO summing_composite_key VALUES ('2000-01-01', 1, [1,2], ['3','4'], [10,11], [0,1,2], [3,4,5], [-1,-2,-3], [1,10,100]), ('2000-01-01', 1, [2,1], ['4','3'], [20,22], [2,2,1], [5,5,0], [-3,-3,-33], [10,100,1000]), ('2000-01-01', 2, [1,2], ['3','4'], [10,11], [0,1,2], [3,4,5], [-1,-2,-3], [1,10,100]), ('2000-01-01', 2, [2,1,1], ['4','3','3'], [20,22,33], [2,2], [5,5], [-3,-3], [10,100]), ('2000-01-01', 2, [1,2], ['3','4'], [10,11], [0,1,2], [3,4,5], [-1,-2,-3], [1,10,100]);
SELECT * FROM summing_composite_key ORDER BY d, k, _part_index;
SELECT * FROM summing_composite_key ORDER BY d, k, FirstMap.k1, FirstMap.k2ID, FirstMap.s, SecondMap.k1ID, SecondMap.k2Key, SecondMap.k3Type, SecondMap.s;
SELECT d, k, m.k1, m.k2ID, m.s FROM summing_composite_key ARRAY JOIN FirstMap AS m ORDER BY d, k, m.k1, m.k2ID, m.s;
SELECT d, k, m.k1, m.k2ID, m.s FROM summing_composite_key ARRAY JOIN FirstMap AS m ORDER BY d, k, m.k1, m.k2ID, m.s, SecondMap.k1ID, SecondMap.k2Key, SecondMap.k3Type, SecondMap.s;
SELECT d, k, m.k1, m.k2ID, sum(m.s) FROM summing_composite_key ARRAY JOIN FirstMap AS m GROUP BY d, k, m.k1, m.k2ID ORDER BY d, k, m.k1, m.k2ID;
SELECT d, k, m.k1, m.k2ID,m. s FROM summing_composite_key FINAL ARRAY JOIN FirstMap AS m ORDER BY d, k, m.k1, m.k2ID, m.s;
SELECT d, k, m.k1, m.k2ID, m.s FROM summing_composite_key FINAL ARRAY JOIN FirstMap AS m ORDER BY d, k, m.k1, m.k2ID, m.s;
SELECT d, k, m.k1ID, m.k2Key, m.k3Type, m.s FROM summing_composite_key ARRAY JOIN SecondMap AS m ORDER BY d, k, m.k1ID, m.k2Key, m.k3Type, m.s;
SELECT d, k, m.k1ID, m.k2Key, m.k3Type, sum(m.s) FROM summing_composite_key ARRAY JOIN SecondMap AS m GROUP BY d, k, m.k1ID, m.k2Key, m.k3Type ORDER BY d, k, m.k1ID, m.k2Key, m.k3Type;
@ -17,7 +17,7 @@ SELECT d, k, m.k1ID, m.k2Key, m.k3Type, m.s FROM summing_composite_key FINAL ARR
OPTIMIZE TABLE summing_composite_key PARTITION 200001 FINAL;
SELECT * FROM summing_composite_key ORDER BY d, k, _part_index;
SELECT * FROM summing_composite_key ORDER BY d, k, FirstMap.k1, FirstMap.k2ID, FirstMap.s, SecondMap.k1ID, SecondMap.k2Key, SecondMap.k3Type, SecondMap.s;;
SELECT d, k, m.k1, m.k2ID, m.s FROM summing_composite_key ARRAY JOIN FirstMap AS m ORDER BY d, k, m.k1, m.k2ID, m.s;
SELECT d, k, m.k1, m.k2ID, sum(m.s) FROM summing_composite_key ARRAY JOIN FirstMap AS m GROUP BY d, k, m.k1, m.k2ID ORDER BY d, k, m.k1, m.k2ID;

View File

@ -5,7 +5,7 @@ CREATE TABLE sum_map(date Date, timeslot DateTime, statusMap Nested(status UInt1
INSERT INTO sum_map VALUES ('2000-01-01', '2000-01-01 00:00:00', [1, 2, 3], [10, 10, 10]), ('2000-01-01', '2000-01-01 00:00:00', [3, 4, 5], [10, 10, 10]), ('2000-01-01', '2000-01-01 00:01:00', [4, 5, 6], [10, 10, 10]), ('2000-01-01', '2000-01-01 00:01:00', [6, 7, 8], [10, 10, 10]);
SELECT * FROM sum_map ORDER BY timeslot;
SELECT * FROM sum_map ORDER BY timeslot, statusMap.status, statusMap.requests;
SELECT sumMap(statusMap.status, statusMap.requests) FROM sum_map;
SELECT sumMap((statusMap.status, statusMap.requests)) FROM sum_map;
SELECT sumMapMerge(s) FROM (SELECT sumMapState(statusMap.status, statusMap.requests) AS s FROM sum_map);

View File

@ -8,11 +8,11 @@ create table test_in_tuple as test_in_tuple_1 engine = Merge(currentDatabase(),
insert into test_in_tuple_1 values (1, 1, [1, 2], [1, 2]);
insert into test_in_tuple_2 values (2, 1, [1, 2], [1, 2]);
select key, arr_x, arr_y, _table from test_in_tuple left array join x as arr_x, y as arr_y order by _table;
select key, arr_x, arr_y, _table from test_in_tuple left array join x as arr_x, y as arr_y order by _table, arr_x, arr_y;
select '-';
select key, arr_x, arr_y, _table from test_in_tuple left array join x as arr_x, y as arr_y where (key_2, arr_x, arr_y) in (1, 1, 1) order by _table;
select key, arr_x, arr_y, _table from test_in_tuple left array join x as arr_x, y as arr_y where (key_2, arr_x, arr_y) in (1, 1, 1) order by _table, arr_x, arr_y;
select '-';
select key, arr_x, arr_y, _table from test_in_tuple left array join arrayFilter((t, x_0, x_1) -> (key_2, x_0, x_1) in (1, 1, 1), x, x ,y) as arr_x, arrayFilter((t, x_0, x_1) -> (key_2, x_0, x_1) in (1, 1, 1), y, x ,y) as arr_y where (key_2, arr_x, arr_y) in (1, 1, 1) order by _table;
select key, arr_x, arr_y, _table from test_in_tuple left array join arrayFilter((t, x_0, x_1) -> (key_2, x_0, x_1) in (1, 1, 1), x, x ,y) as arr_x, arrayFilter((t, x_0, x_1) -> (key_2, x_0, x_1) in (1, 1, 1), y, x ,y) as arr_y where (key_2, arr_x, arr_y) in (1, 1, 1) order by _table, arr_x, arr_y;
drop table if exists test_in_tuple_1;
drop table if exists test_in_tuple_2;

View File

@ -10,68 +10,68 @@ insert into Y (id, y_a) values (1, 'r1'), (1, 'r2'), (2, 'r3'), (3, 'r4'),
insert into Y (id, y_a, y_b) values (4, 'r6', 'nr6'), (6, 'r7', 'nr7'), (7, 'r8', 'nr8'), (9, 'r9', 'nr9');
select 'inner';
select X.*, Y.* from X inner join Y on X.id = Y.id order by id;
select X.*, Y.* from X inner join Y on X.id = Y.id order by X.id, X.x_a, X.x_b, Y.id, Y.y_a, Y.y_b;
select 'inner subs';
select s.*, j.* from (select * from X) as s inner join (select * from Y) as j on s.id = j.id order by id;
select s.*, j.* from (select * from X) as s inner join (select * from Y) as j on s.id = j.id order by s.id, s.x_a, s.x_b, j.id, j.y_a, j.y_b;
select 'inner expr';
select X.*, Y.* from X inner join Y on (X.id + 1) = (Y.id + 1) order by id;
select X.*, Y.* from X inner join Y on (X.id + 1) = (Y.id + 1) order by X.id, X.x_a, X.x_b, Y.id, Y.y_a, Y.y_b;
select 'left';
select X.*, Y.* from X left join Y on X.id = Y.id order by id;
select X.*, Y.* from X left join Y on X.id = Y.id order by X.id, X.x_a, X.x_b, Y.id, Y.y_a, Y.y_b;
select 'left subs';
select s.*, j.* from (select * from X) as s left join (select * from Y) as j on s.id = j.id order by id;
select s.*, j.* from (select * from X) as s left join (select * from Y) as j on s.id = j.id order by s.id, s.x_a, s.x_b, j.id, j.y_a, j.y_b;
select 'left expr';
select X.*, Y.* from X left join Y on (X.id + 1) = (Y.id + 1) order by id;
select X.*, Y.* from X left join Y on (X.id + 1) = (Y.id + 1) order by X.id, X.x_a, X.x_b, Y.id, Y.y_a, Y.y_b;
select 'right';
select X.*, Y.* from X right join Y on X.id = Y.id order by id;
select X.*, Y.* from X right join Y on X.id = Y.id order by X.id, X.x_a, X.x_b, Y.id, Y.y_a, Y.y_b;
select 'right subs';
select s.*, j.* from (select * from X) as s right join (select * from Y) as j on s.id = j.id order by id;
select s.*, j.* from (select * from X) as s right join (select * from Y) as j on s.id = j.id order by s.id, s.x_a, s.x_b, j.id, j.y_a, j.y_b;
--select 'right expr';
--select X.*, Y.* from X right join Y on (X.id + 1) = (Y.id + 1) order by id;
select 'full';
select X.*, Y.* from X full join Y on X.id = Y.id order by id;
select X.*, Y.* from X full join Y on X.id = Y.id order by X.id, X.x_a, X.x_b, Y.id, Y.y_a, Y.y_b;
select 'full subs';
select s.*, j.* from (select * from X) as s full join (select * from Y) as j on s.id = j.id order by id;
select s.*, j.* from (select * from X) as s full join (select * from Y) as j on s.id = j.id order by s.id, s.x_a;
--select 'full expr';
--select X.*, Y.* from X full join Y on (X.id + 1) = (Y.id + 1) order by id;
select 'self inner';
select X.*, s.* from X inner join (select * from X) as s on X.id = s.id order by X.id, X.x_a, s.x_a;
select X.*, s.* from X inner join (select * from X) as s on X.id = s.id order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
select 'self inner nullable';
select X.*, s.* from X inner join (select * from X) as s on X.x_b = s.x_b order by X.id;
select X.*, s.* from X inner join (select * from X) as s on X.x_b = s.x_b order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
select 'self inner nullable vs not nullable';
select X.*, s.* from X inner join (select * from X) as s on X.id = s.x_b order by X.id;
select X.*, s.* from X inner join (select * from X) as s on X.id = s.x_b order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
-- TODO: s.y_b == '' instead of NULL
select 'self inner nullable vs not nullable 2';
select Y.*, s.* from Y inner join (select * from Y) as s on concat('n', Y.y_a) = s.y_b order by id;
select Y.*, s.* from Y inner join (select * from Y) as s on concat('n', Y.y_a) = s.y_b order by Y.id, Y.y_a, Y.y_b, s.id, s.y_a, s.y_b;
select 'self left';
select X.*, s.* from X left join (select * from X) as s on X.id = s.id order by X.id, X.x_a, s.x_a;
select 'self left nullable';
select X.*, s.* from X left join (select * from X) as s on X.x_b = s.x_b order by X.id;
select X.*, s.* from X left join (select * from X) as s on X.x_b = s.x_b order by X.id, X.x_a;
select 'self left nullable vs not nullable';
select X.*, s.* from X left join (select * from X) as s on X.id = s.x_b order by X.id;
select X.*, s.* from X left join (select * from X) as s on X.id = s.x_b order by X.id, X.x_a;
-- TODO: s.y_b == '' instead of NULL
select 'self left nullable vs not nullable 2';
select Y.*, s.* from Y left join (select * from Y) as s on concat('n', Y.y_a) = s.y_b order by id;
select Y.*, s.* from Y left join (select * from Y) as s on concat('n', Y.y_a) = s.y_b order by Y.id, Y.y_a;
select 'self right';
select X.*, s.* from X right join (select * from X) as s on X.id = s.id order by X.id, X.x_a, s.x_a;
select X.*, s.* from X right join (select * from X) as s on X.id = s.id order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
select 'self right nullable';
select X.*, s.* from X right join (select * from X) as s on X.x_b = s.x_b order by X.id;
select X.*, s.* from X right join (select * from X) as s on X.x_b = s.x_b order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
select 'self right nullable vs not nullable';
select X.*, s.* from X right join (select * from X) as s on X.id = s.x_b order by X.id;
select X.*, s.* from X right join (select * from X) as s on X.id = s.x_b order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
--select 'self right nullable vs not nullable 2';
--select Y.*, s.* from Y right join (select * from Y) as s on concat('n', Y.y_a) = s.y_b order by id;
select 'self full';
select X.*, s.* from X full join (select * from X) as s on X.id = s.id order by X.id, X.x_a, s.x_a;
select X.*, s.* from X full join (select * from X) as s on X.id = s.id order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
select 'self full nullable';
select X.*, s.* from X full join (select * from X) as s on X.x_b = s.x_b order by X.id;
select X.*, s.* from X full join (select * from X) as s on X.x_b = s.x_b order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
select 'self full nullable vs not nullable';
select X.*, s.* from X full join (select * from X) as s on X.id = s.x_b order by X.id;
select X.*, s.* from X full join (select * from X) as s on X.id = s.x_b order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
--select 'self full nullable vs not nullable 2';
--select Y.*, s.* from Y full join (select * from Y) as s on concat('n', Y.y_a) = s.y_b order by id;

View File

@ -8,24 +8,24 @@ insert into X (id, x_name) values (1, 'A'), (2, 'B'), (2, 'C'), (3, 'D'), (4, 'E
insert into Y (id, y_name) values (1, 'a'), (1, 'b'), (2, 'c'), (3, 'd'), (3, 'e'), (4, 'f'), (6, 'g'), (7, 'h'), (9, 'i');
select 'inner';
select X.*, Y.* from X inner join Y using id;
select X.*, Y.* from X inner join Y using id order by X.id, Y.id, X.x_name, Y.y_name;
select 'inner subs';
select s.*, j.* from (select * from X) as s inner join (select * from Y) as j using id;
select s.*, j.* from (select * from X) as s inner join (select * from Y) as j using id order by s.id, s.id, s.x_name, j.y_name;
select 'left';
select X.*, Y.* from X left join Y using id;
select X.*, Y.* from X left join Y using id order by X.id, Y.id, X.x_name, Y.y_name;
select 'left subs';
select s.*, j.* from (select * from X) as s left join (select * from Y) as j using id;
select s.*, j.* from (select * from X) as s left join (select * from Y) as j using id order by s.id, j.id, s.x_name, j.y_name;
select 'right';
select X.*, Y.* from X right join Y using id order by id;
select X.*, Y.* from X right join Y using id order by X.id, Y.id, X.x_name, Y.y_name;
select 'right subs';
select s.*, j.* from (select * from X) as s right join (select * from Y) as j using id order by id;
select s.*, j.* from (select * from X) as s right join (select * from Y) as j using id order by s.id, j.id, s.x_name, j.y_name;
select 'full';
select X.*, Y.* from X full join Y using id order by id;
select X.*, Y.* from X full join Y using id order by X.id, Y.id;
select 'full subs';
select s.*, j.* from (select * from X) as s full join (select * from Y) as j using id order by id;
select s.*, j.* from (select * from X) as s full join (select * from Y) as j using id order by s.id, j.id, s.x_name, j.y_name;
drop table X;
drop table Y;

View File

@ -1,9 +1,7 @@
*** Check that the parts are sorted according to the new key. ***
1 2 0 10
1 2 0 20
1 2 2 40
1 2 2 50
1 2 1 30
1 2 0 10
*** Check that the rows are collapsed according to the new key. ***
1 2 0 30
1 2 1 30

View File

@ -21,14 +21,14 @@ ALTER TABLE summing MODIFY ORDER BY (x, y, nonexistent); -- { serverError 47}
/* Can't modyfy ORDER BY so that it is no longer a prefix of the PRIMARY KEY. */
ALTER TABLE summing MODIFY ORDER BY x; -- { serverError 36}
INSERT INTO summing(x, y, val) VALUES (1, 2, 10), (1, 2, 20);
ALTER TABLE summing ADD COLUMN z UInt32 AFTER y, MODIFY ORDER BY (x, y, -z);
INSERT INTO summing(x, y, z, val) values (1, 2, 1, 30), (1, 2, 2, 40), (1, 2, 2, 50);
INSERT INTO summing(x, y, z, val) values (1, 2, 0, 10), (1, 2, 1, 30), (1, 2, 2, 40);
SELECT '*** Check that the parts are sorted according to the new key. ***';
SELECT * FROM summing ORDER BY _part;
SELECT * FROM summing;
INSERT INTO summing(x, y, z, val) values (1, 2, 0, 20), (1, 2, 2, 50);
SELECT '*** Check that the rows are collapsed according to the new key. ***';
SELECT * FROM summing FINAL ORDER BY x, y, z;

View File

@ -1,9 +1,7 @@
*** Check that the parts are sorted according to the new key. ***
1 2 0 10
1 2 0 20
1 2 2 40
1 2 2 50
1 2 1 30
1 2 0 10
*** Check that the rows are collapsed according to the new key. ***
1 2 0 30
1 2 1 30

View File

@ -27,16 +27,16 @@ ALTER TABLE summing_r1 MODIFY ORDER BY (x, y, nonexistent); -- { serverError 47
/* Can't modyfy ORDER BY so that it is no longer a prefix of the PRIMARY KEY. */
ALTER TABLE summing_r1 MODIFY ORDER BY x; -- { serverError 36 }
INSERT INTO summing_r1(x, y, val) VALUES (1, 2, 10), (1, 2, 20);
SYSTEM SYNC REPLICA summing_r2;
ALTER TABLE summing_r1 ADD COLUMN z UInt32 AFTER y, MODIFY ORDER BY (x, y, -z);
INSERT INTO summing_r1(x, y, z, val) values (1, 2, 1, 30), (1, 2, 2, 40), (1, 2, 2, 50);
INSERT INTO summing_r1(x, y, z, val) values (1, 2, 0, 10), (1, 2, 1, 30), (1, 2, 2, 40);
SYSTEM SYNC REPLICA summing_r2;
SELECT '*** Check that the parts are sorted according to the new key. ***';
SELECT * FROM summing_r2 ORDER BY _part;
SELECT * FROM summing_r2;
INSERT INTO summing_r1(x, y, z, val) values (1, 2, 0, 20), (1, 2, 2, 50);
SYSTEM SYNC REPLICA summing_r2;
SELECT '*** Check that the rows are collapsed according to the new key. ***';
SELECT * FROM summing_r2 FINAL ORDER BY x, y, z;

View File

@ -2,8 +2,8 @@
0.3 2018-11-19 13:05:00 \N
0.4 2018-11-19 13:10:00 1
0.5 2018-11-19 13:15:00 1.2
0.6 2018-11-19 13:15:00 1.5
0.7 2018-11-19 13:20:00 1.8
0.8 2018-11-19 13:25:00 2.1
0.9 2018-11-19 13:25:00 2.4
0.5 2018-11-19 13:30:00 2.2
0.6 2018-11-19 13:20:00 1.5
0.7 2018-11-19 13:25:00 1.8
0.8 2018-11-19 13:30:00 2.1
0.9 2018-11-19 13:45:00 2.4
0.5 2018-11-19 13:50:00 2.2

View File

@ -4,7 +4,7 @@ DROP TABLE IF EXISTS bm;
CREATE TABLE bm (amount float, business_dttm DateTime) engine Log;
INSERT INTO bm VALUES (0.3,'2018-11-19 13:00:00'), (0.3,'2018-11-19 13:05:00'), (0.4,'2018-11-19 13:10:00'), (0.5,'2018-11-19 13:15:00'), (0.6,'2018-11-19 13:15:00'), (0.7,'2018-11-19 13:20:00'), (0.8,'2018-11-19 13:25:00'), (0.9,'2018-11-19 13:25:00'), (0.5,'2018-11-19 13:30:00');
INSERT INTO bm VALUES (0.3,'2018-11-19 13:00:00'), (0.3,'2018-11-19 13:05:00'), (0.4,'2018-11-19 13:10:00'), (0.5,'2018-11-19 13:15:00'), (0.6,'2018-11-19 13:20:00'), (0.7,'2018-11-19 13:25:00'), (0.8,'2018-11-19 13:30:00'), (0.9,'2018-11-19 13:45:00'), (0.5,'2018-11-19 13:50:00');
WITH
(
@ -30,6 +30,6 @@ FROM
business_dttm
FROM bm
ORDER BY business_dttm
);
) ORDER BY business_dttm;
DROP TABLE bm;

View File

@ -9,8 +9,8 @@ INSERT INTO table2 VALUES ('D', 'd', '2018-01-01') ('B', 'b', '2018-01-01') ('C'
SELECT * FROM table1 t1 FORMAT PrettyCompact;
SELECT *, c as a, d as b FROM table2 FORMAT PrettyCompact;
SELECT * FROM table1 t1 ALL LEFT JOIN (SELECT *, c, d as b FROM table2) t2 USING (a, b) ORDER BY d FORMAT PrettyCompact;
SELECT * FROM table1 t1 ALL INNER JOIN (SELECT *, c, d as b FROM table2) t2 USING (a, b) ORDER BY d FORMAT PrettyCompact;
SELECT * FROM table1 t1 ALL LEFT JOIN (SELECT *, c, d as b FROM table2) t2 USING (a, b) ORDER BY d, t1.a FORMAT PrettyCompact;
SELECT * FROM table1 t1 ALL INNER JOIN (SELECT *, c, d as b FROM table2) t2 USING (a, b) ORDER BY d, t1.a FORMAT PrettyCompact;
DROP TABLE table1;
DROP TABLE table2;

View File

@ -21,37 +21,37 @@ SELECT s1.other, s2.other, count_a, count_b, toTypeName(s1.other), toTypeName(s2
ALL FULL JOIN
( SELECT other, count() AS count_b FROM table_b GROUP BY other ) s2
ON s1.other = s2.other
ORDER BY s2.other DESC, count_a;
ORDER BY s2.other DESC, count_a, s1.other;
SELECT s1.other, s2.other, count_a, count_b, toTypeName(s1.other), toTypeName(s2.other) FROM
( SELECT other, count() AS count_a FROM table_a GROUP BY other ) s1
ALL FULL JOIN
( SELECT other, count() AS count_b FROM table_b GROUP BY other ) s2
USING other
ORDER BY s2.other DESC, count_a;
ORDER BY s2.other DESC, count_a, s1.other;
SELECT s1.something, s2.something, count_a, count_b, toTypeName(s1.something), toTypeName(s2.something) FROM
( SELECT something, count() AS count_a FROM table_a GROUP BY something ) s1
ALL FULL JOIN
( SELECT something, count() AS count_b FROM table_b GROUP BY something ) s2
ON s1.something = s2.something
ORDER BY count_a DESC;
ORDER BY count_a DESC, something, s2.something;
SELECT s1.something, s2.something, count_a, count_b, toTypeName(s1.something), toTypeName(s2.something) FROM
( SELECT something, count() AS count_a FROM table_a GROUP BY something ) s1
ALL RIGHT JOIN
( SELECT something, count() AS count_b FROM table_b GROUP BY something ) s2
USING (something)
ORDER BY count_a DESC;
ORDER BY count_a DESC, s1.something, s2.something;
SET joined_subquery_requires_alias = 0;
SELECT something, count_a, count_b, toTypeName(something) FROM
( SELECT something, count() AS count_a FROM table_a GROUP BY something )
( SELECT something, count() AS count_a FROM table_a GROUP BY something ) as s1
ALL FULL JOIN
( SELECT something, count() AS count_b FROM table_b GROUP BY something )
( SELECT something, count() AS count_b FROM table_b GROUP BY something ) as s2
USING (something)
ORDER BY count_a DESC;
ORDER BY count_a DESC, something DESC;
DROP TABLE table_a;
DROP TABLE table_b;

View File

@ -110,116 +110,116 @@
77
636
1000
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
http://metric.ru/ 0
http://autometric.ru/ 0
http://metrica.yandex.com/ 0
http://metris.ru/ 0
http://metrika.ru/ 0
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
0
привет как дела?... Херсон 600
пап привет как дела - Яндекс.Видео 684
привет как дела клип - Яндекс.Видео 692
привет братан как дела - Яндекс.Видео 707
привет 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
привет 1000
0
http://metric.ru/ 765
http://metris.ru/ 765
http://metrika.ru/ 778
http://autometric.ru/ 810
http://metrica.yandex.com/ 846
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
привет как дела?... Херсон 297
пап привет как дела - Яндекс.Видео 422
привет как дела клип - Яндекс.Видео 435
привет братан как дела - Яндекс.Видео 500
привет 529
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
привет как дела?... Херсон 459
пап привет как дела - Яндекс.Видео 511
привет 529
привет как дела клип - Яндекс.Видео 565
привет братан как дела - Яндекс.Видео 583
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 524
http://metric.ru/ 700
http://metris.ru/ 700
http://autometric.ru/ 750
http://metrica.yandex.com/ 793
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
http://metric.ru/ 600
http://metrica.yandex.com/ 655
http://autometric.ru/ 667
http://metris.ru/ 700
http://metrika.ru/ 714
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
http://metrika.ru/ 619
http://metric.ru/ 700
http://metris.ru/ 700
http://autometric.ru/ 750
http://metrica.yandex.com/ 793
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
http://metric.ru/ 600
http://autometric.ru/ 667
http://metris.ru/ 700
http://metrika.ru/ 714
http://metrica.yandex.com/ 724
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
http://metrica.yandex.com/ 714
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
0
0
0
@ -332,138 +332,138 @@ http://metrika.ru/ 1000
77
636
1000
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
http://metric.ru/ 0
http://autometric.ru/ 0
http://metrica.yandex.com/ 0
http://metris.ru/ 0
http://metrika.ru/ 0
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
0
привет как дела?... Херсон 600
пап привет как дела - Яндекс.Видео 684
привет как дела клип - Яндекс.Видео 692
привет братан как дела - Яндекс.Видео 707
привет 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
привет 1000
0
http://metric.ru/ 765
http://metris.ru/ 765
http://metrika.ru/ 778
http://autometric.ru/ 810
http://metrica.yandex.com/ 846
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
привет как дела?... Херсон 297
пап привет как дела - Яндекс.Видео 422
привет как дела клип - Яндекс.Видео 435
привет братан как дела - Яндекс.Видео 500
привет 529
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
привет как дела?... Херсон 459
пап привет как дела - Яндекс.Видео 511
привет 529
привет как дела клип - Яндекс.Видео 565
привет братан как дела - Яндекс.Видео 583
http://metric.ru/ 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
http://metris.ru/ 1000
http://metrika.ru/ 524
http://metric.ru/ 700
http://metris.ru/ 700
http://autometric.ru/ 750
http://metrica.yandex.com/ 793
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
http://metrika.ru/ 524
http://metric.ru/ 700
http://metris.ru/ 700
http://autometric.ru/ 750
http://metrica.yandex.com/ 793
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
http://metric.ru/ 600
http://metrica.yandex.com/ 655
http://autometric.ru/ 667
http://metris.ru/ 700
http://metrika.ru/ 714
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
http://metrika.ru/ 619
http://metric.ru/ 700
http://metris.ru/ 700
http://autometric.ru/ 750
http://metrica.yandex.com/ 793
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
http://metric.ru/ 600
http://autometric.ru/ 667
http://metris.ru/ 700
http://metrika.ru/ 714
http://metrica.yandex.com/ 724
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
http://metrica.yandex.com/ 714
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 0
пап привет как дела - Яндекс.Видео 169
привет братан как дела - Яндекс.Видео 235
привет как дела?... Херсон 544
привет 784
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
0
0
0
@ -581,78 +581,78 @@ http://metrika.ru/ 1000
привет как дела клип - Яндекс.Видео 412
привет братан как дела - Яндекс.Видео 461
привет 471
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
привет как дела?... Херсон 343
пап привет как дела - Яндекс.Видео 446
привет 471
привет как дела клип - Яндекс.Видео 482
привет братан как дела - Яндекс.Видео 506
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 579
http://metric.ru/ 778
http://metris.ru/ 778
http://autometric.ru/ 818
http://metrica.yandex.com/ 852
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
http://metric.ru/ 667
http://metrica.yandex.com/ 704
http://autometric.ru/ 727
http://metris.ru/ 778
http://metrika.ru/ 789
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
http://metrika.ru/ 684
http://metric.ru/ 778
http://metris.ru/ 778
http://autometric.ru/ 818
http://metrica.yandex.com/ 852
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
http://metric.ru/ 667
http://autometric.ru/ 727
http://metrica.yandex.com/ 778
http://metris.ru/ 778
http://metrika.ru/ 789
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
http://metrica.yandex.com/ 769
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
0
0
0
@ -769,87 +769,87 @@ http://metrika.ru/ 1000
пап привет как дела - Яндекс.Видео 928
привет как дела клип - Яндекс.Видео 929
привет братан как дела - Яндекс.Видео 955
привет 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
привет 1000
привет как дела?... Херсон 672
пап привет как дела - Яндекс.Видео 735
привет как дела клип - Яндекс.Видео 741
привет братан как дела - Яндекс.Видео 753
привет 1000
http://metric.ru/ 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
http://metris.ru/ 1000
привет 1000
http://metrika.ru/ 579
http://metric.ru/ 778
http://metris.ru/ 778
http://autometric.ru/ 818
http://metrica.yandex.com/ 852
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
http://metrika.ru/ 579
http://metric.ru/ 778
http://metris.ru/ 778
http://autometric.ru/ 818
http://metrica.yandex.com/ 852
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
http://metric.ru/ 667
http://metrica.yandex.com/ 704
http://autometric.ru/ 727
http://metris.ru/ 778
http://metrika.ru/ 789
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
http://metrika.ru/ 684
http://metric.ru/ 778
http://metris.ru/ 778
http://autometric.ru/ 818
http://metrica.yandex.com/ 852
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
http://metric.ru/ 667
http://autometric.ru/ 727
http://metrica.yandex.com/ 778
http://metris.ru/ 778
http://metrika.ru/ 789
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
http://metrica.yandex.com/ 769
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000

View File

@ -34,17 +34,17 @@ drop table if exists test_distance;
create table test_distance (Title String) engine = Memory;
insert into test_distance values ('привет как дела?... Херсон'), ('привет как дела клип - Яндекс.Видео'), ('привет'), ('пап привет как дела - Яндекс.Видео'), ('привет братан как дела - Яндекс.Видео'), ('http://metric.ru/'), ('http://autometric.ru/'), ('http://metrica.yandex.com/'), ('http://metris.ru/'), ('http://metrika.ru/'), ('');
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, Title) as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, extract(Title, 'как дела')) as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, extract(Title, 'metr')) as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, Title) as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, extract(Title, 'как дела')) as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, extract(Title, 'metr')) as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'привет как дела') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'как привет дела') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'metrika') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'metrica') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'metriks') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'metrics') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'yandex') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'привет как дела') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'как привет дела') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'metrika') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'metrica') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'metriks') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'metrics') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceUTF8(Title, 'yandex') as distance, Title;
select round(1000 * ngramDistanceCaseInsensitiveUTF8(materialize(''), '')) from system.numbers limit 5;
@ -80,19 +80,19 @@ select round(1000 * ngramDistanceCaseInsensitiveUTF8('аБВГдеёЖз', 'Аб
select round(1000 * ngramDistanceCaseInsensitiveUTF8('абвгдеёжз', 'гдеёЗД'));
select round(1000 * ngramDistanceCaseInsensitiveUTF8('АБВГДеёжз', 'ЁЁЁЁЁЁЁЁ'));
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, Title) as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, extract(Title, 'как дела')) as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, extract(Title, 'metr')) as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, Title) as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, extract(Title, 'как дела')) as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, extract(Title, 'metr')) as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'ПрИвЕт кАК ДЕЛа') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'как ПРИВЕТ дела') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'metrika') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'Metrika') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'mEtrica') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'metriKS') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'metrics') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'YanDEX') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'приВЕТ КАк ДеЛа КлИп - яндеКс.видео') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'ПрИвЕт кАК ДЕЛа') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'как ПРИВЕТ дела') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'metrika') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'Metrika') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'mEtrica') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'metriKS') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'metrics') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'YanDEX') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitiveUTF8(Title, 'приВЕТ КАк ДеЛа КлИп - яндеКс.видео') as distance, Title;
select round(1000 * ngramDistance(materialize(''), '')) from system.numbers limit 5;
@ -128,13 +128,13 @@ select round(1000 * ngramDistance('abcdefgh', 'abcdefg'));
select round(1000 * ngramDistance('abcdefgh', 'defgh'));
select round(1000 * ngramDistance('abcdefgh', 'aaaaaaaa'));
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(Title, 'привет как дела') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(Title, 'как привет дела') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(Title, 'metrika') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(Title, 'metrica') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(Title, 'metriks') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(Title, 'metrics') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(Title, 'yandex') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(Title, 'привет как дела') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(Title, 'как привет дела') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(Title, 'metrika') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(Title, 'metrica') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(Title, 'metriks') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(Title, 'metrics') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistance(Title, 'yandex') as distance, Title;
select round(1000 * ngramDistanceCaseInsensitive(materialize(''), '')) from system.numbers limit 5;
select round(1000 * ngramDistanceCaseInsensitive(materialize('abc'), '')) from system.numbers limit 5;
@ -168,13 +168,13 @@ select round(1000 * ngramDistanceCaseInsensitive('abcdefgh', 'abcdeFG'));
select round(1000 * ngramDistanceCaseInsensitive('AAAAbcdefgh', 'defgh'));
select round(1000 * ngramDistanceCaseInsensitive('ABCdefgH', 'aaaaaaaa'));
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'ПрИвЕт кАК ДЕЛа') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'как ПРИВЕТ дела') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'metrika') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'Metrika') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'mEtrica') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'metriKS') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'metrics') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'YanDEX') as distance;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'ПрИвЕт кАК ДЕЛа') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'как ПРИВЕТ дела') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'metrika') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'Metrika') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'mEtrica') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'metriKS') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'metrics') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_distance ORDER BY ngramDistanceCaseInsensitive(Title, 'YanDEX') as distance, Title;
drop table if exists test_distance;

View File

@ -13,7 +13,7 @@ insert into ttl values (toDateTime('2100-10-10 00:00:00'), 3);
insert into ttl values (toDateTime('2100-10-10 00:00:00'), 4);
optimize table ttl partition 10 final;
select * from ttl order by d;
select * from ttl order by d, a;
alter table ttl modify ttl a; -- { serverError 450 }

View File

@ -110,115 +110,115 @@
1000
500
0
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
1000
http://metric.ru/ 0
http://autometric.ru/ 0
http://metrica.yandex.com/ 0
http://metris.ru/ 0
http://metrika.ru/ 0
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
привет 308
привет братан как дела - Яндекс.Видео 923
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
пап привет как дела - Яндекс.Видео 1000
http://metric.ru/ 0
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metris.ru/ 0
http://metrika.ru/ 0
0
http://metris.ru/ 0
привет 308
привет как дела?... Херсон 769
привет как дела клип - Яндекс.Видео 769
привет братан как дела - Яндекс.Видео 769
привет как дела клип - Яндекс.Видео 769
привет как дела?... Херсон 769
пап привет как дела - Яндекс.Видео 846
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
0
http://metric.ru/ 600
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://autometric.ru/ 600
http://metric.ru/ 600
http://metrica.yandex.com/ 600
http://metris.ru/ 600
http://metrika.ru/ 1000
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
0
http://metris.ru/ 600
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://metrika.ru/ 600
http://metric.ru/ 800
http://metris.ru/ 600
http://autometric.ru/ 800
http://metric.ru/ 800
http://metrica.yandex.com/ 1000
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
0
http://metric.ru/ 600
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://autometric.ru/ 600
http://metric.ru/ 600
http://metrica.yandex.com/ 600
http://metris.ru/ 600
http://metrika.ru/ 800
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
0
http://metris.ru/ 600
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://metrika.ru/ 600
http://metric.ru/ 800
http://metris.ru/ 600
http://autometric.ru/ 800
http://metric.ru/ 800
http://metrica.yandex.com/ 800
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
http://metric.ru/ 0
http://autometric.ru/ 0
http://metris.ru/ 0
http://metrika.ru/ 0
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://metrica.yandex.com/ 1000
1000
1000
@ -332,133 +332,133 @@ http://metrica.yandex.com/ 1000
1000
500
0
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
1000
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
привет 1000
пап привет как дела - Яндекс.Видео 1000
привет братан как дела - Яндекс.Видео 1000
http://metric.ru/ 1000
http://autometric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metris.ru/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
1000
http://metric.ru/ 0
http://autometric.ru/ 0
http://metrica.yandex.com/ 0
http://metris.ru/ 0
http://metrika.ru/ 0
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
1000
http://autometric.ru/ 1000
http://metric.ru/ 1000
http://metrica.yandex.com/ 1000
http://metrika.ru/ 1000
http://metris.ru/ 1000
пап привет как дела - Яндекс.Видео 1000
привет 1000
привет братан как дела - Яндекс.Видео 1000
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
привет 308
привет братан как дела - Яндекс.Видео 923
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
пап привет как дела - Яндекс.Видео 1000
http://metric.ru/ 0
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metris.ru/ 0
http://metrika.ru/ 0
0
http://metris.ru/ 0
привет 308
привет как дела?... Херсон 769
привет как дела клип - Яндекс.Видео 769
привет братан как дела - Яндекс.Видео 769
привет как дела клип - Яндекс.Видео 769
привет как дела?... Херсон 769
пап привет как дела - Яндекс.Видео 846
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
0
http://metric.ru/ 600
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://autometric.ru/ 600
http://metric.ru/ 600
http://metrica.yandex.com/ 600
http://metris.ru/ 600
http://metrika.ru/ 1000
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
0
http://metric.ru/ 600
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://autometric.ru/ 600
http://metric.ru/ 600
http://metrica.yandex.com/ 600
http://metris.ru/ 600
http://metrika.ru/ 1000
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
0
http://metris.ru/ 600
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://metrika.ru/ 600
http://metric.ru/ 800
http://metris.ru/ 600
http://autometric.ru/ 800
http://metric.ru/ 800
http://metrica.yandex.com/ 1000
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
0
http://metric.ru/ 600
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://autometric.ru/ 600
http://metric.ru/ 600
http://metrica.yandex.com/ 600
http://metris.ru/ 600
http://metrika.ru/ 800
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
0
http://metris.ru/ 600
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://metrika.ru/ 600
http://metric.ru/ 800
http://metris.ru/ 600
http://autometric.ru/ 800
http://metric.ru/ 800
http://metrica.yandex.com/ 800
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
http://metric.ru/ 0
http://autometric.ru/ 0
http://metris.ru/ 0
http://metrika.ru/ 0
0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://metrica.yandex.com/ 1000
http://metric.ru/ 0
http://autometric.ru/ 0
http://metrica.yandex.com/ 0
http://metris.ru/ 0
http://metrika.ru/ 0
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
привет 121
привет как дела?... Херсон 394
привет братан как дела - Яндекс.Видео 788
@ -576,82 +576,82 @@ http://metrika.ru/ 0
1000
1000
1000
http://metric.ru/ 0
http://autometric.ru/ 0
http://metrica.yandex.com/ 0
http://metris.ru/ 0
http://metrika.ru/ 0
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
привет 360
привет братан как дела - Яндекс.Видео 960
привет как дела?... Херсон 1000
привет как дела клип - Яндекс.Видео 1000
пап привет как дела - Яндекс.Видео 1000
http://metric.ru/ 0
привет как дела клип - Яндекс.Видео 1000
привет как дела?... Херсон 1000
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metris.ru/ 0
http://metrika.ru/ 0
0
http://metris.ru/ 0
привет 360
привет как дела?... Херсон 880
привет как дела клип - Яндекс.Видео 880
привет братан как дела - Яндекс.Видео 880
привет как дела клип - Яндекс.Видео 880
привет как дела?... Херсон 880
пап привет как дела - Яндекс.Видео 920
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
0
http://metric.ru/ 500
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://autometric.ru/ 500
http://metric.ru/ 500
http://metrica.yandex.com/ 500
http://metris.ru/ 500
http://metrika.ru/ 1000
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
0
http://metris.ru/ 500
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://metrika.ru/ 500
http://metric.ru/ 750
http://metris.ru/ 500
http://autometric.ru/ 750
http://metric.ru/ 750
http://metrica.yandex.com/ 1000
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
0
http://metric.ru/ 500
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://autometric.ru/ 500
http://metric.ru/ 500
http://metrica.yandex.com/ 500
http://metris.ru/ 500
http://metrika.ru/ 750
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
0
http://metris.ru/ 500
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://metrika.ru/ 500
http://metric.ru/ 750
http://metris.ru/ 500
http://autometric.ru/ 750
http://metric.ru/ 750
http://metrica.yandex.com/ 750
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
http://metric.ru/ 0
http://autometric.ru/ 0
http://metris.ru/ 0
http://metrika.ru/ 0
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://metrica.yandex.com/ 1000
1000
1000
@ -765,91 +765,91 @@ http://metrica.yandex.com/ 1000
1000
1000
1000
привет 0
http://metric.ru/ 0
http://autometric.ru/ 0
http://metrica.yandex.com/ 0
http://metris.ru/ 0
http://metrika.ru/ 0
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
привет 0
привет братан как дела - Яндекс.Видео 80
привет как дела?... Херсон 120
привет как дела клип - Яндекс.Видео 120
пап привет как дела - Яндекс.Видео 120
привет 0
http://metric.ru/ 0
http://autometric.ru/ 0
http://metrica.yandex.com/ 0
http://metris.ru/ 0
http://metrika.ru/ 0
привет как дела клип - Яндекс.Видео 120
привет как дела?... Херсон 120
0
привет как дела?... Херсон 440
привет как дела клип - Яндекс.Видео 440
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrica.yandex.com/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
привет 0
пап привет как дела - Яндекс.Видео 440
привет братан как дела - Яндекс.Видео 440
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 440
привет как дела?... Херсон 440
0
http://metric.ru/ 500
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://autometric.ru/ 500
http://metric.ru/ 500
http://metrica.yandex.com/ 500
http://metris.ru/ 500
http://metrika.ru/ 1000
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
0
http://metric.ru/ 500
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://autometric.ru/ 500
http://metric.ru/ 500
http://metrica.yandex.com/ 500
http://metris.ru/ 500
http://metrika.ru/ 1000
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
0
http://metris.ru/ 500
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://metrika.ru/ 500
http://metric.ru/ 750
http://metris.ru/ 500
http://autometric.ru/ 750
http://metric.ru/ 750
http://metrica.yandex.com/ 1000
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
0
http://metric.ru/ 500
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://autometric.ru/ 500
http://metric.ru/ 500
http://metrica.yandex.com/ 500
http://metris.ru/ 500
http://metrika.ru/ 750
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
0
http://metris.ru/ 500
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://metrika.ru/ 500
http://metric.ru/ 750
http://metris.ru/ 500
http://autometric.ru/ 750
http://metric.ru/ 750
http://metrica.yandex.com/ 750
привет как дела?... Херсон 0
привет как дела клип - Яндекс.Видео 0
привет 0
пап привет как дела - Яндекс.Видео 0
привет братан как дела - Яндекс.Видео 0
http://metric.ru/ 0
http://autometric.ru/ 0
http://metris.ru/ 0
http://metrika.ru/ 0
0
http://autometric.ru/ 0
http://metric.ru/ 0
http://metrika.ru/ 0
http://metris.ru/ 0
пап привет как дела - Яндекс.Видео 0
привет 0
привет братан как дела - Яндекс.Видео 0
привет как дела клип - Яндекс.Видео 0
привет как дела?... Херсон 0
http://metrica.yandex.com/ 1000

View File

@ -34,17 +34,17 @@ drop table if exists test_entry_distance;
create table test_entry_distance (Title String) engine = Memory;
insert into test_entry_distance values ('привет как дела?... Херсон'), ('привет как дела клип - Яндекс.Видео'), ('привет'), ('пап привет как дела - Яндекс.Видео'), ('привет братан как дела - Яндекс.Видео'), ('http://metric.ru/'), ('http://autometric.ru/'), ('http://metrica.yandex.com/'), ('http://metris.ru/'), ('http://metrika.ru/'), ('');
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, Title) as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, extract(Title, 'как дела')) as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, extract(Title, 'metr')) as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, Title) as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, extract(Title, 'как дела')) as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, extract(Title, 'metr')) as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, 'привет как дела') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, 'как привет дела') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, 'metrika') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, 'metrica') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, 'metriks') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, 'metrics') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, 'yandex') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, 'привет как дела') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, 'как привет дела') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, 'metrika') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, 'metrica') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, 'metriks') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, 'metrics') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchUTF8(Title, 'yandex') as distance, Title;
select round(1000 * ngramSearchCaseInsensitiveUTF8(materialize(''), '')) from system.numbers limit 5;
@ -80,19 +80,19 @@ select round(1000 * ngramSearchCaseInsensitiveUTF8('аБВГдеёЖз', 'Абв
select round(1000 * ngramSearchCaseInsensitiveUTF8('абвгдеёжз', 'гдеёЗД'));
select round(1000 * ngramSearchCaseInsensitiveUTF8('АБВГДеёжз', 'ЁЁЁЁЁЁЁЁ'));
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, Title) as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, extract(Title, 'как дела')) as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, extract(Title, 'metr')) as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, Title) as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, extract(Title, 'как дела')) as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, extract(Title, 'metr')) as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'ПрИвЕт кАК ДЕЛа') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'как ПРИВЕТ дела') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'metrika') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'Metrika') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'mEtrica') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'metriKS') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'metrics') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'YanDEX') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'приВЕТ КАк ДеЛа КлИп - яндеКс.видео') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'ПрИвЕт кАК ДЕЛа') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'как ПРИВЕТ дела') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'metrika') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'Metrika') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'mEtrica') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'metriKS') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'metrics') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'YanDEX') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitiveUTF8(Title, 'приВЕТ КАк ДеЛа КлИп - яндеКс.видео') as distance, Title;
select round(1000 * ngramSearch(materialize(''), '')) from system.numbers limit 5;
@ -128,13 +128,13 @@ select round(1000 * ngramSearch('abcdefgh', 'abcdefg'));
select round(1000 * ngramSearch('abcdefgh', 'defgh'));
select round(1000 * ngramSearch('abcdefghaaaaaaaaaa', 'aaaaaaaa'));
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearch(Title, 'привет как дела') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearch(Title, 'как привет дела') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearch(Title, 'metrika') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearch(Title, 'metrica') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearch(Title, 'metriks') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearch(Title, 'metrics') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearch(Title, 'yandex') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearch(Title, 'привет как дела') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearch(Title, 'как привет дела') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearch(Title, 'metrika') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearch(Title, 'metrica') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearch(Title, 'metriks') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearch(Title, 'metrics') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearch(Title, 'yandex') as distance, Title;
select round(1000 * ngramSearchCaseInsensitive(materialize(''), '')) from system.numbers limit 5;
select round(1000 * ngramSearchCaseInsensitive(materialize('abc'), '')) from system.numbers limit 5;
@ -168,13 +168,13 @@ select round(1000 * ngramSearchCaseInsensitive('abcdefgh', 'abcdeFG'));
select round(1000 * ngramSearchCaseInsensitive('AAAAbcdefgh', 'defgh'));
select round(1000 * ngramSearchCaseInsensitive('ABCdefgHaAaaaAaaaAA', 'aaaaaaaa'));
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'ПрИвЕт кАК ДЕЛа') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'как ПРИВЕТ дела') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'metrika') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'Metrika') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'mEtrica') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'metriKS') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'metrics') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'YanDEX') as distance;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'ПрИвЕт кАК ДЕЛа') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'как ПРИВЕТ дела') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'metrika') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'Metrika') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'mEtrica') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'metriKS') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'metrics') as distance, Title;
SELECT Title, round(1000 * distance) FROM test_entry_distance ORDER BY ngramSearchCaseInsensitive(Title, 'YanDEX') as distance, Title;
drop table if exists test_entry_distance;

View File

@ -13,6 +13,6 @@ alter table ttl modify ttl d + interval 1 day;
select sleep(1) format Null; -- wait if very fast merge happen
optimize table ttl partition 10 final;
select * from ttl order by d;
select * from ttl order by d, a;
drop table if exists ttl;

View File

@ -6,10 +6,36 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g')
$CLICKHOUSE_CLIENT --query="SELECT * FROM (SELECT number % 5 AS a, count() AS b, c FROM numbers(10) ARRAY JOIN [1,2] AS c GROUP BY a,c) AS table ORDER BY a LIMIT 3 WITH TIES BY a" 2>&1 | grep -q "Code: 498." && echo 'OK' || echo 'FAIL' ||:
$CLICKHOUSE_CLIENT --query="""
SELECT * FROM (SELECT number % 5 AS a, count() AS b, c FROM numbers(10)
ARRAY JOIN [1,2] AS c GROUP BY a,c) AS table
ORDER BY a LIMIT 3 WITH TIES BY a""" 2>&1 | grep -q "Code: 498." && echo 'OK' || echo 'FAIL' ||:
$CLICKHOUSE_CLIENT --query="SELECT * FROM VALUES('Phrase String, Payload String', ('hello', 'x'), ('world', 'x'), ('hello', 'z'), ('upyachka', 'a'), ('test', 'b'), ('foo', 'c'), ('bar', 'd')) ORDER BY Payload LIMIT 1 WITH TIES BY Phrase LIMIT 5;" 2>&1 | grep -q "Code: 498." && echo 'OK' || echo 'FAIL' ||:
$CLICKHOUSE_CLIENT --query="""
SELECT * FROM VALUES('Phrase String, Payload String',
('hello', 'x'), ('world', 'x'), ('hello', 'z'),
('upyachka', 'a'), ('test', 'b'), ('foo', 'c'),
('bar', 'd'))
ORDER BY Payload LIMIT 1 WITH TIES BY Phrase LIMIT 5;""" 2>&1 | grep -q "Code: 498." && echo 'OK' || echo 'FAIL' ||:
$CLICKHOUSE_CLIENT --query="SELECT * FROM VALUES('Phrase String, Payload String', ('hello', 'x'), ('world', 'x'), ('hello', 'z'), ('upyachka', 'a'), ('test', 'b'), ('foo', 'c'), ('bar', 'd')) ORDER BY Payload LIMIT 1 BY Phrase LIMIT 5 WITH TIES"
$CLICKHOUSE_CLIENT --query="""
SELECT * FROM
(
SELECT * FROM VALUES('Phrase String, Payload String',
('hello', 'x'), ('world', 'x'), ('hello', 'z'),
('upyachka', 'a'), ('test', 'b'), ('foo', 'c'),
('bar', 'd'))
ORDER BY Payload LIMIT 1 BY Phrase LIMIT 5 WITH TIES
) ORDER BY Payload, Phrase
"""
$CLICKHOUSE_CLIENT --query="SELECT TOP 5 WITH TIES * FROM VALUES('Phrase String, Payload String', ('hello', 'x'), ('world', 'x'), ('hello', 'z'), ('upyachka', 'a'), ('test', 'b'), ('foo', 'c'), ('bar', 'd')) ORDER BY Payload LIMIT 1 BY Phrase"
$CLICKHOUSE_CLIENT --query="""
SELECT * FROM
(
SELECT TOP 5 WITH TIES * FROM VALUES('Phrase String, Payload String',
('hello', 'x'), ('world', 'x'), ('hello', 'z'),
('upyachka', 'a'), ('test', 'b'), ('foo', 'c'),
('bar', 'd'))
ORDER BY Payload LIMIT 1 BY Phrase
) ORDER BY Payload, Payload
"""

View File

@ -7,10 +7,7 @@ any left
any left (rev)
0 5 b6
2 a3 2 b1
2 a3 2 b2
4 a5 4 b3
4 a5 4 b4
4 a5 4 b5
any inner
2 a3 2 b1
4 a5 4 b3
@ -20,10 +17,7 @@ any inner (rev)
any right
0 5 b6
2 a3 2 b1
2 a3 2 b2
4 a5 4 b3
4 a5 4 b4
4 a5 4 b5
any right (rev)
0 a1 0
1 a2 0

View File

@ -5,7 +5,7 @@ CREATE TABLE t1 (x UInt32, s String) engine = Memory;
CREATE TABLE t2 (x UInt32, s String) engine = Memory;
INSERT INTO t1 (x, s) VALUES (0, 'a1'), (1, 'a2'), (2, 'a3'), (3, 'a4'), (4, 'a5');
INSERT INTO t2 (x, s) VALUES (2, 'b1'), (2, 'b2'), (4, 'b3'), (4, 'b4'), (4, 'b5'), (5, 'b6');
INSERT INTO t2 (x, s) VALUES (2, 'b1'), (4, 'b3'), (5, 'b6');
SET join_use_nulls = 0;
SET any_join_distinct_right_table_keys = 0;

View File

@ -3,45 +3,36 @@ any left
1 a2 0
2 a3 2 b1
3 a4 0
4 a5 4 b3
4 a5 4 b2
any left (rev)
0 5 b6
0 5 b4
2 a3 2 b1
2 a3 2 b2
4 a5 4 b3
4 a5 4 b4
4 a5 4 b5
4 a5 4 b2
any inner
2 a3 2 b1
4 a5 4 b3
4 a5 4 b2
any inner (rev)
2 a3 2 b1
4 a5 4 b3
4 a5 4 b2
any right
0 5 b6
0 5 b4
2 a3 2 b1
2 a3 2 b2
4 a5 4 b3
4 a5 4 b4
4 a5 4 b5
4 a5 4 b2
any right (rev)
0 a1 0
1 a2 0
2 a3 2 b1
3 a4 0
4 a5 4 b3
4 a5 4 b2
semi left
2 a3 2 b1
4 a5 4 b3
4 a5 4 b2
semi right
2 a3 2 b1
2 a3 2 b2
4 a5 4 b3
4 a5 4 b4
4 a5 4 b5
4 a5 4 b2
anti left
0 a1 0
1 a2 1
3 a4 3
anti right
0 5 b6
0 5 b4

View File

@ -5,7 +5,7 @@ CREATE TABLE t1 (x UInt32, s String) engine = Memory;
CREATE TABLE t2 (x UInt32, s String) engine = Memory;
INSERT INTO t1 (x, s) VALUES (0, 'a1'), (1, 'a2'), (2, 'a3'), (3, 'a4'), (4, 'a5');
INSERT INTO t2 (x, s) VALUES (2, 'b1'), (2, 'b2'), (4, 'b3'), (4, 'b4'), (4, 'b5'), (5, 'b6');
INSERT INTO t2 (x, s) VALUES (2, 'b1'), (4, 'b2'), (5, 'b4');
SET join_algorithm = 'prefer_partial_merge';
SET join_use_nulls = 0;

View File

@ -1,12 +1,12 @@
select number, intDiv(number,5) value from numbers(20) order by value limit 3 with ties;
select * from (select number, intDiv(number,5) value from numbers(20) order by value limit 3 with ties) ORDER BY number, value;
drop table if exists wt;
create table wt (a Int, b Int) engine = Memory;
insert into wt select 0, number from numbers(5);
select 1 from wt order by a limit 3 with ties;
select b from wt order by a limit 3 with ties;
with a * 2 as c select a, b from wt order by c limit 3 with ties;
select a * 2 as c, b from wt order by c limit 3 with ties;
select b from (select b from wt order by a limit 3 with ties) order by b;
select * from (select * from (with a * 2 as c select a, b from wt order by c limit 3 with ties) order by a, b);
select * from (select * from (select a * 2 as c, b from wt order by c limit 3 with ties) order by c, b);
drop table if exists wt;

View File

@ -11,8 +11,8 @@
2
1 3 3
1 4 3
2 5 4
2 2 4
2 5 4
2
1 4 3
1 3 3
@ -27,6 +27,6 @@
2
1 3 3
1 4 3
2 5 4
2 2 4
2 5 4
2

View File

@ -7,7 +7,7 @@ SELECT * FROM test ORDER BY toFloat32(x), -y, -z DESC;
SELECT * FROM test ORDER BY toFloat32(x), -(-y), -z DESC;
SELECT max(x) as k FROM test ORDER BY k;
SELECT roundToExp2(x) as k FROM test GROUP BY k ORDER BY k;
SELECT roundToExp2(x) as k, y, z FROM test WHERE k >= 1 ORDER BY k;
SELECT roundToExp2(x) as k, y, z FROM test WHERE k >= 1 ORDER BY k, y, z;
SELECT max(x) as k FROM test HAVING k > 0 ORDER BY k;
SET optimize_monotonous_functions_in_order_by = 0;
@ -15,7 +15,7 @@ SELECT * FROM test ORDER BY toFloat32(x), -y, -z DESC;
SELECT * FROM test ORDER BY toFloat32(x), -(-y), -z DESC;
SELECT max(x) as k FROM test ORDER BY k;
SELECT roundToExp2(x) as k From test GROUP BY k ORDER BY k;
SELECT roundToExp2(x) as k, y, z FROM test WHERE k >= 1 ORDER BY k;
SELECT roundToExp2(x) as k, y, z FROM test WHERE k >= 1 ORDER BY k, y, z;
SELECT max(x) as k FROM test HAVING k > 0 ORDER BY k;
DROP TABLE test;

View File

@ -5,7 +5,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
opts=(
--input-format CSV
-q 'SELECT number FROM numbers(2)'
)

View File

@ -6,5 +6,5 @@
3 3
1 1
2 1
3 4
3 3
3 4

View File

@ -4,5 +4,5 @@ DROP TABLE IF EXISTS test_fetch;
CREATE TABLE test_fetch(a Int32, b Int32) Engine = Memory;
INSERT INTO test_fetch VALUES(1, 1), (2, 1), (3, 4), (3, 3), (5, 4), (0, 6), (5, 7);
SELECT * FROM test_fetch ORDER BY a OFFSET 1 ROW FETCH FIRST 3 ROWS ONLY;
SELECT * FROM test_fetch ORDER BY a OFFSET 1 ROW FETCH FIRST 3 ROWS WITH TIES;
SELECT * FROM (SELECT * FROM test_fetch ORDER BY a OFFSET 1 ROW FETCH FIRST 3 ROWS WITH TIES) ORDER BY a, b;
DROP TABLE test_fetch;

View File

@ -7,9 +7,9 @@ CREATE TABLE test_collate (x UInt32, s Nullable(String)) ENGINE=Memory();
INSERT INTO test_collate VALUES (1, 'Ё'), (1, 'ё'), (1, 'а'), (1, null), (2, 'А'), (2, 'я'), (2, 'Я'), (2, null);
SELECT 'Order by without collate';
SELECT * FROM test_collate ORDER BY s;
SELECT * FROM test_collate ORDER BY s, x;
SELECT 'Order by with collate';
SELECT * FROM test_collate ORDER BY s COLLATE 'ru';
SELECT * FROM test_collate ORDER BY s COLLATE 'ru', x;
SELECT 'Order by tuple without collate';
SELECT * FROM test_collate ORDER BY x, s;

View File

@ -39,7 +39,7 @@ select number, avg(number) over (order by number rows unbounded preceding) from
8 4
9 4.5
-- no order by
select number, quantileExact(number) over (partition by intDiv(number, 3) rows unbounded preceding) from numbers(10);
select number, quantileExact(number) over (partition by intDiv(number, 3) AS value order by value rows unbounded preceding) from numbers(10);
0 0
1 1
2 1
@ -51,7 +51,7 @@ select number, quantileExact(number) over (partition by intDiv(number, 3) rows u
8 7
9 9
-- can add an alias after window spec
select number, quantileExact(number) over (partition by intDiv(number, 3) rows unbounded preceding) q from numbers(10);
select number, quantileExact(number) over (partition by intDiv(number, 3) AS value order by value rows unbounded preceding) q from numbers(10);
0 0
1 1
2 1
@ -198,7 +198,7 @@ select sum(number) over w1, sum(number) over w2
from numbers(10)
window
w1 as (rows unbounded preceding),
w2 as (partition by intDiv(number, 3) rows unbounded preceding)
w2 as (partition by intDiv(number, 3) as value order by value rows unbounded preceding)
;
0 0
1 1
@ -214,7 +214,7 @@ window
-- EXPLAIN test for this.
select
sum(number) over w1,
sum(number) over (partition by intDiv(number, 3) rows unbounded preceding)
sum(number) over (partition by intDiv(number, 3) as value order by value rows unbounded preceding)
from numbers(10)
window
w1 as (partition by intDiv(number, 3) rows unbounded preceding)

View File

@ -13,10 +13,10 @@ select number, abs(number) over (partition by toString(intDiv(number, 3)) rows u
select number, avg(number) over (order by number rows unbounded preceding) from numbers(10);
-- no order by
select number, quantileExact(number) over (partition by intDiv(number, 3) rows unbounded preceding) from numbers(10);
select number, quantileExact(number) over (partition by intDiv(number, 3) AS value order by value rows unbounded preceding) from numbers(10);
-- can add an alias after window spec
select number, quantileExact(number) over (partition by intDiv(number, 3) rows unbounded preceding) q from numbers(10);
select number, quantileExact(number) over (partition by intDiv(number, 3) AS value order by value rows unbounded preceding) q from numbers(10);
-- can't reference it yet -- the window functions are calculated at the
-- last stage of select, after all other functions.
@ -81,14 +81,14 @@ select sum(number) over w1, sum(number) over w2
from numbers(10)
window
w1 as (rows unbounded preceding),
w2 as (partition by intDiv(number, 3) rows unbounded preceding)
w2 as (partition by intDiv(number, 3) as value order by value rows unbounded preceding)
;
-- FIXME both functions should use the same window, but they don't. Add an
-- EXPLAIN test for this.
select
sum(number) over w1,
sum(number) over (partition by intDiv(number, 3) rows unbounded preceding)
sum(number) over (partition by intDiv(number, 3) as value order by value rows unbounded preceding)
from numbers(10)
window
w1 as (partition by intDiv(number, 3) rows unbounded preceding)

View File

@ -43,7 +43,7 @@ SELECT
product_name,
group_name,
price,
row_number() OVER (PARTITION BY group_name ORDER BY price desc) rn
row_number() OVER (PARTITION BY group_name ORDER BY price desc, product_name asc) rn
FROM products INNER JOIN product_groups USING (group_id)
ORDER BY group_name, rn;

View File

@ -38,6 +38,8 @@ FROM
SELECT *
FROM neighbor_test
ORDER BY val_string ASC
) format PrettyCompact;
)
ORDER BY rowNr, val_string, str_m1, str_p1, val_low, low_m1, low_p1
format PrettyCompact;
drop table if exists neighbor_test;

View File

@ -9,7 +9,7 @@ INSERT INTO table2 VALUES ('D', 'd', '2018-01-01') ('B', 'b', '2018-01-01') ('C'
set join_algorithm = 'partial_merge';
SELECT * FROM table1 AS t1 ALL LEFT JOIN (SELECT *, '0.10', c, d AS b FROM table2) AS t2 USING (a, b) ORDER BY d ASC FORMAT PrettyCompact settings max_rows_in_join = 1;
SELECT * FROM table1 AS t1 ALL LEFT JOIN (SELECT *, '0.10', c, d AS b FROM table2) AS t2 USING (a, b) ORDER BY d, t1.a ASC FORMAT PrettyCompact settings max_rows_in_join = 1;
SELECT pow('0.0000000257', NULL), pow(pow(NULL, NULL), NULL) - NULL, (val + NULL) = (rval * 0), * FROM (SELECT (val + 256) = (NULL * NULL), toLowCardinality(toNullable(dummy)) AS val FROM system.one) AS s1 ANY LEFT JOIN (SELECT toLowCardinality(dummy) AS rval FROM system.one) AS s2 ON (val + 0) = (rval * 255) settings max_rows_in_join = 1;

View File

@ -3,8 +3,42 @@ select deltaSum(arrayJoin([1, 2, 3, 0, 3, 4]));
select deltaSum(arrayJoin([1, 2, 3, 0, 3, 4, 2, 3]));
select deltaSum(arrayJoin([1, 2, 3, 0, 3, 3, 3, 3, 3, 4, 2, 3]));
select deltaSum(arrayJoin([1, 2, 3, 0, 0, 0, 0, 3, 3, 3, 3, 3, 4, 2, 3]));
select deltaSumMerge(rows) from (select deltaSumState(arrayJoin([0, 1])) as rows union all select deltaSumState(arrayJoin([4, 5])) as rows);
select deltaSumMerge(rows) from (select deltaSumState(arrayJoin([4, 5])) as rows union all select deltaSumState(arrayJoin([0, 1])) as rows);
select deltaSumMerge(rows) from
(
select * from
(
select deltaSumState(arrayJoin([0, 1])) as rows
union all
select deltaSumState(arrayJoin([4, 5])) as rows
) order by rows
);
select deltaSumMerge(rows) from
(
select * from
(
select deltaSumState(arrayJoin([4, 5])) as rows
union all
select deltaSumState(arrayJoin([0, 1])) as rows
) order by rows
);
select deltaSum(arrayJoin([2.25, 3, 4.5]));
select deltaSumMerge(rows) from (select deltaSumState(arrayJoin([0.1, 0.3, 0.5])) as rows union all select deltaSumState(arrayJoin([4.1, 5.1, 6.6])) as rows);
select deltaSumMerge(rows) from (select deltaSumState(arrayJoin([3, 5])) as rows union all select deltaSumState(arrayJoin([1, 2])) as rows union all select deltaSumState(arrayJoin([4, 6])) as rows);
select deltaSumMerge(rows) from
(
select * from
(
select deltaSumState(arrayJoin([0.1, 0.3, 0.5])) as rows
union all
select deltaSumState(arrayJoin([4.1, 5.1, 6.6])) as rows
) order by rows
);
select deltaSumMerge(rows) from
(
select * from
(
select deltaSumState(arrayJoin([3, 5])) as rows
union all
select deltaSumState(arrayJoin([1, 2])) as rows
union all
select deltaSumState(arrayJoin([4, 6])) as rows
) order by rows
);

View File

@ -142,7 +142,7 @@ CREATE TABLE t_sparse_1 (id UInt64, v Int64)
ENGINE = MergeTree ORDER BY tuple()
SETTINGS ratio_of_defaults_for_sparse_serialization = 0;
INSERT INTO t_sparse_1 VALUES (1, 6), (2, 1), (3, 0), (4, -1), (5, 0), (6, 0), (7, -2), (8, 0), (9, 0), (10, 4), (11, 0);
SELECT * FROM t_sparse_1 ORDER BY v;
SELECT * FROM t_sparse_1 ORDER BY v, id;
7 -2
4 -1
3 0
@ -154,7 +154,7 @@ SELECT * FROM t_sparse_1 ORDER BY v;
2 1
10 4
1 6
SELECT * FROM t_sparse_1 ORDER BY v DESC;
SELECT * FROM t_sparse_1 ORDER BY v DESC, id;
1 6
10 4
2 1

View File

@ -35,8 +35,8 @@ SETTINGS ratio_of_defaults_for_sparse_serialization = 0;
INSERT INTO t_sparse_1 VALUES (1, 6), (2, 1), (3, 0), (4, -1), (5, 0), (6, 0), (7, -2), (8, 0), (9, 0), (10, 4), (11, 0);
SELECT * FROM t_sparse_1 ORDER BY v;
SELECT * FROM t_sparse_1 ORDER BY v DESC;
SELECT * FROM t_sparse_1 ORDER BY v, id;
SELECT * FROM t_sparse_1 ORDER BY v DESC, id;
SELECT * FROM t_sparse_1 ORDER BY v, id LIMIT 5;
SELECT * FROM t_sparse_1 ORDER BY v DESC, id LIMIT 5;

View File

@ -1,6 +1,6 @@
id Default
u Sparse
s Sparse
u Sparse
182 155
id Default
t Sparse

View File

@ -12,14 +12,14 @@ INSERT INTO t_sparse_alter SELECT
if (number % 13 = 0, toString(number), '')
FROM numbers(2000);
SELECT column, serialization_kind FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_sparse_alter' AND active ORDER BY name;
SELECT column, serialization_kind FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_sparse_alter' AND active ORDER BY column;
SELECT uniqExact(u), uniqExact(s) FROM t_sparse_alter;
ALTER TABLE t_sparse_alter DROP COLUMN s, RENAME COLUMN u TO t;
ALTER TABLE t_sparse_alter MODIFY COLUMN t UInt16;
SELECT column, serialization_kind FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_sparse_alter' AND active ORDER BY name;
SELECT column, serialization_kind FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_sparse_alter' AND active ORDER BY column;
SELECT uniqExact(t) FROM t_sparse_alter;

View File

@ -6,7 +6,7 @@ CREATE TABLE map_comb(a int, statusMap Map(UInt16, UInt32)) ENGINE = Log;
INSERT INTO map_comb VALUES (1, map(1, 10, 2, 10, 3, 10)),(1, map(3, 10, 4, 10, 5, 10)),(2, map(4, 10, 5, 10, 6, 10)),(2, map(6, 10, 7, 10, 8, 10)),(3, map(1, 10, 2, 10, 3, 10)),(4, map(3, 10, 4, 10, 5, 10)),(5, map(4, 10, 5, 10, 6, 10)),(5, map(6, 10, 7, 10, 8, 10));
SELECT * FROM map_comb ORDER BY a;
SELECT * FROM map_comb ORDER BY a, statusMap;
SELECT toTypeName(res), sumMap(statusMap) as res FROM map_comb;
SELECT toTypeName(res), sumWithOverflowMap(statusMap) as res FROM map_comb;
SELECT toTypeName(res), sumMapMerge(s) as res FROM (SELECT sumMapState(statusMap) AS s FROM map_comb);

View File

@ -15,29 +15,23 @@ select x3, x2, x1 from test order by x3 desc;
100 10 1
10 1 10
1 100 100
insert into test values (1, 10, 200), (10, 1, 200), (100, 100, 1);
insert into test values (1, 10, 100), (10, 1, 10), (100, 100, 1);
select x3, x2 from test group by x3, x2;
200 1
10 1
200 10
1 100
100 10
select x3, x2 from test group by 1, 2;
200 1
10 1
200 10
1 100
100 10
select x1, x2, x3 from test order by x3 limit 1 by x3;
100 100 1
10 1 10
1 10 100
1 10 200
select x1, x2, x3 from test order by 3 limit 1 by 3;
100 100 1
10 1 10
1 10 100
1 10 200
select x1, x2, x3 from test order by x3 limit 1 by x1;
100 100 1
10 1 10

View File

@ -13,7 +13,7 @@ select x3, x2, x1 from test order by x3;
select x3, x2, x1 from test order by 1 desc;
select x3, x2, x1 from test order by x3 desc;
insert into test values (1, 10, 200), (10, 1, 200), (100, 100, 1);
insert into test values (1, 10, 100), (10, 1, 10), (100, 100, 1);
select x3, x2 from test group by x3, x2;
select x3, x2 from test group by 1, 2;

View File

@ -2,20 +2,20 @@ SELECT
1 AS max_size,
groupArray(max_size)(col)
FROM
(
(SELECT col FROM (
SELECT 1 AS col
UNION ALL
SELECT 2
);
) ORDER BY col);
WITH 1 AS max_size
SELECT groupArray(max_size)(col)
FROM
(
(SELECT col FROM (
SELECT 1 as col
UNION ALL
SELECT 2
);
) ORDER BY col);
WITH 0.1 AS level
SELECT quantile(level)(number)

File diff suppressed because one or more lines are too long

View File

@ -2,10 +2,10 @@ Tuple(UInt64, UInt64) (9007199254740994,3)
Tuple(UInt64, UInt64) (9007199254740994,3)
Tuple(UInt64, UInt64) (9007199254740994,3)
Tuple(UInt64, UInt64) (9007199254740994,3)
Tuple(Float64, UInt64) (9007199254740992,3)
Tuple(Float64, UInt64) (9007199254740992,3)
Tuple(Float64, UInt64) (9007199254740992,3)
Tuple(Float64, UInt64) (9007199254740992,3)
Tuple(Float64, UInt64) (9007199254740994,3)
Tuple(Float64, UInt64) (9007199254740994,3)
Tuple(Float64, UInt64) (9007199254740994,3)
Tuple(Float64, UInt64) (9007199254740994,3)
Tuple(Float64, UInt64) (16777218,3)
Tuple(Float64, UInt64) (16777218,3)
Tuple(Float64, UInt64) (16777218,3)

View File

@ -1,20 +1,150 @@
-- Integer types are added as integers
SELECT toTypeName(sumCount(v)), sumCount(v) FROM (SELECT '9007199254740992'::UInt64 AS v UNION ALL SELECT '1'::UInt64 AS v UNION ALL SELECT '1'::UInt64 AS v);
SELECT toTypeName(sumCount(v)), sumCount(v) FROM (SELECT '9007199254740992'::Nullable(UInt64) AS v UNION ALL SELECT '1'::Nullable(UInt64) AS v UNION ALL SELECT '1'::Nullable(UInt64) AS v );
SELECT toTypeName(sumCount(v)), sumCount(v) FROM (SELECT '9007199254740992'::LowCardinality(UInt64) AS v UNION ALL SELECT '1'::LowCardinality(UInt64) AS v UNION ALL SELECT '1'::LowCardinality(UInt64) AS v );
SELECT toTypeName(sumCount(v)), sumCount(v) FROM (SELECT '9007199254740992'::LowCardinality(Nullable(UInt64)) AS v UNION ALL SELECT '1'::LowCardinality(Nullable(UInt64)) AS v UNION ALL SELECT '1'::LowCardinality(Nullable(UInt64)) AS v );
SELECT toTypeName(sumCount(v)), sumCount(v) FROM
(
SELECT v FROM
(
SELECT '9007199254740992'::UInt64 AS v
UNION ALL
SELECT '1'::UInt64 AS v
UNION ALL SELECT '1'::UInt64 AS v
)
ORDER BY v
);
SELECT toTypeName(sumCount(v)), sumCount(v) FROM
(
SELECT v FROM
(
SELECT '9007199254740992'::Nullable(UInt64) AS v
UNION ALL
SELECT '1'::Nullable(UInt64) AS v
UNION ALL
SELECT '1'::Nullable(UInt64) AS v
)
ORDER BY v
);
SELECT toTypeName(sumCount(v)), sumCount(v) FROM
(
SELECT v FROM
(
SELECT '9007199254740992'::LowCardinality(UInt64) AS v
UNION ALL
SELECT '1'::LowCardinality(UInt64) AS v
UNION ALL
SELECT '1'::LowCardinality(UInt64) AS v
)
ORDER BY v
);
SELECT toTypeName(sumCount(v)), sumCount(v) FROM
(
SELECT v FROM
(
SELECT '9007199254740992'::LowCardinality(Nullable(UInt64)) AS v
UNION ALL
SELECT '1'::LowCardinality(Nullable(UInt64)) AS v
UNION ALL
SELECT '1'::LowCardinality(Nullable(UInt64)) AS v
)
ORDER BY v
);
-- Float64 types are added as Float64
SELECT toTypeName(sumCount(v)), sumCount(v) FROM (SELECT '9007199254740992'::Float64 AS v UNION ALL SELECT '1'::Float64 AS v UNION ALL SELECT '1'::Float64 AS v );
SELECT toTypeName(sumCount(v)), sumCount(v) FROM (SELECT '9007199254740992'::Nullable(Float64) AS v UNION ALL SELECT '1'::Nullable(Float64) AS v UNION ALL SELECT '1'::Nullable(Float64) AS v );
SELECT toTypeName(sumCount(v)), sumCount(v) FROM (SELECT '9007199254740992'::LowCardinality(Float64) AS v UNION ALL SELECT '1'::LowCardinality(Float64) AS v UNION ALL SELECT '1'::LowCardinality(Float64) AS v);
SELECT toTypeName(sumCount(v)), sumCount(v) FROM (SELECT '9007199254740992'::LowCardinality(Nullable(Float64)) AS v UNION ALL SELECT '1'::LowCardinality(Nullable(Float64)) AS v UNION ALL SELECT '1'::LowCardinality(Nullable(Float64)) AS v );
-- -- Float64 types are added as Float64
SELECT toTypeName(sumCount(v)), sumCount(v) FROM
(
SELECT v FROM
(
SELECT '9007199254740992'::Float64 AS v
UNION ALL
SELECT '1'::Float64 AS v
UNION ALL SELECT '1'::Float64 AS v
)
ORDER BY v
);
SELECT toTypeName(sumCount(v)), sumCount(v) FROM
(
SELECT v FROM
(
SELECT '9007199254740992'::Nullable(Float64) AS v
UNION ALL
SELECT '1'::Nullable(Float64) AS v
UNION ALL
SELECT '1'::Nullable(Float64) AS v
)
ORDER BY v
);
SELECT toTypeName(sumCount(v)), sumCount(v) FROM
(
SELECT v FROM
(
SELECT '9007199254740992'::LowCardinality(Float64) AS v
UNION ALL
SELECT '1'::LowCardinality(Float64) AS v
UNION ALL
SELECT '1'::LowCardinality(Float64) AS v
)
ORDER BY v
);
SELECT toTypeName(sumCount(v)), sumCount(v) FROM
(
SELECT v FROM
(
SELECT '9007199254740992'::LowCardinality(Nullable(Float64)) AS v
UNION ALL
SELECT '1'::LowCardinality(Nullable(Float64)) AS v
UNION ALL
SELECT '1'::LowCardinality(Nullable(Float64)) AS v
)
ORDER BY v
);
-- Float32 are added as Float64
SELECT toTypeName(sumCount(v)), sumCount(v) FROM (SELECT '16777216'::Float32 AS v UNION ALL SELECT '1'::Float32 AS v UNION ALL SELECT '1'::Float32 AS v );
SELECT toTypeName(sumCount(v)), sumCount(v) FROM (SELECT '16777216'::Nullable(Float32) AS v UNION ALL SELECT '1'::Nullable(Float32) AS v UNION ALL SELECT '1'::Nullable(Float32) AS v );
SELECT toTypeName(sumCount(v)), sumCount(v) FROM (SELECT '16777216'::LowCardinality(Float32) AS v UNION ALL SELECT '1'::LowCardinality(Float32) AS v UNION ALL SELECT '1'::LowCardinality(Float32) AS v );
SELECT toTypeName(sumCount(v)), sumCount(v) FROM (SELECT '16777216'::LowCardinality(Nullable(Float32)) AS v UNION ALL SELECT '1'::LowCardinality(Nullable(Float32)) AS v UNION ALL SELECT '1'::LowCardinality(Nullable(Float32)) AS v );
-- -- Float32 are added as Float64
SELECT toTypeName(sumCount(v)), sumCount(v) FROM
(
SELECT v FROM
(
SELECT '16777216'::Float32 AS v
UNION ALL
SELECT '1'::Float32 AS v
UNION ALL
SELECT '1'::Float32 AS v
)
ORDER BY v
);
SELECT toTypeName(sumCount(v)), sumCount(v) FROM
(
SELECT v FROM
(
SELECT '16777216'::Nullable(Float32) AS v
UNION ALL
SELECT '1'::Nullable(Float32) AS v
UNION ALL
SELECT '1'::Nullable(Float32) AS v
)
ORDER BY v
);
SELECT toTypeName(sumCount(v)), sumCount(v) FROM
(
SELECT v FROM
(
SELECT '16777216'::LowCardinality(Float32) AS v
UNION ALL
SELECT '1'::LowCardinality(Float32) AS v
UNION ALL
SELECT '1'::LowCardinality(Float32) AS v
)
ORDER BY v
);
SELECT toTypeName(sumCount(v)), sumCount(v) FROM
(
SELECT v FROM
(
SELECT '16777216'::LowCardinality(Nullable(Float32)) AS v
UNION ALL
SELECT '1'::LowCardinality(Nullable(Float32)) AS v
UNION ALL
SELECT '1'::LowCardinality(Nullable(Float32)) AS v
)
ORDER BY v
);
-- Small integer types use their sign/unsigned 64 byte supertype
SELECT toTypeName(sumCount(number::Int8)), sumCount(number::Int8) FROM numbers(120);

View File

@ -8,3 +8,4 @@
7
8
9
1 String [1,2,3]

View File

@ -9,3 +9,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT -q "select * from numbers(10) format Parquet" > $CLICKHOUSE_TMP/data.parquet
$CLICKHOUSE_LOCAL -q "select * from table" --file="-" < $CLICKHOUSE_TMP/data.parquet
echo "1,\"String\", \"[1, 2, 3]\"" | $CLICKHOUSE_LOCAL -q "select * from table" --input-format=CSV