forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-17030][table-planner-blink] Add primary key syntax for ALTER T…
…ABLE This closes apache#11950
- Loading branch information
Showing
10 changed files
with
506 additions
and
4 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
73 changes: 73 additions & 0 deletions
73
...src/main/java/org/apache/flink/table/operations/ddl/AlterTableAddConstraintOperation.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,73 @@ | ||
/* | ||
* 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 | ||
* | ||
* https://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.table.operations.ddl; | ||
|
||
import org.apache.flink.table.catalog.ObjectIdentifier; | ||
import org.apache.flink.table.operations.Operation; | ||
import org.apache.flink.table.operations.OperationUtils; | ||
|
||
import javax.annotation.Nullable; | ||
|
||
import java.util.Collections; | ||
import java.util.LinkedHashMap; | ||
import java.util.Map; | ||
import java.util.Optional; | ||
|
||
/** | ||
* Operation of "ALTER TABLE ADD [CONSTRAINT constraintName] ..." clause. | ||
* | ||
* <p>Note: only primary key is supported now. | ||
*/ | ||
public class AlterTableAddConstraintOperation extends AlterTableOperation { | ||
private final String constraintName; | ||
private final String[] columnNames; | ||
|
||
public AlterTableAddConstraintOperation( | ||
ObjectIdentifier tableIdentifier, | ||
@Nullable String constraintName, | ||
String[] columnNames) { | ||
super(tableIdentifier); | ||
this.constraintName = constraintName; | ||
this.columnNames = columnNames; | ||
} | ||
|
||
public Optional<String> getConstraintName() { | ||
return Optional.ofNullable(constraintName); | ||
} | ||
|
||
public String[] getColumnNames() { | ||
return columnNames; | ||
} | ||
|
||
@Override | ||
public String asSummaryString() { | ||
Map<String, Object> params = new LinkedHashMap<>(); | ||
params.put("identifier", tableIdentifier); | ||
if (getConstraintName().isPresent()) { | ||
params.put("constraintName", this.constraintName); | ||
} | ||
params.put("columns", this.columnNames); | ||
|
||
return OperationUtils.formatWithChildren( | ||
"ALTER TABLE ADD CONSTRAINT", | ||
params, | ||
Collections.emptyList(), | ||
Operation::asSummaryString); | ||
} | ||
} |
43 changes: 43 additions & 0 deletions
43
...rc/main/java/org/apache/flink/table/operations/ddl/AlterTableDropConstraintOperation.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,43 @@ | ||
/* | ||
* 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 | ||
* | ||
* https://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.table.operations.ddl; | ||
|
||
import org.apache.flink.table.catalog.ObjectIdentifier; | ||
|
||
/** Operation of "ALTER TABLE ADD [CONSTRAINT constraintName] ..." clause. **/ | ||
public class AlterTableDropConstraintOperation extends AlterTableOperation { | ||
private final String constraintName; | ||
|
||
public AlterTableDropConstraintOperation( | ||
ObjectIdentifier tableIdentifier, | ||
String constraintName) { | ||
super(tableIdentifier); | ||
this.constraintName = constraintName; | ||
} | ||
|
||
public String getConstraintName() { | ||
return constraintName; | ||
} | ||
|
||
@Override | ||
public String asSummaryString() { | ||
return String.format("ALTER TABLE %s DROP CONSTRAINT %s", | ||
tableIdentifier, constraintName); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
73 changes: 73 additions & 0 deletions
73
...e/flink-table-common/src/test/java/org/apache/flink/table/utils/TableSchemaUtilsTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,73 @@ | ||
/* | ||
* 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 | ||
* | ||
* https://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.table.utils; | ||
|
||
import org.apache.flink.table.api.DataTypes; | ||
import org.apache.flink.table.api.TableSchema; | ||
import org.apache.flink.table.api.ValidationException; | ||
|
||
import org.junit.Rule; | ||
import org.junit.Test; | ||
import org.junit.rules.ExpectedException; | ||
|
||
import static org.junit.Assert.assertEquals; | ||
|
||
/** Tests for TableSchemaUtils. */ | ||
public class TableSchemaUtilsTest { | ||
@Rule | ||
public ExpectedException exceptionRule = ExpectedException.none(); | ||
|
||
@Test | ||
public void testBuilderWithGivenSchema() { | ||
TableSchema oriSchema = TableSchema.builder() | ||
.field("a", DataTypes.INT().notNull()) | ||
.field("b", DataTypes.STRING()) | ||
.field("c", DataTypes.INT(), "a + 1") | ||
.field("t", DataTypes.TIMESTAMP(3)) | ||
.primaryKey("ct1", new String[] {"a"}) | ||
.watermark("t", "t", DataTypes.TIMESTAMP(3)) | ||
.build(); | ||
TableSchema newSchema = TableSchemaUtils.builderWithGivenSchema(oriSchema).build(); | ||
assertEquals(oriSchema, newSchema); | ||
} | ||
|
||
@Test | ||
public void testDropConstraint() { | ||
TableSchema oriSchema = TableSchema.builder() | ||
.field("a", DataTypes.INT().notNull()) | ||
.field("b", DataTypes.STRING()) | ||
.field("c", DataTypes.INT(), "a + 1") | ||
.field("t", DataTypes.TIMESTAMP(3)) | ||
.primaryKey("ct1", new String[] {"a"}) | ||
.watermark("t", "t", DataTypes.TIMESTAMP(3)) | ||
.build(); | ||
TableSchema newSchema = TableSchemaUtils.dropConstraint(oriSchema, "ct1"); | ||
final String expected = "root\n" + | ||
" |-- a: INT NOT NULL\n" + | ||
" |-- b: STRING\n" + | ||
" |-- c: INT AS a + 1\n" + | ||
" |-- t: TIMESTAMP(3)\n" + | ||
" |-- WATERMARK FOR t AS t\n"; | ||
assertEquals(expected, newSchema.toString()); | ||
// Drop non-exist constraint. | ||
exceptionRule.expect(ValidationException.class); | ||
exceptionRule.expectMessage("Constraint ct2 to drop does not exist"); | ||
TableSchemaUtils.dropConstraint(oriSchema, "ct2"); | ||
} | ||
} |
Oops, something went wrong.