mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
Merge branch 'master' into get-rid-of-ast-set
This commit is contained in:
commit
200a1e27e3
@ -1,10 +1,10 @@
|
||||
#pragma once
|
||||
|
||||
#include <memory>
|
||||
|
||||
#include <Common/PODArray.h>
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <common/StringRef.h>
|
||||
|
||||
|
@ -85,7 +85,7 @@ Block ColumnGathererStream::readImpl()
|
||||
if (sources.empty())
|
||||
init();
|
||||
|
||||
if (row_sources_buf.eof())
|
||||
if (!source_to_fully_copy && row_sources_buf.eof())
|
||||
return Block();
|
||||
|
||||
output_block = Block{column.cloneEmpty()};
|
||||
|
@ -1409,6 +1409,22 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(ASTPtr & node, const Block & sampl
|
||||
}
|
||||
|
||||
|
||||
static std::pair<String, String> getDatabaseAndTableNameFromIdentifier(const ASTIdentifier & identifier)
|
||||
{
|
||||
std::pair<String, String> res;
|
||||
res.second = identifier.name;
|
||||
if (!identifier.children.empty())
|
||||
{
|
||||
if (identifier.children.size() != 2)
|
||||
throw Exception("Qualified table name could have only two components", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
res.first = typeid_cast<const ASTIdentifier &>(*identifier.children[0]).name;
|
||||
res.second = typeid_cast<const ASTIdentifier &>(*identifier.children[1]).name;
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
static std::shared_ptr<InterpreterSelectQuery> interpretSubquery(
|
||||
ASTPtr & subquery_or_table_name, const Context & context, size_t subquery_depth, const Names & required_columns)
|
||||
{
|
||||
@ -1446,7 +1462,8 @@ static std::shared_ptr<InterpreterSelectQuery> interpretSubquery(
|
||||
select_query->children.emplace_back(select_query->select_expression_list);
|
||||
|
||||
/// get columns list for target table
|
||||
const auto & storage = context.getTable("", table->name);
|
||||
auto database_table = getDatabaseAndTableNameFromIdentifier(*table);
|
||||
const auto & storage = context.getTable(database_table.first, database_table.second);
|
||||
const auto & columns = storage->getColumnsListNonMaterialized();
|
||||
select_expression_list->children.reserve(columns.size());
|
||||
|
||||
@ -1455,7 +1472,7 @@ static std::shared_ptr<InterpreterSelectQuery> interpretSubquery(
|
||||
select_expression_list->children.emplace_back(std::make_shared<ASTIdentifier>(
|
||||
StringRange{}, column.name));
|
||||
|
||||
select_query->replaceDatabaseAndTable("", table->name);
|
||||
select_query->replaceDatabaseAndTable(database_table.first, database_table.second);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1526,11 +1543,12 @@ void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_
|
||||
/// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery.
|
||||
String set_id = arg->getColumnName();
|
||||
|
||||
/// A special case is if the name of the table is specified on the right side of the IN statement, and the table has the type Set (a previously prepared set).
|
||||
/// TODO This syntax does not support the specification of the database name.
|
||||
/// A special case is if the name of the table is specified on the right side of the IN statement,
|
||||
/// and the table has the type Set (a previously prepared set).
|
||||
if (identifier)
|
||||
{
|
||||
StoragePtr table = context.tryGetTable("", identifier->name);
|
||||
auto database_table = getDatabaseAndTableNameFromIdentifier(*identifier);
|
||||
StoragePtr table = context.tryGetTable(database_table.first, database_table.second);
|
||||
|
||||
if (table)
|
||||
{
|
||||
@ -2376,7 +2394,8 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
|
||||
/// TODO This syntax does not support specifying a database name.
|
||||
if (table_to_join.database_and_table_name)
|
||||
{
|
||||
StoragePtr table = context.tryGetTable("", static_cast<const ASTIdentifier &>(*table_to_join.database_and_table_name).name);
|
||||
auto database_table = getDatabaseAndTableNameFromIdentifier(static_cast<const ASTIdentifier &>(*table_to_join.database_and_table_name));
|
||||
StoragePtr table = context.tryGetTable(database_table.first, database_table.second);
|
||||
|
||||
if (table)
|
||||
{
|
||||
@ -2773,7 +2792,8 @@ void ExpressionAnalyzer::collectJoinedColumns(NameSet & joined_columns, NamesAnd
|
||||
Block nested_result_sample;
|
||||
if (table_expression.database_and_table_name)
|
||||
{
|
||||
const auto & table = context.getTable("", static_cast<const ASTIdentifier &>(*table_expression.database_and_table_name).name);
|
||||
auto database_table = getDatabaseAndTableNameFromIdentifier(static_cast<const ASTIdentifier &>(*table_expression.database_and_table_name));
|
||||
const auto & table = context.getTable(database_table.first, database_table.second);
|
||||
nested_result_sample = table->getSampleBlockNonMaterialized();
|
||||
}
|
||||
else if (table_expression.subquery)
|
||||
|
@ -30,6 +30,7 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
|
||||
@ -53,6 +54,7 @@ BlockIO InterpreterAlterQuery::execute()
|
||||
PartitionCommands partition_commands;
|
||||
parseAlter(alter.parameters, alter_commands, partition_commands);
|
||||
|
||||
partition_commands.validate(table.get());
|
||||
for (const PartitionCommand & command : partition_commands)
|
||||
{
|
||||
switch (command.type)
|
||||
@ -89,7 +91,6 @@ BlockIO InterpreterAlterQuery::execute()
|
||||
return {};
|
||||
|
||||
alter_commands.validate(table.get(), context);
|
||||
|
||||
table->alter(alter_commands, database_name, table_name, context);
|
||||
|
||||
return {};
|
||||
@ -230,4 +231,23 @@ void InterpreterAlterQuery::parseAlter(
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void InterpreterAlterQuery::PartitionCommands::validate(const IStorage * table)
|
||||
{
|
||||
for (const PartitionCommand & command : *this)
|
||||
{
|
||||
if (command.type == PartitionCommand::CLEAR_COLUMN)
|
||||
{
|
||||
String column_name = command.column_name.safeGet<String>();
|
||||
|
||||
if (!table->hasRealColumn(column_name))
|
||||
{
|
||||
throw Exception("Wrong column name. Cannot find column " + column_name + " to clear it from partition",
|
||||
DB::ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -112,7 +112,11 @@ private:
|
||||
}
|
||||
};
|
||||
|
||||
using PartitionCommands = std::vector<PartitionCommand>;
|
||||
class PartitionCommands : public std::vector<PartitionCommand>
|
||||
{
|
||||
public:
|
||||
void validate(const IStorage * table);
|
||||
};
|
||||
|
||||
ASTPtr query_ptr;
|
||||
|
||||
|
@ -13,14 +13,14 @@ $chl "INSERT INTO test.partition_428 (p, k) VALUES(toDate(31), 1)"
|
||||
$chl "INSERT INTO test.partition_428 (p, k) VALUES(toDate(1), 2)"
|
||||
|
||||
for part in `$chl "SELECT name FROM system.parts WHERE database='test' AND table='partition_428'"`; do
|
||||
cat $ch_dir/data/test/partition_428/$part/columns.txt | wc -l # 2 header lines + 3 columns
|
||||
sudo cat $ch_dir/data/test/partition_428/$part/columns.txt | wc -l # 2 header lines + 3 columns
|
||||
done
|
||||
|
||||
$chl "ALTER TABLE test.partition_428 DETACH PARTITION 197001"
|
||||
$chl "ALTER TABLE test.partition_428 ATTACH PARTITION 197001"
|
||||
|
||||
for part in `$chl "SELECT name FROM system.parts WHERE database='test' AND table='partition_428'"`; do
|
||||
cat $ch_dir/data/test/partition_428/$part/columns.txt | wc -l # 2 header lines + 3 columns
|
||||
sudo cat $ch_dir/data/test/partition_428/$part/columns.txt | wc -l # 2 header lines + 3 columns
|
||||
done
|
||||
|
||||
$chl "ALTER TABLE test.partition_428 MODIFY COLUMN v1 Int8"
|
||||
|
@ -3,18 +3,24 @@
|
||||
0 a
|
||||
2 b
|
||||
all
|
||||
2000-01-01 0
|
||||
2000-01-01 1 a
|
||||
2000-01-01 3 c
|
||||
2000-02-01 0
|
||||
2000-02-01 2 b
|
||||
2000-02-01 4 d
|
||||
w/o i 1
|
||||
2000-01-01 0
|
||||
2000-01-01 0 a
|
||||
2000-01-01 0 c
|
||||
2000-02-01 0
|
||||
2000-02-01 2 b
|
||||
2000-02-01 4 d
|
||||
w/o is 1
|
||||
2000-01-01 0
|
||||
2000-01-01 0
|
||||
2000-01-01 0
|
||||
2000-02-01 0
|
||||
2000-02-01 2 b
|
||||
2000-02-01 4 d
|
||||
w/o is 12
|
||||
|
@ -13,8 +13,14 @@ DROP TABLE test.clear_column;
|
||||
|
||||
DROP TABLE IF EXISTS test.clear_column1;
|
||||
DROP TABLE IF EXISTS test.clear_column2;
|
||||
CREATE TABLE test.clear_column1 (d Date, i Int64, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/clear_column', '1', d, d, 8192);
|
||||
CREATE TABLE test.clear_column2 (d Date, i Int64, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/clear_column', '2', d, d, 8192);
|
||||
CREATE TABLE test.clear_column1 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/clear_column', '1', d, d, 8192);
|
||||
CREATE TABLE test.clear_column2 (d Date, i Int64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/clear_column', '2', d, d, 8192);
|
||||
|
||||
INSERT INTO test.clear_column1 (d) VALUES ('2000-01-01'), ('2000-02-01');
|
||||
|
||||
SET replication_alter_partitions_sync=2;
|
||||
ALTER TABLE test.clear_column1 ADD COLUMN s String;
|
||||
ALTER TABLE test.clear_column1 CLEAR COLUMN s IN PARTITION '200001';
|
||||
|
||||
INSERT INTO test.clear_column1 VALUES ('2000-01-01', 1, 'a'), ('2000-02-01', 2, 'b');
|
||||
INSERT INTO test.clear_column1 VALUES ('2000-01-01', 3, 'c'), ('2000-02-01', 4, 'd');
|
||||
@ -22,8 +28,6 @@ INSERT INTO test.clear_column1 VALUES ('2000-01-01', 3, 'c'), ('2000-02-01', 4,
|
||||
SELECT 'all';
|
||||
SELECT * FROM test.clear_column1 ORDER BY d, i, s;
|
||||
|
||||
SET replication_alter_partitions_sync=2;
|
||||
|
||||
SELECT 'w/o i 1';
|
||||
ALTER TABLE test.clear_column1 CLEAR COLUMN i IN PARTITION '200001';
|
||||
SELECT * FROM test.clear_column2 ORDER BY d, i, s;
|
||||
@ -45,7 +49,7 @@ SELECT sum(data_uncompressed_bytes) FROM system.columns WHERE database='test' AN
|
||||
ALTER TABLE test.clear_column1 CLEAR COLUMN s IN PARTITION '200001';
|
||||
ALTER TABLE test.clear_column1 CLEAR COLUMN s IN PARTITION '200002';
|
||||
|
||||
-- check optimize for non-leader replica
|
||||
-- check optimize for non-leader replica (it is not related with CLEAR COLUMN)
|
||||
OPTIMIZE TABLE test.clear_column1;
|
||||
OPTIMIZE TABLE test.clear_column2;
|
||||
|
||||
|
@ -1,5 +1,4 @@
|
||||
#!/bin/bash
|
||||
set -e
|
||||
|
||||
ch="clickhouse-client --stacktrace -q"
|
||||
|
||||
@ -8,6 +7,14 @@ $ch "DROP TABLE IF EXISTS test.clear_column2"
|
||||
$ch "CREATE TABLE test.clear_column1 (d Date, i Int64, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/clear_column', '1', d, d, 8192)"
|
||||
$ch "CREATE TABLE test.clear_column2 (d Date, i Int64, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/clear_column', '2', d, d, 8192)"
|
||||
|
||||
$ch "ALTER TABLE test.clear_column1 CLEAR COLUMN VasyaUnexistingColumn IN PARTITION '200001'" 1>/dev/null 2>/dev/null
|
||||
rc=$?
|
||||
if [ $rc -eq 0 ]; then
|
||||
echo "An unexisisting column was ALTERed. Code: $rc"
|
||||
exit -1
|
||||
fi
|
||||
|
||||
set -e
|
||||
$ch "INSERT INTO test.clear_column1 VALUES ('2000-01-01', 1, 'a'), ('2000-02-01', 2, 'b')"
|
||||
$ch "INSERT INTO test.clear_column1 VALUES ('2000-01-01', 3, 'c'), ('2000-02-01', 4, 'd')"
|
||||
|
||||
@ -24,5 +31,5 @@ wait
|
||||
$ch "SELECT DISTINCT * FROM test.clear_column1 WHERE d != toDate('2000-03-01') ORDER BY d, i, s"
|
||||
$ch "SELECT DISTINCT * FROM test.clear_column2 WHERE d != toDate('2000-03-01') ORDER BY d, i, s"
|
||||
|
||||
$ch "DROP TABLE IF EXISTS test.clear_column1"
|
||||
$ch "DROP TABLE IF EXISTS test.clear_column2"
|
||||
#$ch "DROP TABLE IF EXISTS test.clear_column1"
|
||||
#$ch "DROP TABLE IF EXISTS test.clear_column2"
|
||||
|
@ -0,0 +1,8 @@
|
||||
1 hello
|
||||
0 world
|
||||
1 hello
|
||||
0 world
|
||||
1 hello
|
||||
2
|
||||
1 hello
|
||||
2
|
23
dbms/tests/queries/0_stateless/00475_in_join_db_table.sql
Normal file
23
dbms/tests/queries/0_stateless/00475_in_join_db_table.sql
Normal file
@ -0,0 +1,23 @@
|
||||
DROP TABLE IF EXISTS test.set;
|
||||
CREATE TABLE test.set (x String) ENGINE = Memory;
|
||||
INSERT INTO test.set VALUES ('hello');
|
||||
SELECT (arrayJoin(['hello', 'world']) AS s) IN test.set, s;
|
||||
|
||||
DROP TABLE test.set;
|
||||
CREATE TABLE test.set (x String) ENGINE = Set;
|
||||
INSERT INTO test.set VALUES ('hello');
|
||||
SELECT (arrayJoin(['hello', 'world']) AS s) IN test.set, s;
|
||||
|
||||
DROP TABLE test.set;
|
||||
|
||||
DROP TABLE IF EXISTS test.join;
|
||||
CREATE TABLE test.join (k UInt8, x String) ENGINE = Memory;
|
||||
INSERT INTO test.join VALUES (1, 'hello');
|
||||
SELECT k, x FROM (SELECT arrayJoin([1, 2]) AS k) ANY LEFT JOIN test.join USING k;
|
||||
|
||||
DROP TABLE test.join;
|
||||
CREATE TABLE test.join (k UInt8, x String) ENGINE = Join(ANY, LEFT, k);
|
||||
INSERT INTO test.join VALUES (1, 'hello');
|
||||
SELECT k, x FROM (SELECT arrayJoin([1, 2]) AS k) ANY LEFT JOIN test.join USING k;
|
||||
|
||||
DROP TABLE test.join;
|
@ -0,0 +1,68 @@
|
||||
┏━━━━━━━━━━━━┳━━━━━━━━━━━━┓
|
||||
┃ [1m x[0m ┃ [1ms [0m ┃
|
||||
┡━━━━━━━━━━━━╇━━━━━━━━━━━━┩
|
||||
│ 1 │ 1 │
|
||||
├────────────┼────────────┤
|
||||
│ 10 │ 10 │
|
||||
├────────────┼────────────┤
|
||||
│ 100 │ 100 │
|
||||
├────────────┼────────────┤
|
||||
│ 1001 │ 1001 │
|
||||
├────────────┼────────────┤
|
||||
│ 10001 │ 10001 │
|
||||
├────────────┼────────────┤
|
||||
│ 100000 │ 100000 │
|
||||
├────────────┼────────────┤
|
||||
│ 1000000 │ 1000000 │
|
||||
├────────────┼────────────┤
|
||||
│ 10000000 │ 10000000 │
|
||||
├────────────┼────────────┤
|
||||
│ 100000000 │ 100000000 │
|
||||
├────────────┼────────────┤
|
||||
│ 1000000000 │ 1000000000 │
|
||||
└────────────┴────────────┘
|
||||
┌──────────[1mx[0m─┬─[1ms[0m──────────┐
|
||||
│ 1 │ 1 │
|
||||
│ 10 │ 10 │
|
||||
│ 100 │ 100 │
|
||||
│ 1001 │ 1001 │
|
||||
│ 10001 │ 10001 │
|
||||
│ 100000 │ 100000 │
|
||||
│ 1000000 │ 1000000 │
|
||||
│ 10000000 │ 10000000 │
|
||||
│ 100000000 │ 100000000 │
|
||||
│ 1000000000 │ 1000000000 │
|
||||
└────────────┴────────────┘
|
||||
[1mx[0m [1ms[0m
|
||||
|
||||
1 1
|
||||
10 10
|
||||
100 100
|
||||
1001 1001
|
||||
10001 10001
|
||||
100000 100000
|
||||
1000000 1000000
|
||||
10000000 10000000
|
||||
100000000 100000000
|
||||
1000000000 1000000000
|
||||
┌──────────[1mx[0m─┬─[1ms[0m──────────┐
|
||||
│ 1 │ 1 │
|
||||
│ 10 │ 10 │
|
||||
│ 100 │ 100 │
|
||||
│ 1001 │ 1001 │
|
||||
│ 10001 │ 10001 │
|
||||
│ 100000 │ 100000 │
|
||||
│ 1000000 │ 1000000 │
|
||||
│ 10000000 │ 10000000 │
|
||||
│ 100000000 │ 100000000 │
|
||||
│ 1000000000 │ 1000000000 │
|
||||
└────────────┴────────────┘
|
||||
┏━━━━━━━━━━━━━━━━━━┓
|
||||
┃ [1m\'\\\\\\\'\\\'\'[0m ┃
|
||||
┡━━━━━━━━━━━━━━━━━━┩
|
||||
│ \\\'\' │
|
||||
└──────────────────┘
|
||||
Row 1:
|
||||
──────
|
||||
\'\\\\\\\'\\\'\': \\\'\'
|
||||
1: 1
|
@ -0,0 +1,7 @@
|
||||
SELECT toUInt64(ceil(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT Pretty;
|
||||
SELECT toUInt64(ceil(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettyCompact;
|
||||
SELECT toUInt64(ceil(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettySpace;
|
||||
SET max_block_size = 5;
|
||||
SELECT toUInt64(ceil(exp10(number))) AS x, toString(x) AS s FROM system.numbers LIMIT 10 FORMAT PrettyCompactMonoBlock;
|
||||
SELECT '\\''\'' FORMAT Pretty;
|
||||
SELECT '\\''\'', 1 FORMAT Vertical;
|
Loading…
Reference in New Issue
Block a user