Merge branch 'master' into get-rid-of-ast-set

This commit is contained in:
Alexey Milovidov 2017-07-15 03:23:20 +03:00
commit 200a1e27e3
13 changed files with 189 additions and 22 deletions

View File

@ -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>

View File

@ -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()};

View File

@ -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)

View File

@ -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);
}
}
}
}
}

View File

@ -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;

View File

@ -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"

View File

@ -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

View File

@ -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;

View File

@ -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"

View File

@ -0,0 +1,8 @@
1 hello
0 world
1 hello
0 world
1 hello
2
1 hello
2

View 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;

View File

@ -0,0 +1,68 @@
┏━━━━━━━━━━━━┳━━━━━━━━━━━━┓
 x ┃ s  ┃
┡━━━━━━━━━━━━╇━━━━━━━━━━━━┩
│ 1 │ 1 │
├────────────┼────────────┤
│ 10 │ 10 │
├────────────┼────────────┤
│ 100 │ 100 │
├────────────┼────────────┤
│ 1001 │ 1001 │
├────────────┼────────────┤
│ 10001 │ 10001 │
├────────────┼────────────┤
│ 100000 │ 100000 │
├────────────┼────────────┤
│ 1000000 │ 1000000 │
├────────────┼────────────┤
│ 10000000 │ 10000000 │
├────────────┼────────────┤
│ 100000000 │ 100000000 │
├────────────┼────────────┤
│ 1000000000 │ 1000000000 │
└────────────┴────────────┘
┌──────────x─┬─s──────────┐
│ 1 │ 1 │
│ 10 │ 10 │
│ 100 │ 100 │
│ 1001 │ 1001 │
│ 10001 │ 10001 │
│ 100000 │ 100000 │
│ 1000000 │ 1000000 │
│ 10000000 │ 10000000 │
│ 100000000 │ 100000000 │
│ 1000000000 │ 1000000000 │
└────────────┴────────────┘
x s
1 1
10 10
100 100
1001 1001
10001 10001
100000 100000
1000000 1000000
10000000 10000000
100000000 100000000
1000000000 1000000000
┌──────────x─┬─s──────────┐
│ 1 │ 1 │
│ 10 │ 10 │
│ 100 │ 100 │
│ 1001 │ 1001 │
│ 10001 │ 10001 │
│ 100000 │ 100000 │
│ 1000000 │ 1000000 │
│ 10000000 │ 10000000 │
│ 100000000 │ 100000000 │
│ 1000000000 │ 1000000000 │
└────────────┴────────────┘
┏━━━━━━━━━━━━━━━━━━┓
\'\\\\\\\'\\\'\' ┃
┡━━━━━━━━━━━━━━━━━━┩
│ \\\'\' │
└──────────────────┘
Row 1:
──────
\'\\\\\\\'\\\'\': \\\'\'
1: 1

View File

@ -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;