Skip to content

Commit

Permalink
[FLINK-34172] Add support for altering a distribution via ALTER TABLE
Browse files Browse the repository at this point in the history
  • Loading branch information
jnh5y authored Jun 13, 2024
1 parent 9d16903 commit d0f9bb4
Show file tree
Hide file tree
Showing 16 changed files with 627 additions and 69 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@
"org.apache.flink.sql.parser.ddl.SqlAlterTableCompact"
"org.apache.flink.sql.parser.ddl.SqlAlterTableDropColumn"
"org.apache.flink.sql.parser.ddl.SqlAlterTableDropConstraint"
"org.apache.flink.sql.parser.ddl.SqlAlterTableDropDistribution"
"org.apache.flink.sql.parser.ddl.SqlAlterTableDropPrimaryKey"
"org.apache.flink.sql.parser.ddl.SqlAlterTableDropWatermark"
"org.apache.flink.sql.parser.ddl.SqlAlterTableModify"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -780,6 +780,7 @@ SqlAlterTable SqlAlterTable() :
SqlNodeList propertyList = SqlNodeList.EMPTY;
SqlNodeList propertyKeyList = SqlNodeList.EMPTY;
SqlNodeList partitionSpec = null;
SqlDistribution distribution = null;
SqlIdentifier constraintName;
SqlTableConstraint constraint;
SqlIdentifier originColumnIdentifier;
Expand Down Expand Up @@ -866,6 +867,7 @@ SqlAlterTable SqlAlterTable() :
new SqlNodeList(ctx.columnPositions, startPos.plus(getPos())),
ctx.constraints,
ctx.watermark,
ctx.distribution,
ifExists);
}
)
Expand All @@ -888,6 +890,7 @@ SqlAlterTable SqlAlterTable() :
new SqlNodeList(ctx.columnPositions, startPos.plus(getPos())),
ctx.constraints,
ctx.watermark,
ctx.distribution,
ifExists);
}

Expand Down Expand Up @@ -938,6 +941,13 @@ SqlAlterTable SqlAlterTable() :
constraintName,
ifExists);
}
|
<DISTRIBUTION> {
return new SqlAlterTableDropDistribution(
startPos.plus(getPos()),
tableIdentifier,
ifExists);
}
|
<WATERMARK> {
return new SqlAlterTableDropWatermark(
Expand Down Expand Up @@ -1177,6 +1187,9 @@ void AlterTableAddOrModify(AlterTableContext context) :
}
|
Watermark(context)
|
<DISTRIBUTION>
context.distribution = SqlDistribution(getPos())
)
}

Expand Down Expand Up @@ -1442,7 +1455,6 @@ SqlDistribution SqlDistribution(SqlParserPos startPos) :
String distributionKind = null;
SqlNumericLiteral bucketCount = null;
SqlNodeList bucketColumns = SqlNodeList.EMPTY;
SqlDistribution distribution = null;
}
{
(
Expand Down Expand Up @@ -1474,11 +1486,7 @@ SqlCreate SqlCreateTable(Span s, boolean replace, boolean isTemporary) :
SqlCharStringLiteral comment = null;
SqlTableLike tableLike = null;
SqlNode asQuery = null;

SqlNodeList propertyList = SqlNodeList.EMPTY;
String distributionKind = null;
SqlNumericLiteral bucketCount = null;
SqlNodeList bucketColumns = SqlNodeList.EMPTY;
SqlDistribution distribution = null;
SqlNodeList partitionColumns = SqlNodeList.EMPTY;
SqlParserPos pos = startPos;
Expand Down Expand Up @@ -1680,9 +1688,6 @@ SqlNode SqlReplaceTable() :
List<SqlTableConstraint> constraints = new ArrayList<SqlTableConstraint>();
SqlWatermark watermark = null;
SqlNodeList columnList = SqlNodeList.EMPTY;
String distributionKind = null;
SqlNumericLiteral bucketCount = null;
SqlNodeList bucketColumns = SqlNodeList.EMPTY;
SqlDistribution distribution = null;
SqlNodeList partitionColumns = SqlNodeList.EMPTY;
boolean ifNotExists = false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@

package org.apache.flink.sql.parser.ddl;

import org.apache.flink.sql.parser.SqlUnparseUtils;
import org.apache.flink.sql.parser.ddl.constraint.SqlTableConstraint;

import org.apache.calcite.sql.SqlIdentifier;
Expand Down Expand Up @@ -57,16 +56,16 @@ public SqlAlterTableAdd(
SqlNodeList addedColumns,
List<SqlTableConstraint> constraint,
@Nullable SqlWatermark sqlWatermark,
@Nullable SqlDistribution distribution,
boolean ifTableExists) {
super(pos, tableName, addedColumns, constraint, sqlWatermark, ifTableExists);
super(pos, tableName, addedColumns, constraint, sqlWatermark, distribution, ifTableExists);
}

@Override
public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
super.unparse(writer, leftPrec, rightPrec);
writer.keyword("ADD");
// unparse table schema
SqlUnparseUtils.unparseTableSchema(
writer, leftPrec, rightPrec, columnList, constraints, watermark);
// unparse table schema and distribution
unparseSchemaAndDistribution(writer, leftPrec, rightPrec);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.sql.parser.ddl;

import org.apache.calcite.sql.SqlIdentifier;
import org.apache.calcite.sql.SqlNode;
import org.apache.calcite.sql.SqlWriter;
import org.apache.calcite.sql.parser.SqlParserPos;

import java.util.Collections;
import java.util.List;

/**
* SqlNode to describe ALTER TABLE [IF EXISTS] table_name DROP DISTRIBUTION clause.
*
* <p>Example: DDL like the below for drop distribution.
*
* <pre>{@code
* -- drop distribution
* ALTER TABLE prod.db.sample DROP DISTRIBUTION;
* }</pre>
*/
public class SqlAlterTableDropDistribution extends SqlAlterTable {

public SqlAlterTableDropDistribution(
SqlParserPos pos, SqlIdentifier tableName, boolean ifTableExists) {
super(pos, tableName, ifTableExists);
}

@Override
public List<SqlNode> getOperandList() {
return Collections.emptyList();
}

@Override
public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
super.unparse(writer, leftPrec, rightPrec);
writer.keyword("DROP DISTRIBUTION");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@

package org.apache.flink.sql.parser.ddl;

import org.apache.flink.sql.parser.SqlUnparseUtils;
import org.apache.flink.sql.parser.ddl.constraint.SqlTableConstraint;

import org.apache.calcite.sql.SqlIdentifier;
Expand Down Expand Up @@ -57,16 +56,16 @@ public SqlAlterTableModify(
SqlNodeList modifiedColumns,
List<SqlTableConstraint> constraints,
@Nullable SqlWatermark watermark,
@Nullable SqlDistribution distribution,
boolean ifTableExists) {
super(pos, tableName, modifiedColumns, constraints, watermark, ifTableExists);
super(pos, tableName, modifiedColumns, constraints, watermark, distribution, ifTableExists);
}

@Override
public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
super.unparse(writer, leftPrec, rightPrec);
writer.keyword("MODIFY");
// unparse table schema
SqlUnparseUtils.unparseTableSchema(
writer, leftPrec, rightPrec, columnList, constraints, watermark);
// unparse table schema and distribution
unparseSchemaAndDistribution(writer, leftPrec, rightPrec);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,13 +20,15 @@

import org.apache.flink.sql.parser.ExtendedSqlNode;
import org.apache.flink.sql.parser.SqlConstraintValidator;
import org.apache.flink.sql.parser.SqlUnparseUtils;
import org.apache.flink.sql.parser.ddl.constraint.SqlTableConstraint;
import org.apache.flink.sql.parser.ddl.position.SqlTableColumnPosition;
import org.apache.flink.sql.parser.error.SqlValidateException;

import org.apache.calcite.sql.SqlIdentifier;
import org.apache.calcite.sql.SqlNode;
import org.apache.calcite.sql.SqlNodeList;
import org.apache.calcite.sql.SqlWriter;
import org.apache.calcite.sql.parser.SqlParserPos;
import org.apache.calcite.util.ImmutableNullableList;

Expand All @@ -42,6 +44,7 @@ public abstract class SqlAlterTableSchema extends SqlAlterTable implements Exten

protected final SqlNodeList columnList;
@Nullable protected final SqlWatermark watermark;
@Nullable protected final SqlDistribution distribution;
protected final List<SqlTableConstraint> constraints;

public SqlAlterTableSchema(
Expand All @@ -50,10 +53,12 @@ public SqlAlterTableSchema(
SqlNodeList columnList,
List<SqlTableConstraint> constraints,
@Nullable SqlWatermark sqlWatermark,
@Nullable SqlDistribution distribution,
boolean ifTableExists) {
super(pos, tableName, ifTableExists);
this.columnList = columnList;
this.constraints = constraints;
this.distribution = distribution;
this.watermark = sqlWatermark;
}

Expand All @@ -80,6 +85,10 @@ public Optional<SqlWatermark> getWatermark() {
return Optional.ofNullable(watermark);
}

public Optional<SqlDistribution> getDistribution() {
return Optional.ofNullable(distribution);
}

public List<SqlTableConstraint> getConstraints() {
return constraints;
}
Expand All @@ -97,4 +106,16 @@ private SqlNodeList getColumns() {
.collect(Collectors.toList()),
SqlParserPos.ZERO);
}

void unparseSchemaAndDistribution(SqlWriter writer, int leftPrec, int rightPrec) {
if ((columnList != null && columnList.size() > 0)
|| (constraints != null && constraints.size() > 0)
|| watermark != null) {
SqlUnparseUtils.unparseTableSchema(
writer, leftPrec, rightPrec, columnList, constraints, watermark);
}
if (distribution != null) {
distribution.unparseAlter(writer, leftPrec, rightPrec);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -296,6 +296,7 @@ public static class TableCreationContext {
public List<SqlNode> columnList = new ArrayList<>();
public List<SqlTableConstraint> constraints = new ArrayList<>();
@Nullable public SqlWatermark watermark;
@Nullable public SqlDistribution distribution;
}

public String[] fullTableName() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,17 +70,28 @@ public List<SqlNode> getOperandList() {

@Override
public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
writer.newlineAndIndent();
unparse(writer, leftPrec, rightPrec, "DISTRIBUTED", true);
}

public void unparseAlter(SqlWriter writer, int leftPrec, int rightPrec) {
unparse(writer, leftPrec, rightPrec, "DISTRIBUTION", false);
}

private void unparse(
SqlWriter writer, int leftPrec, int rightPrec, String keyword, boolean printNewlines) {
if (printNewlines) {
writer.newlineAndIndent();
}

if (bucketColumns.size() == 0 && bucketCount != null) {
writer.keyword("DISTRIBUTED INTO");
writer.keyword(keyword + " INTO");
bucketCount.unparse(writer, leftPrec, rightPrec);
writer.keyword("BUCKETS");
writer.newlineAndIndent();
return;
}

writer.keyword("DISTRIBUTED BY");
writer.keyword(keyword + " BY");
if (distributionKind != null) {
writer.print(distributionKind);
}
Expand All @@ -93,7 +104,9 @@ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
bucketCount.unparse(writer, leftPrec, rightPrec);
writer.keyword("BUCKETS");
}
writer.newlineAndIndent();
if (printNewlines) {
writer.newlineAndIndent();
}
}

public Optional<String> getDistributionKind() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -612,6 +612,59 @@ void testAlterTableAddWatermark() {
.fails("Multiple WATERMARK statements is not supported yet.");
}

@Test
void testAlterTableAddDistribution() {
sql("alter table t1 add DISTRIBUTION BY HASH(a) INTO 6 BUCKETS")
.ok("ALTER TABLE `T1` ADD DISTRIBUTION BY HASH(`A`) INTO 6 BUCKETS");

sql("alter table t1 add DISTRIBUTION BY HASH(a, h) INTO 6 BUCKETS")
.ok("ALTER TABLE `T1` ADD DISTRIBUTION BY HASH(`A`, `H`) INTO 6 BUCKETS");

sql("alter table tbl1 add DISTRIBUTION BY RANGE(a, h) INTO 6 BUCKETS")
.ok("ALTER TABLE `TBL1` ADD DISTRIBUTION BY RANGE(`A`, `H`) INTO 6 BUCKETS");

sql("alter table tbl1 add DISTRIBUTION BY ^RANDOM^(a, h) INTO 6 BUCKETS")
.fails("(?s).*Encountered \"RANDOM\" at line 1, column 38.*");

sql("alter table tbl1 add DISTRIBUTION BY (a, h) INTO 6 BUCKETS")
.ok("ALTER TABLE `TBL1` ADD DISTRIBUTION BY (`A`, `H`) INTO 6 BUCKETS");

sql("alter table tbl1 add DISTRIBUTION BY RANGE(a, h)")
.ok("ALTER TABLE `TBL1` ADD DISTRIBUTION BY RANGE(`A`, `H`)");

sql("alter table tbl1 add DISTRIBUTION BY (a, h)")
.ok("ALTER TABLE `TBL1` ADD DISTRIBUTION BY (`A`, `H`)");
}

@Test
void testAlterTableModifyDistribution() {
sql("alter table t1 modify DISTRIBUTION BY HASH(a) INTO 6 BUCKETS")
.ok("ALTER TABLE `T1` MODIFY DISTRIBUTION BY HASH(`A`) INTO 6 BUCKETS");

sql("alter table tbl1 modify DISTRIBUTION BY HASH(a, h) INTO 6 BUCKETS")
.ok("ALTER TABLE `TBL1` MODIFY DISTRIBUTION BY HASH(`A`, `H`) INTO 6 BUCKETS");

sql("alter table tbl1 modify DISTRIBUTION BY RANGE(a, h) INTO 6 BUCKETS")
.ok("ALTER TABLE `TBL1` MODIFY DISTRIBUTION BY RANGE(`A`, `H`) INTO 6 BUCKETS");

sql("alter table tbl1 modify DISTRIBUTION BY ^RANDOM^(a, h) INTO 6 BUCKETS")
.fails("(?s).*Encountered \"RANDOM\" at line 1, column 41.*");

sql("alter table tbl1 modify DISTRIBUTION BY (a, h) INTO 6 BUCKETS")
.ok("ALTER TABLE `TBL1` MODIFY DISTRIBUTION BY (`A`, `H`) INTO 6 BUCKETS");

sql("alter table tbl1 modify DISTRIBUTION BY RANGE(a, h)")
.ok("ALTER TABLE `TBL1` MODIFY DISTRIBUTION BY RANGE(`A`, `H`)");

sql("alter table tbl1 modify DISTRIBUTION BY (a, h)")
.ok("ALTER TABLE `TBL1` MODIFY DISTRIBUTION BY (`A`, `H`)");
}

@Test
void testAlterTableDropDistribution() {
sql("alter table t1 drop DISTRIBUTION").ok("ALTER TABLE `T1` DROP DISTRIBUTION");
}

@Test
void testAlterTableAddMultipleColumn() {
final String sql1 =
Expand Down Expand Up @@ -970,7 +1023,7 @@ String buildDistributionOutput(final String distributionClause) {

@Test
void testCreateTableWithDistribution() {
final String sql = buildDistributionInput("DISTRIBUTED BY HASH(a, h) INTO 6 BUCKETS\n");
final String sql = buildDistributionInput("DISTRIBUTED BY HASH(a, h) INTO 6 BUCKETS");
final String expected =
buildDistributionOutput("DISTRIBUTED BY HASH(`A`, `H`) INTO 6 BUCKETS\n");
sql(sql).ok(expected);
Expand Down Expand Up @@ -1063,7 +1116,7 @@ void testCreateTableWithDistributionIfNotExists() {
+ " h varchar, \n"
+ " PRIMARY KEY (a, b)\n"
+ ")\n"
+ "DISTRIBUTED BY HASH(a, h) INTO 6 BUCKETS\n"
+ "DISTRIBUTED BY HASH(a, h) INTO 6 BUCKETS"
+ " with (\n"
+ " 'connector' = 'kafka', \n"
+ " 'kafka.topic' = 'log.test'\n"
Expand Down
Loading

0 comments on commit d0f9bb4

Please sign in to comment.