Rename DROP COLUMN FROM PARTITION -> CLEAR COLUMN IN PARTITION. [#CLICKHOUSE-2896]

This commit is contained in:
Vitaliy Lyudvichenko 2017-06-22 14:01:30 +03:00
parent 53adb891b3
commit eff3588f28
14 changed files with 62 additions and 37 deletions

View File

@ -29,6 +29,7 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int BAD_ARGUMENTS;
}
@ -78,7 +79,7 @@ BlockIO InterpreterAlterQuery::execute()
command.coordinator, context);
break;
case PartitionCommand::DROP_COLUMN:
case PartitionCommand::CLEAR_COLUMN:
table->dropColumnFromPartition(query_ptr, command.partition, command.column_name, context.getSettingsRef());
break;
}
@ -131,13 +132,19 @@ void InterpreterAlterQuery::parseAlter(
{
if (params.partition)
{
if (!params.clear_column)
throw Exception("Can't DROP COLUMN from partition. It is possible only CLEAR COLUMN in partition", ErrorCodes::BAD_ARGUMENTS);
const Field & partition = typeid_cast<const ASTLiteral &>(*(params.partition)).value;
const Field & column_name = typeid_cast<const ASTIdentifier &>(*(params.column)).name;
out_partition_commands.emplace_back(PartitionCommand::dropColumnFromPartition(partition, column_name));
out_partition_commands.emplace_back(PartitionCommand::clearColumn(partition, column_name));
}
else
{
if (params.clear_column)
throw Exception("\"ALTER TABLE table CLEAR COLUMN column\" queries are not supported yet. Use \"CLEAR COLUMN column IN PARTITION\".", ErrorCodes::NOT_IMPLEMENTED);
AlterCommand command;
command.type = AlterCommand::DROP_COLUMN;
command.column_name = typeid_cast<const ASTIdentifier &>(*(params.column)).name;

View File

@ -30,7 +30,7 @@ private:
FETCH_PARTITION,
FREEZE_PARTITION,
RESHARD_PARTITION,
DROP_COLUMN,
CLEAR_COLUMN,
};
Type type;
@ -62,10 +62,10 @@ private:
return res;
}
static PartitionCommand dropColumnFromPartition(const Field & partition, const Field & column_name)
static PartitionCommand clearColumn(const Field & partition, const Field & column_name)
{
PartitionCommand res;
res.type = DROP_COLUMN;
res.type = CLEAR_COLUMN;
res.partition = partition;
res.column_name = column_name;
return res;

View File

@ -113,11 +113,12 @@ void ASTAlterQuery::formatImpl(const FormatSettings & settings, FormatState & st
}
else if (p.type == ASTAlterQuery::DROP_COLUMN)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "DROP COLUMN " << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str
<< (p.clear_column ? "CLEAR " : "DROP ") << "COLUMN " << (settings.hilite ? hilite_none : "");
p.column->formatImpl(settings, state, frame);
if (p.partition)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " FROM PARTITION " << (settings.hilite ? hilite_none : "");
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str<< " IN PARTITION " << (settings.hilite ? hilite_none : "");
p.partition->formatImpl(settings, state, frame);
}
}

View File

@ -62,11 +62,13 @@ public:
/** In DROP PARTITION and RESHARD PARTITION queries, the name of the partition is stored here.
*/
ASTPtr partition;
bool detach = false; /// true for DETACH PARTITION.
bool detach = false; /// true for DETACH PARTITION
bool part = false; /// true for ATTACH PART
bool do_copy = false; /// for RESHARD PARTITION.
bool do_copy = false; /// for RESHARD PARTITION
bool clear_column = false; /// for CLEAR COLUMN (do not drop column from metadata)
/** For FETCH PARTITION - the path in ZK to the shard, from which to download the partition.
*/

View File

@ -17,6 +17,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_pa
ParserKeyword s_alter_table("ALTER TABLE");
ParserKeyword s_add_column("ADD COLUMN");
ParserKeyword s_drop_column("DROP COLUMN");
ParserKeyword s_clear_column("CLEAR COLUMN");
ParserKeyword s_modify_column("MODIFY COLUMN");
ParserKeyword s_modify_primary_key("MODIFY PRIMARY KEY");
@ -31,7 +32,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_pa
ParserKeyword s_after("AFTER");
ParserKeyword s_from("FROM");
ParserKeyword s_from_partition("FROM PARTITION");
ParserKeyword s_in_partition("IN PARTITION");
ParserKeyword s_copy("COPY");
ParserKeyword s_to("TO");
ParserKeyword s_using("USING");
@ -135,8 +136,21 @@ bool ParserAlterQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_pa
params.type = ASTAlterQuery::DROP_COLUMN;
params.detach = false;
}
else if (s_clear_column.ignore(pos, end, max_parsed_pos, expected))
{
ws.ignore(pos, end);
if (s_from_partition.ignore(pos, end, max_parsed_pos, expected))
if (!parser_name.parse(pos, end, params.column, max_parsed_pos, expected))
return false;
params.type = ASTAlterQuery::DROP_COLUMN;
params.clear_column = true;
params.detach = false;
ws.ignore(pos, end);
if (s_in_partition.ignore(pos, end, max_parsed_pos, expected))
{
ws.ignore(pos, end);

View File

@ -9,8 +9,9 @@ namespace DB
/** Query like this:
* ALTER TABLE [db.]name [ON CLUSTER cluster]
* [ADD COLUMN col_name type [AFTER col_after],]
* [DROP COLUMN col_drop, ...]
* [MODIFY COLUMN col_modify type, ...]
* [DROP COLUMN col_to_drop, ...]
* [CLEAR COLUMN col_to_clear [IN PARTITION partition],]
* [MODIFY COLUMN col_to_modify type, ...]
* [MODIFY PRIMARY KEY (a, b, c...)]
* [DROP|DETACH|ATTACH PARTITION|PART partition, ...]
* [FETCH PARTITION partition FROM ...]

View File

@ -44,8 +44,8 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const
<< source_part_name << "\ninto\n" << new_part_name;
break;
case DROP_COLUMN:
out << "drop_column\n"
case CLEAR_COLUMN:
out << "clear_column\n"
<< escape << column_name
<< "\nfrom\n"
<< new_part_name;
@ -113,9 +113,9 @@ void ReplicatedMergeTreeLogEntryData::readText(ReadBuffer & in)
detach = type_str == "detach";
in >> new_part_name;
}
else if (type_str == "drop_column")
else if (type_str == "clear_column")
{
type = DROP_COLUMN;
type = CLEAR_COLUMN;
in >> escape >> column_name >> "\nfrom\n" >> new_part_name;
}
else if (type_str == "attach")

View File

@ -36,7 +36,7 @@ struct ReplicatedMergeTreeLogEntryData
MERGE_PARTS, /// Merge the parts.
DROP_RANGE, /// Delete the parts in the specified month in the specified number range.
ATTACH_PART, /// Move a part from the `detached` directory.
DROP_COLUMN, /// Drop specific column.
CLEAR_COLUMN, /// Drop specific column from specified partition.
};
String typeToString() const
@ -47,7 +47,7 @@ struct ReplicatedMergeTreeLogEntryData
case ReplicatedMergeTreeLogEntryData::MERGE_PARTS: return "MERGE_PARTS";
case ReplicatedMergeTreeLogEntryData::DROP_RANGE: return "DROP_RANGE";
case ReplicatedMergeTreeLogEntryData::ATTACH_PART: return "ATTACH_PART";
case ReplicatedMergeTreeLogEntryData::DROP_COLUMN: return "DROP_COLUMN";
case ReplicatedMergeTreeLogEntryData::CLEAR_COLUMN: return "CLEAR_COLUMN";
default:
throw Exception("Unknown log entry type: " + DB::toString<int>(type), ErrorCodes::LOGICAL_ERROR);
}

View File

@ -464,7 +464,7 @@ ReplicatedMergeTreeQueue::Queue ReplicatedMergeTreeQueue::getConflictsForClearCo
conflicts.emplace_back(elem);
}
if (elem->type == LogEntry::DROP_COLUMN)
if (elem->type == LogEntry::CLEAR_COLUMN)
{
ActiveDataPartSet::Part cur_part;
ActiveDataPartSet::parsePartName(elem->new_part_name, cur_part);
@ -625,7 +625,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry(
}
}
if (entry.type == LogEntry::DROP_COLUMN)
if (entry.type == LogEntry::CLEAR_COLUMN)
{
String conflicts_description;
if (!getConflictsForClearColumnCommand(entry, &conflicts_description).empty())

View File

@ -1020,7 +1020,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry)
return true;
}
if (entry.type == LogEntry::DROP_COLUMN)
if (entry.type == LogEntry::CLEAR_COLUMN)
{
executeClearColumnFromPartition(entry);
return true;
@ -1561,7 +1561,7 @@ void StorageReplicatedMergeTree::executeClearColumnFromPartition(const LogEntry
if (!ActiveDataPartSet::contains(entry.new_part_name, part->name))
continue;
LOG_DEBUG(log, "Clear column " << entry.column_name << " from part " << part->name);
LOG_DEBUG(log, "Clean column " << entry.column_name << " from part " << part->name);
auto transaction = data.alterDataPart(part, columns_for_parts, data.primary_expr_ast, false);
if (!transaction)
@ -1580,7 +1580,7 @@ void StorageReplicatedMergeTree::executeClearColumnFromPartition(const LogEntry
++modified_parts;
}
LOG_DEBUG(log, "Cleared column " << entry.column_name << " from " << modified_parts << " parts");
LOG_DEBUG(log, "Cleaned column " << entry.column_name << " from " << modified_parts << " parts");
data.recalculateColumnSizes();
}
@ -2687,7 +2687,7 @@ void StorageReplicatedMergeTree::dropColumnFromPartition(
String fake_part_name = getFakePartNameCoveringAllPartsInPartition(month_name);
LogEntry entry;
entry.type = LogEntry::DROP_COLUMN;
entry.type = LogEntry::CLEAR_COLUMN;
entry.new_part_name = fake_part_name;
entry.column_name = column_name.safeGet<String>();
entry.create_time = time(0);

View File

@ -4,7 +4,7 @@ CREATE TABLE test.drop_column (d Date, num Int64, str String) ENGINE = MergeTree
INSERT INTO test.drop_column VALUES ('2016-12-12', 1, 'a'), ('2016-11-12', 2, 'b');
SELECT num, str FROM test.drop_column ORDER BY num;
ALTER TABLE test.drop_column DROP COLUMN num FROM PARTITION '201612';
ALTER TABLE test.drop_column CLEAR COLUMN num IN PARTITION '201612';
SELECT num, str FROM test.drop_column ORDER BY num;
DROP TABLE test.drop_column;
@ -25,16 +25,16 @@ SELECT * FROM test.drop_column1 ORDER BY d, i, s;
SET replication_alter_partitions_sync=2;
SELECT 'w/o i 1';
ALTER TABLE test.drop_column1 DROP COLUMN i FROM PARTITION '200001';
ALTER TABLE test.drop_column1 CLEAR COLUMN i IN PARTITION '200001';
SELECT * FROM test.drop_column2 ORDER BY d, i, s;
SELECT 'w/o is 1';
ALTER TABLE test.drop_column1 DROP COLUMN s FROM PARTITION '200001';
ALTER TABLE test.drop_column1 CLEAR COLUMN s IN PARTITION '200001';
SELECT * FROM test.drop_column2 ORDER BY d, i, s;
SELECT 'w/o is 12';
ALTER TABLE test.drop_column1 DROP COLUMN i FROM PARTITION '200002';
ALTER TABLE test.drop_column1 DROP COLUMN s FROM PARTITION '200002';
ALTER TABLE test.drop_column1 CLEAR COLUMN i IN PARTITION '200002';
ALTER TABLE test.drop_column1 CLEAR COLUMN s IN PARTITION '200002';
SELECT DISTINCT * FROM test.drop_column2 ORDER BY d, i, s;
SELECT DISTINCT * FROM test.drop_column2 ORDER BY d, i, s;
@ -42,8 +42,8 @@ SELECT 'sizes';
SELECT sum(data_uncompressed_bytes) FROM system.columns WHERE database='test' AND table LIKE 'drop_column_' AND (name = 'i' OR name = 's') GROUP BY table;
-- double call should be OK
ALTER TABLE test.drop_column1 DROP COLUMN s FROM PARTITION '200001';
ALTER TABLE test.drop_column1 DROP COLUMN s FROM PARTITION '200002';
ALTER TABLE test.drop_column1 CLEAR COLUMN s IN PARTITION '200001';
ALTER TABLE test.drop_column1 CLEAR COLUMN s IN PARTITION '200002';
DROP TABLE IF EXISTS test.drop_column1;
DROP TABLE IF EXISTS test.drop_column2;

View File

@ -13,10 +13,10 @@ $ch "INSERT INTO test.drop_column1 VALUES ('2000-01-01', 3, 'c'), ('2000-02-01',
for i in `seq 3`; do
$ch "INSERT INTO test.drop_column1 VALUES ('2000-03-01', 3, 'c'), ('2000-03-01', 4, 'd')" &
$ch "ALTER TABLE test.drop_column1 DROP COLUMN i FROM PARTITION '200001'" --replication_alter_partitions_sync=2 &
$ch "ALTER TABLE test.drop_column1 DROP COLUMN s FROM PARTITION '200001'" --replication_alter_partitions_sync=2 &
$ch "ALTER TABLE test.drop_column1 DROP COLUMN i FROM PARTITION '200002'" --replication_alter_partitions_sync=2 &
$ch "ALTER TABLE test.drop_column1 DROP COLUMN s FROM PARTITION '200002'" --replication_alter_partitions_sync=2 &
$ch "ALTER TABLE test.drop_column1 CLEAR COLUMN i IN PARTITION '200001'" --replication_alter_partitions_sync=2 &
$ch "ALTER TABLE test.drop_column1 CLEAR COLUMN s IN PARTITION '200001'" --replication_alter_partitions_sync=2 &
$ch "ALTER TABLE test.drop_column1 CLEAR COLUMN i IN PARTITION '200002'" --replication_alter_partitions_sync=2 &
$ch "ALTER TABLE test.drop_column1 CLEAR COLUMN s IN PARTITION '200002'" --replication_alter_partitions_sync=2 &
$ch "INSERT INTO test.drop_column1 VALUES ('2000-03-01', 3, 'c'), ('2000-03-01', 4, 'd')" &
done
wait