mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 10:02:01 +00:00
Merge branch 'master' into rs/show-indexes
This commit is contained in:
commit
6c28966324
@ -109,7 +109,7 @@ For the query to run successfully, the following conditions must be met:
|
||||
|
||||
- Both tables must have the same structure.
|
||||
- Both tables must have the same partition key, the same order by key and the same primary key.
|
||||
- Both tables must have the same storage policy (a disk where the partition is stored should be available for both tables).
|
||||
- Both tables must have the same storage policy.
|
||||
|
||||
## REPLACE PARTITION
|
||||
|
||||
@ -123,7 +123,7 @@ For the query to run successfully, the following conditions must be met:
|
||||
|
||||
- Both tables must have the same structure.
|
||||
- Both tables must have the same partition key, the same order by key and the same primary key.
|
||||
- Both tables must have the same storage policy (a disk where the partition is stored should be available for both tables).
|
||||
- Both tables must have the same storage policy.
|
||||
|
||||
## MOVE PARTITION TO TABLE
|
||||
|
||||
@ -137,7 +137,7 @@ For the query to run successfully, the following conditions must be met:
|
||||
|
||||
- Both tables must have the same structure.
|
||||
- Both tables must have the same partition key, the same order by key and the same primary key.
|
||||
- Both tables must have the same storage policy (a disk where the partition is stored should be available for both tables).
|
||||
- Both tables must have the same storage policy.
|
||||
- Both tables must be the same engine family (replicated or non-replicated).
|
||||
|
||||
## CLEAR COLUMN IN PARTITION
|
||||
|
@ -331,7 +331,7 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder
|
||||
all_replicas_count = cluster->getShardsInfo().size();
|
||||
}
|
||||
|
||||
/// Find local shard
|
||||
/// Find local shard. It might happen that there is no local shard, but that's fine
|
||||
for (const auto & shard: cluster->getShardsInfo())
|
||||
{
|
||||
if (shard.isLocal())
|
||||
@ -346,9 +346,6 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder
|
||||
}
|
||||
}
|
||||
|
||||
if (pipes.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "No local shard");
|
||||
|
||||
auto current_shard = cluster->getShardsInfo().begin();
|
||||
while (pipes.size() != all_replicas_count)
|
||||
{
|
||||
|
@ -84,7 +84,15 @@ namespace
|
||||
}
|
||||
void operator() (const UUID & x) const
|
||||
{
|
||||
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
|
||||
auto tmp_x = x.toUnderType();
|
||||
char * start = reinterpret_cast<char *>(&tmp_x);
|
||||
char * end = start + sizeof(tmp_x);
|
||||
std::reverse(start, end);
|
||||
operator()(tmp_x);
|
||||
#else
|
||||
operator()(x.toUnderType());
|
||||
#endif
|
||||
}
|
||||
void operator() (const IPv4 & x) const
|
||||
{
|
||||
|
@ -345,6 +345,7 @@ private:
|
||||
PreparedInsert(pqxx::connection & connection_, const String & table, const String & schema,
|
||||
const ColumnsWithTypeAndName & columns, const String & on_conflict_)
|
||||
: Inserter(connection_)
|
||||
, statement_name("insert_" + getHexUIntLowercase(thread_local_rng()))
|
||||
{
|
||||
WriteBufferFromOwnString buf;
|
||||
buf << getInsertQuery(schema, table, columns, IdentifierQuotingStyle::DoubleQuotes);
|
||||
@ -357,12 +358,14 @@ private:
|
||||
}
|
||||
buf << ") ";
|
||||
buf << on_conflict_;
|
||||
connection.prepare("insert", buf.str());
|
||||
connection.prepare(statement_name, buf.str());
|
||||
prepared = true;
|
||||
}
|
||||
|
||||
void complete() override
|
||||
{
|
||||
connection.unprepare("insert");
|
||||
connection.unprepare(statement_name);
|
||||
prepared = false;
|
||||
tx.commit();
|
||||
}
|
||||
|
||||
@ -371,8 +374,24 @@ private:
|
||||
pqxx::params params;
|
||||
params.reserve(row.size());
|
||||
params.append_multi(row);
|
||||
tx.exec_prepared("insert", params);
|
||||
tx.exec_prepared(statement_name, params);
|
||||
}
|
||||
|
||||
~PreparedInsert() override
|
||||
{
|
||||
try
|
||||
{
|
||||
if (prepared)
|
||||
connection.unprepare(statement_name);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
const String statement_name;
|
||||
bool prepared = false;
|
||||
};
|
||||
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: long
|
||||
# Tags: long, no-debug
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -3,15 +3,27 @@
|
||||
1 l \N Nullable(String)
|
||||
2 \N Nullable(String)
|
||||
-
|
||||
1 l Nullable(String) \N Nullable(String)
|
||||
0 \N Nullable(String) \N Nullable(String)
|
||||
0 \N Nullable(String) \N Nullable(String)
|
||||
1 l Nullable(String) \N Nullable(String)
|
||||
-
|
||||
1 l LowCardinality(String) \N Nullable(String)
|
||||
0 LowCardinality(String) \N Nullable(String)
|
||||
0 LowCardinality(String) \N Nullable(String)
|
||||
1 l LowCardinality(String) \N Nullable(String)
|
||||
-
|
||||
1 l \N Nullable(String)
|
||||
0 \N \N Nullable(String)
|
||||
0 \N \N Nullable(String)
|
||||
1 l \N Nullable(String)
|
||||
-
|
||||
1 l \N Nullable(String)
|
||||
0 \N Nullable(String)
|
||||
0 \N Nullable(String)
|
||||
1 l \N Nullable(String)
|
||||
-
|
||||
1 l \N Nullable(String)
|
||||
0 \N Nullable(String)
|
||||
0 \N Nullable(String)
|
||||
1 l \N Nullable(String)
|
||||
0 \N
|
||||
-
|
||||
0
|
||||
-
|
||||
|
@ -15,19 +15,37 @@ SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l FULL JOIN nr AS r USING (x) ORD
|
||||
|
||||
SELECT '-';
|
||||
|
||||
SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l LEFT JOIN nr AS r USING (lc) ORDER BY x;
|
||||
SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x;
|
||||
SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l FULL JOIN nr AS r USING (lc) ORDER BY x;
|
||||
-- lc should be supertype for l.lc and r.lc, so expect Nullable(String)
|
||||
SELECT x, lc, toTypeName(lc), r.lc, toTypeName(r.lc) FROM t AS l LEFT JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 1;
|
||||
SELECT x, lc, toTypeName(lc), r.lc, toTypeName(r.lc) FROM t AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 1;
|
||||
SELECT x, lc, toTypeName(lc), r.lc, toTypeName(r.lc) FROM t AS l FULL JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 1;
|
||||
|
||||
SELECT '-';
|
||||
|
||||
SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l LEFT JOIN nr AS r USING (lc) ORDER BY x;
|
||||
SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x;
|
||||
SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l FULL JOIN nr AS r USING (lc) ORDER BY x;
|
||||
-- old behavior is different
|
||||
SELECT x, lc, toTypeName(lc), r.lc, toTypeName(r.lc) FROM t AS l LEFT JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 0;
|
||||
SELECT x, lc, toTypeName(lc), r.lc, toTypeName(r.lc) FROM t AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 0;
|
||||
SELECT x, lc, toTypeName(lc), r.lc, toTypeName(r.lc) FROM t AS l FULL JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 0;
|
||||
|
||||
SELECT '-';
|
||||
|
||||
SELECT x, lc FROM t AS l RIGHT JOIN nr AS r USING (lc);
|
||||
SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l LEFT JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 1;
|
||||
SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 1;
|
||||
SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l FULL JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 1;
|
||||
|
||||
SELECT '-';
|
||||
|
||||
SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l LEFT JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 0;
|
||||
SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 0;
|
||||
SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l FULL JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 0;
|
||||
|
||||
SELECT '-';
|
||||
|
||||
SELECT x, lc FROM t AS l RIGHT JOIN nr AS r USING (lc) SETTINGS allow_experimental_analyzer = 1;
|
||||
|
||||
SELECT '-';
|
||||
|
||||
SELECT x, lc FROM t AS l RIGHT JOIN nr AS r USING (lc) SETTINGS allow_experimental_analyzer = 0;
|
||||
|
||||
SELECT '-';
|
||||
|
||||
|
@ -4,6 +4,16 @@
|
||||
2 \N Nullable(String)
|
||||
-
|
||||
1 l \N Nullable(String)
|
||||
0 \N \N Nullable(String)
|
||||
0 \N \N Nullable(String)
|
||||
1 l \N Nullable(String)
|
||||
-
|
||||
1 l \N Nullable(String)
|
||||
0 \N \N Nullable(String)
|
||||
0 \N \N Nullable(String)
|
||||
1 l \N Nullable(String)
|
||||
-
|
||||
1 l \N Nullable(String)
|
||||
0 \N Nullable(String)
|
||||
0 \N Nullable(String)
|
||||
1 l \N Nullable(String)
|
||||
|
@ -17,15 +17,27 @@ SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l FULL JOIN nr AS r USING (x) ORD
|
||||
|
||||
SELECT '-';
|
||||
|
||||
SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l LEFT JOIN nr AS r USING (lc) ORDER BY x;
|
||||
SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x;
|
||||
SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l FULL JOIN nr AS r USING (lc) ORDER BY x;
|
||||
SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l LEFT JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 1;
|
||||
SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 1;
|
||||
SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l FULL JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 1;
|
||||
|
||||
SELECT '-';
|
||||
|
||||
SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l LEFT JOIN nr AS r USING (lc) ORDER BY x;
|
||||
SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x;
|
||||
SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l FULL JOIN nr AS r USING (lc) ORDER BY x;
|
||||
SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l LEFT JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 1;
|
||||
SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 1;
|
||||
SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l FULL JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 1;
|
||||
|
||||
SELECT '-';
|
||||
|
||||
SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l LEFT JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 0;
|
||||
SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 0;
|
||||
SELECT x, lc, r.lc, toTypeName(r.lc) FROM t AS l FULL JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 0;
|
||||
|
||||
SELECT '-';
|
||||
|
||||
SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l LEFT JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 0;
|
||||
SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l RIGHT JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 0;
|
||||
SELECT x, lc, materialize(r.lc) y, toTypeName(y) FROM t AS l FULL JOIN nr AS r USING (lc) ORDER BY x SETTINGS allow_experimental_analyzer = 0;
|
||||
|
||||
SELECT '-';
|
||||
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: long, no-parallel
|
||||
# Tags: long, no-parallel, no-debug
|
||||
|
||||
set -e
|
||||
|
||||
|
@ -3,6 +3,16 @@
|
||||
1 l \N LowCardinality(String) Nullable(String)
|
||||
2 \N LowCardinality(String) Nullable(String)
|
||||
-
|
||||
\N \N Nullable(String) LowCardinality(String)
|
||||
1 \N l Nullable(String) LowCardinality(String)
|
||||
1 \N l Nullable(String) LowCardinality(String)
|
||||
\N \N Nullable(String) LowCardinality(String)
|
||||
-
|
||||
1 l \N LowCardinality(String) Nullable(String)
|
||||
2 \N LowCardinality(String) Nullable(String)
|
||||
1 l \N LowCardinality(String) Nullable(String)
|
||||
2 \N LowCardinality(String) Nullable(String)
|
||||
-
|
||||
0 \N Nullable(String) LowCardinality(String)
|
||||
1 \N l Nullable(String) LowCardinality(String)
|
||||
0 \N Nullable(String) LowCardinality(String)
|
||||
|
@ -10,8 +10,27 @@ CREATE TABLE nr (`x` Nullable(UInt32), `s` Nullable(String)) ENGINE = Memory;
|
||||
INSERT INTO t VALUES (1, 'l');
|
||||
INSERT INTO nr VALUES (2, NULL);
|
||||
|
||||
|
||||
SET join_use_nulls = 0;
|
||||
|
||||
SET allow_experimental_analyzer = 1;
|
||||
|
||||
-- t.x is supertupe for `x` from left and right since `x` is inside `USING`.
|
||||
SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l LEFT JOIN nr AS r USING (x) ORDER BY t.x;
|
||||
SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l RIGHT JOIN nr AS r USING (x) ORDER BY t.x;
|
||||
SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l FULL JOIN nr AS r USING (x) ORDER BY t.x;
|
||||
|
||||
SELECT '-';
|
||||
|
||||
SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l LEFT JOIN t AS r USING (x) ORDER BY t.x;
|
||||
SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l RIGHT JOIN t AS r USING (x) ORDER BY t.x;
|
||||
SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l FULL JOIN t AS r USING (x) ORDER BY t.x;
|
||||
|
||||
SELECT '-';
|
||||
|
||||
SET allow_experimental_analyzer = 0;
|
||||
|
||||
-- t.x is supertupe for `x` from left and right since `x` is inside `USING`.
|
||||
SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l LEFT JOIN nr AS r USING (x) ORDER BY t.x;
|
||||
SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l RIGHT JOIN nr AS r USING (x) ORDER BY t.x;
|
||||
SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l FULL JOIN nr AS r USING (x) ORDER BY t.x;
|
||||
|
@ -17,7 +17,7 @@
|
||||
1 \N l Nullable(String) LowCardinality(String)
|
||||
0 \N Nullable(String) LowCardinality(String)
|
||||
1 \N l Nullable(String) LowCardinality(String)
|
||||
-
|
||||
- join_use_nulls -
|
||||
1 l \N LowCardinality(String) Nullable(String)
|
||||
2 \N \N LowCardinality(Nullable(String)) Nullable(String)
|
||||
1 l \N LowCardinality(Nullable(String)) Nullable(String)
|
||||
@ -33,3 +33,47 @@
|
||||
1 l \N LowCardinality(Nullable(String)) Nullable(String)
|
||||
\N \N \N LowCardinality(Nullable(String)) Nullable(String)
|
||||
-
|
||||
\N \N \N Nullable(String) LowCardinality(Nullable(String))
|
||||
1 \N l Nullable(String) LowCardinality(String)
|
||||
1 \N l Nullable(String) LowCardinality(Nullable(String))
|
||||
\N \N \N Nullable(String) LowCardinality(Nullable(String))
|
||||
- analyzer -
|
||||
1 l \N LowCardinality(String) Nullable(String)
|
||||
2 \N LowCardinality(String) Nullable(String)
|
||||
1 l \N LowCardinality(String) Nullable(String)
|
||||
2 \N LowCardinality(String) Nullable(String)
|
||||
-
|
||||
\N \N Nullable(String) LowCardinality(String)
|
||||
1 \N l Nullable(String) LowCardinality(String)
|
||||
1 \N l Nullable(String) LowCardinality(String)
|
||||
\N \N Nullable(String) LowCardinality(String)
|
||||
-
|
||||
1 l \N Nullable(String) Nullable(String)
|
||||
0 \N \N Nullable(String) Nullable(String)
|
||||
0 \N \N Nullable(String) Nullable(String)
|
||||
1 l \N Nullable(String) Nullable(String)
|
||||
-
|
||||
0 \N \N Nullable(String) Nullable(String)
|
||||
1 \N l Nullable(String) Nullable(String)
|
||||
0 \N \N Nullable(String) Nullable(String)
|
||||
1 \N l Nullable(String) Nullable(String)
|
||||
- join_use_nulls -
|
||||
1 l \N LowCardinality(String) Nullable(String)
|
||||
2 \N \N LowCardinality(Nullable(String)) Nullable(String)
|
||||
1 l \N LowCardinality(Nullable(String)) Nullable(String)
|
||||
2 \N \N LowCardinality(Nullable(String)) Nullable(String)
|
||||
-
|
||||
\N \N \N Nullable(String) LowCardinality(Nullable(String))
|
||||
1 \N l Nullable(String) LowCardinality(String)
|
||||
1 \N l Nullable(String) LowCardinality(Nullable(String))
|
||||
\N \N \N Nullable(String) LowCardinality(Nullable(String))
|
||||
-
|
||||
1 l \N Nullable(String) Nullable(String)
|
||||
\N \N \N Nullable(String) Nullable(String)
|
||||
1 l \N Nullable(String) Nullable(String)
|
||||
\N \N \N Nullable(String) Nullable(String)
|
||||
-
|
||||
\N \N \N Nullable(String) Nullable(String)
|
||||
1 \N l Nullable(String) Nullable(String)
|
||||
1 \N l Nullable(String) Nullable(String)
|
||||
\N \N \N Nullable(String) Nullable(String)
|
||||
|
@ -10,6 +10,14 @@ CREATE TABLE nr (`x` Nullable(UInt32), `s` Nullable(String)) ENGINE = Memory;
|
||||
INSERT INTO t VALUES (1, 'l');
|
||||
INSERT INTO nr VALUES (2, NULL);
|
||||
|
||||
{% for allow_experimental_analyzer in [0, 1] -%}
|
||||
|
||||
SET allow_experimental_analyzer = {{ allow_experimental_analyzer }};
|
||||
|
||||
{% if allow_experimental_analyzer -%}
|
||||
SELECT '- analyzer -';
|
||||
{% endif -%}
|
||||
|
||||
SET join_use_nulls = 0;
|
||||
|
||||
SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l LEFT JOIN nr AS r USING (x) ORDER BY t.x;
|
||||
@ -36,7 +44,7 @@ SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l FULL JOIN t
|
||||
|
||||
SET join_use_nulls = 1;
|
||||
|
||||
SELECT '-';
|
||||
SELECT '- join_use_nulls -';
|
||||
|
||||
SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l LEFT JOIN nr AS r USING (x) ORDER BY t.x;
|
||||
SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l RIGHT JOIN nr AS r USING (x) ORDER BY t.x;
|
||||
@ -56,10 +64,11 @@ SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM t AS l FULL JOIN nr
|
||||
|
||||
SELECT '-';
|
||||
|
||||
-- TODO
|
||||
-- SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l LEFT JOIN t AS r USING (s) ORDER BY t.x;
|
||||
-- SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l RIGHT JOIN t AS r USING (s) ORDER BY t.x;
|
||||
-- SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l FULL JOIN t AS r USING (s) ORDER BY t.x;
|
||||
SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l LEFT JOIN t AS r USING (s) ORDER BY t.x;
|
||||
SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l RIGHT JOIN t AS r USING (s) ORDER BY t.x;
|
||||
SELECT t.x, l.s, r.s, toTypeName(l.s), toTypeName(r.s) FROM nr AS l FULL JOIN t AS r USING (s) ORDER BY t.x;
|
||||
|
||||
{% endfor %}
|
||||
|
||||
DROP TABLE t;
|
||||
DROP TABLE nr;
|
@ -1 +1,2 @@
|
||||
6ba51fa36c625adab5d58007c96e32bf
|
||||
ebc1c2f37455caea601feeb840757dd3
|
||||
|
@ -1,7 +1,32 @@
|
||||
drop table if exists tab;
|
||||
create table tab (i8 Int8, i16 Int16, i32 Int32, i64 Int64, i128 Int128, i256 Int256, u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64, u128 UInt128, u256 UInt256, id UUID, s String, fs FixedString(33), a Array(UInt8), t Tuple(UInt16, UInt32), d Date, dt DateTime('Asia/Istanbul'), dt64 DateTime64(3, 'Asia/Istanbul'), dec128 Decimal128(3), dec256 Decimal256(4), lc LowCardinality(String)) engine = MergeTree PARTITION BY (i8, i16, i32, i64, i128, i256, u8, u16, u32, u64, u128, u256, id, s, fs, a, t, d, dt, dt64, dec128, dec256, lc) order by tuple();
|
||||
insert into tab values (-1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, '61f0c404-5cb3-11e7-907b-a6006ad3dba0', 'a', 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', [1, 2, 3], (-1, -2), '2020-01-01', '2020-01-01 01:01:01', '2020-01-01 01:01:01', '123.456', '78.9101', 'a');
|
||||
DROP TABLE IF EXISTS tab;
|
||||
CREATE TABLE tab (
|
||||
i8 Int8,
|
||||
i16 Int16,
|
||||
i32 Int32,
|
||||
i64 Int64,
|
||||
i128 Int128,
|
||||
i256 Int256,
|
||||
u8 UInt8,
|
||||
u16 UInt16,
|
||||
u32 UInt32,
|
||||
u64 UInt64,
|
||||
u128 UInt128,
|
||||
u256 UInt256,
|
||||
id UUID,
|
||||
s String,
|
||||
fs FixedString(33),
|
||||
a Array(UInt8),
|
||||
t Tuple(UInt16, UInt32),
|
||||
d Date,
|
||||
dt DateTime('Asia/Istanbul'),
|
||||
dt64 DateTime64(3, 'Asia/Istanbul'),
|
||||
dec128 Decimal128(3),
|
||||
dec256 Decimal256(4),
|
||||
lc LowCardinality(String))
|
||||
engine = MergeTree PARTITION BY (i8, i16, i32, i64, i128, i256, u8, u16, u32, u64, u128, u256, id, s, fs, a, t, d, dt, dt64, dec128, dec256, lc) ORDER BY tuple();
|
||||
INSERT INTO tab VALUES (-1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, '61f0c404-5cb3-11e7-907b-a6006ad3dba0', 'a', 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', [1, 2, 3], (-1, -2), '2020-01-01', '2020-01-01 01:01:01', '2020-01-01 01:01:01', '123.456', '78.9101', 'a');
|
||||
INSERT INTO tab VALUES (123, 12345, 1234567890, 1234567890000000000, 123456789000000000000000000000000000000, 123456789000000000000000000000000000000000000000000000000000000000000000000000, 123, 12345, 1234567890, 1234567890000000000, 123456789000000000000000000000000000000, 123456789000000000000000000000000000000000000000000000000000000000000000000000, '61f0c404-5cb3-11e7-907b-a6006ad3dba0', 'a', 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa', [1, 2, 3], (-1, -2), '2020-01-01', '2020-01-01 01:01:01', '2020-01-01 01:01:01', '123.456', '78.9101', 'a');
|
||||
-- Here we check that partition id did not change.
|
||||
-- Different result means Backward Incompatible Change. Old partitions will not be accepted by new server.
|
||||
select partition_id from system.parts where table = 'tab' and database = currentDatabase();
|
||||
drop table if exists tab;
|
||||
SELECT partition_id FROM system.parts WHERE table = 'tab' AND database = currentDatabase();
|
||||
DROP TABLE IF EXISTS tab;
|
||||
|
Loading…
Reference in New Issue
Block a user