From f8656bd0472f918103fc6baaa758ca5a4cc0d122 Mon Sep 17 00:00:00 2001 From: yuxiqian <34335406+yuxiqian@users.noreply.github.com> Date: Fri, 9 Aug 2024 09:36:40 +0800 Subject: [PATCH] [FLINK-35243] Extends schema change event types support --- .../cdc/common/event/AddColumnEvent.java | 2 +- .../common/event/AlterColumnCommentEvent.java | 149 +++++++++++++ .../common/event/AlterColumnTypeEvent.java | 81 ++++++- .../common/event/AlterTableCommentEvent.java | 77 +++++++ .../common/event/ColumnSchemaChangeEvent.java | 61 ++++++ .../event/ColumnSchemaChangeEventVisitor.java | 31 +++ .../ColumnSchemaChangeEventVisitorVoid.java | 31 +++ .../cdc/common/event/CreateTableEvent.java | 2 +- .../cdc/common/event/DropColumnEvent.java | 2 +- .../cdc/common/event/DropTableEvent.java | 67 ++++++ .../cdc/common/event/RenameColumnEvent.java | 2 +- .../cdc/common/event/SchemaChangeEvent.java | 56 +++++ .../common/event/SchemaChangeEventType.java | 22 +- .../event/SchemaChangeEventTypeFamily.java | 45 ++-- .../event/SchemaChangeEventVisitor.java | 22 ++ .../event/SchemaChangeEventVisitorVoid.java | 22 ++ .../event/SchemaChangeEventWithPreSchema.java | 35 ++++ .../common/event/TableSchemaChangeEvent.java | 58 +++++ .../event/TableSchemaChangeEventVisitor.java | 29 +++ .../TableSchemaChangeEventVisitorVoid.java | 29 +++ .../cdc/common/event/TruncateTableEvent.java | 67 ++++++ .../flink/cdc/common/schema/Column.java | 3 + .../cdc/common/schema/MetadataColumn.java | 5 + .../cdc/common/schema/PhysicalColumn.java | 5 + .../flink/cdc/common/schema/Schema.java | 10 + .../cdc/common/utils/ChangeEventUtils.java | 79 ++++--- .../flink/cdc/common/utils/SchemaUtils.java | 85 ++++++-- .../common/utils/ChangeEventUtilsTest.java | 194 ++++++++++------- .../cdc/common/utils/SchemaUtilsTest.java | 4 +- .../flink/FlinkPipelineComposerITCase.java | 4 +- .../doris/sink/DorisEventSerializer.java | 9 +- .../doris/sink/DorisMetadataApplier.java | 82 ++++++-- .../canal/CanalJsonSerializationSchema.java | 56 ++--- .../DebeziumJsonSerializationSchema.java | 57 ++--- .../CustomAlterTableParserListener.java | 32 +++ .../parser/CustomMySqlAntlrDdlParser.java | 3 +- .../mysql/source/MySqlPipelineITCase.java | 198 +++++++++++++++++- .../paimon/sink/PaimonMetadataApplier.java | 89 ++++++-- .../sink/v2/PaimonRecordEventSerializer.java | 29 +-- .../sink/EventRecordSerializationSchema.java | 20 +- .../sink/StarRocksMetadataApplier.java | 77 +++++-- .../cdc/connectors/values/ValuesDatabase.java | 39 ++++ .../values/sink/ValuesDataSink.java | 22 +- .../values/sink/ValuesDataSinkFunction.java | 22 +- .../cdc/pipeline/tests/MysqlE2eITCase.java | 30 ++- .../cdc/pipeline/tests/RouteE2eITCase.java | 14 +- .../pipeline/tests/SchemaEvolveE2eITCase.java | 6 +- .../schema/coordinator/SchemaDerivation.java | 94 ++++++--- .../schema/coordinator/SchemaManager.java | 60 +++--- .../SchemaRegistryRequestHandler.java | 17 +- .../transform/PostTransformOperator.java | 2 +- .../transform/PreTransformOperator.java | 43 +++- .../AlterColumnCommentEventSerializer.java | 119 +++++++++++ .../event/AlterColumnTypeEventSerializer.java | 9 +- .../AlterTableCommentEventSerializer.java | 108 ++++++++++ .../event/DropTableEventSerializer.java | 108 ++++++++++ .../event/SchemaChangeEventSerializer.java | 168 +++++++++++---- .../event/TruncateTableEventSerializer.java | 108 ++++++++++ .../operators/schema/SchemaEvolveTest.java | 28 ++- .../coordinator/SchemaDerivationTest.java | 4 +- ...AlterColumnCommentEventSerializerTest.java | 64 ++++++ .../AlterColumnTypeEventSerializerTest.java | 9 +- .../AlterTableCommentEventSerializerTest.java | 52 +++++ .../event/DropTableEventSerializerTest.java | 50 +++++ .../TruncateTableEventSerializerTest.java | 51 +++++ 65 files changed, 2710 insertions(+), 448 deletions(-) create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterColumnCommentEvent.java create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterTableCommentEvent.java create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEvent.java create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEventVisitor.java create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEventVisitorVoid.java create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/DropTableEvent.java create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventVisitor.java create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventVisitorVoid.java create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventWithPreSchema.java create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEvent.java create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEventVisitor.java create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEventVisitorVoid.java create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TruncateTableEvent.java create mode 100644 flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/AlterColumnCommentEventSerializer.java create mode 100644 flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/AlterTableCommentEventSerializer.java create mode 100644 flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/DropTableEventSerializer.java create mode 100644 flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/TruncateTableEventSerializer.java create mode 100644 flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/AlterColumnCommentEventSerializerTest.java create mode 100644 flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/AlterTableCommentEventSerializerTest.java create mode 100644 flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/DropTableEventSerializerTest.java create mode 100644 flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/TruncateTableEventSerializerTest.java diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AddColumnEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AddColumnEvent.java index 3966ff46ff..a54cfe14f8 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AddColumnEvent.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AddColumnEvent.java @@ -31,7 +31,7 @@ * lenient column type changes. */ @PublicEvolving -public final class AddColumnEvent implements SchemaChangeEvent { +public final class AddColumnEvent implements ColumnSchemaChangeEvent { private static final long serialVersionUID = 1L; diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterColumnCommentEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterColumnCommentEvent.java new file mode 100644 index 0000000000..2ee3eaef9f --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterColumnCommentEvent.java @@ -0,0 +1,149 @@ +/* + * 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.cdc.common.event; + +import org.apache.flink.cdc.common.schema.Schema; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * A {@link SchemaChangeEvent} that represents an {@code ALTER COLUMN} DDL, which may contain the + * comment changes. + */ +public class AlterColumnCommentEvent + implements ColumnSchemaChangeEvent, SchemaChangeEventWithPreSchema { + + private static final long serialVersionUID = 1L; + + private final TableId tableId; + + /** key => column name, value => column type after changing. */ + private final Map commentMapping; + + private final Map oldCommentMapping; + + public AlterColumnCommentEvent(TableId tableId, Map commentMapping) { + this.tableId = tableId; + this.commentMapping = commentMapping; + this.oldCommentMapping = new HashMap<>(); + } + + public AlterColumnCommentEvent( + TableId tableId, + Map commentMapping, + Map oldCommentMapping) { + this.tableId = tableId; + this.commentMapping = commentMapping; + this.oldCommentMapping = oldCommentMapping; + } + + /** Returns the type mapping. */ + public Map getCommentMapping() { + return commentMapping; + } + + public Map getOldCommentMapping() { + return oldCommentMapping; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof AlterColumnCommentEvent)) { + return false; + } + AlterColumnCommentEvent that = (AlterColumnCommentEvent) o; + return Objects.equals(tableId, that.tableId) + && Objects.equals(commentMapping, that.commentMapping) + && Objects.equals(oldCommentMapping, that.oldCommentMapping); + } + + @Override + public int hashCode() { + return Objects.hash(tableId, commentMapping, oldCommentMapping); + } + + @Override + public String toString() { + if (hasPreSchema()) { + return "AlterColumnCommentEvent{" + + "tableId=" + + tableId + + ", commentMapping=" + + commentMapping + + ", oldCommentMapping=" + + oldCommentMapping + + '}'; + } else { + return "AlterColumnCommentEvent{" + + "tableId=" + + tableId + + ", commentMapping=" + + commentMapping + + '}'; + } + } + + @Override + public TableId tableId() { + return tableId; + } + + @Override + public boolean hasPreSchema() { + return !oldCommentMapping.isEmpty(); + } + + @Override + public void fillPreSchema(Schema oldTypeSchema) { + oldCommentMapping.clear(); + oldTypeSchema.getColumns().stream() + .filter(e -> commentMapping.containsKey(e.getName())) + .forEach(e -> oldCommentMapping.put(e.getName(), e.getComment())); + } + + @Override + public boolean trimRedundantChanges() { + if (hasPreSchema()) { + Set redundantlyChangedColumns = + commentMapping.keySet().stream() + .filter( + e -> + Objects.equals( + commentMapping.get(e), + oldCommentMapping.get(e))) + .collect(Collectors.toSet()); + + // Remove redundant alter column type records that doesn't really change the type + commentMapping.keySet().removeAll(redundantlyChangedColumns); + oldCommentMapping.keySet().removeAll(redundantlyChangedColumns); + } + return !commentMapping.isEmpty(); + } + + @Override + public SchemaChangeEventType getType() { + return SchemaChangeEventType.ALTER_COLUMN_COMMENT; + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterColumnTypeEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterColumnTypeEvent.java index a5a104140f..1ace639422 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterColumnTypeEvent.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterColumnTypeEvent.java @@ -17,16 +17,22 @@ package org.apache.flink.cdc.common.event; +import org.apache.flink.cdc.common.schema.Column; +import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.types.DataType; +import java.util.HashMap; import java.util.Map; import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; /** * A {@link SchemaChangeEvent} that represents an {@code ALTER COLUMN} DDL, which may contain the * lenient column type changes. */ -public class AlterColumnTypeEvent implements SchemaChangeEvent { +public class AlterColumnTypeEvent + implements ColumnSchemaChangeEvent, SchemaChangeEventWithPreSchema { private static final long serialVersionUID = 1L; @@ -35,9 +41,21 @@ public class AlterColumnTypeEvent implements SchemaChangeEvent { /** key => column name, value => column type after changing. */ private final Map typeMapping; + private final Map oldTypeMapping; + public AlterColumnTypeEvent(TableId tableId, Map typeMapping) { this.tableId = tableId; this.typeMapping = typeMapping; + this.oldTypeMapping = new HashMap<>(); + } + + public AlterColumnTypeEvent( + TableId tableId, + Map typeMapping, + Map oldTypeMapping) { + this.tableId = tableId; + this.typeMapping = typeMapping; + this.oldTypeMapping = oldTypeMapping; } /** Returns the type mapping. */ @@ -55,22 +73,34 @@ public boolean equals(Object o) { } AlterColumnTypeEvent that = (AlterColumnTypeEvent) o; return Objects.equals(tableId, that.tableId) - && Objects.equals(typeMapping, that.typeMapping); + && Objects.equals(typeMapping, that.typeMapping) + && Objects.equals(oldTypeMapping, that.oldTypeMapping); } @Override public int hashCode() { - return Objects.hash(tableId, typeMapping); + return Objects.hash(tableId, typeMapping, oldTypeMapping); } @Override public String toString() { - return "AlterColumnTypeEvent{" - + "tableId=" - + tableId - + ", nameMapping=" - + typeMapping - + '}'; + if (hasPreSchema()) { + return "AlterColumnTypeEvent{" + + "tableId=" + + tableId + + ", typeMapping=" + + typeMapping + + ", oldTypeMapping=" + + oldTypeMapping + + '}'; + } else { + return "AlterColumnTypeEvent{" + + "tableId=" + + tableId + + ", typeMapping=" + + typeMapping + + '}'; + } } @Override @@ -78,6 +108,39 @@ public TableId tableId() { return tableId; } + public Map getOldTypeMapping() { + return oldTypeMapping; + } + + @Override + public boolean hasPreSchema() { + return !oldTypeMapping.isEmpty(); + } + + @Override + public void fillPreSchema(Schema oldTypeSchema) { + oldTypeMapping.clear(); + oldTypeMapping.putAll( + oldTypeSchema.getColumns().stream() + .filter(e -> typeMapping.containsKey(e.getName()) && e.getType() != null) + .collect(Collectors.toMap(Column::getName, Column::getType))); + } + + @Override + public boolean trimRedundantChanges() { + if (hasPreSchema()) { + Set redundantlyChangedColumns = + typeMapping.keySet().stream() + .filter(e -> Objects.equals(typeMapping.get(e), oldTypeMapping.get(e))) + .collect(Collectors.toSet()); + + // Remove redundant alter column type records that doesn't really change the type + typeMapping.keySet().removeAll(redundantlyChangedColumns); + oldTypeMapping.keySet().removeAll(redundantlyChangedColumns); + } + return !typeMapping.isEmpty(); + } + @Override public SchemaChangeEventType getType() { return SchemaChangeEventType.ALTER_COLUMN_TYPE; diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterTableCommentEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterTableCommentEvent.java new file mode 100644 index 0000000000..be21f08fde --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterTableCommentEvent.java @@ -0,0 +1,77 @@ +/* + * 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.cdc.common.event; + +import java.util.Objects; + +/** A {@link SchemaChangeEvent} that represents an {@code ALTER TABLE COMMENT = ...} DDL. */ +public class AlterTableCommentEvent implements TableSchemaChangeEvent { + private static final long serialVersionUID = 1L; + + private final TableId tableId; + + /** key => column name, value => column type after changing. */ + private final String tableComment; + + public AlterTableCommentEvent(TableId tableId, String tableComment) { + this.tableId = tableId; + this.tableComment = tableComment; + } + + public String getTableComment() { + return tableComment; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof AlterTableCommentEvent)) { + return false; + } + AlterTableCommentEvent that = (AlterTableCommentEvent) o; + return Objects.equals(tableId, that.tableId) + && Objects.equals(tableComment, that.tableComment); + } + + @Override + public int hashCode() { + return Objects.hash(tableId, tableComment); + } + + @Override + public String toString() { + return "AlterTableCommentEvent{" + + "tableId=" + + tableId + + ", tableComment=" + + tableComment + + '}'; + } + + @Override + public TableId tableId() { + return tableId; + } + + @Override + public SchemaChangeEventType getType() { + return SchemaChangeEventType.ALTER_TABLE_COMMENT; + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEvent.java new file mode 100644 index 0000000000..996d4327bb --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEvent.java @@ -0,0 +1,61 @@ +/* + * 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.cdc.common.event; + +/** Schema change events on column-level. */ +public interface ColumnSchemaChangeEvent extends SchemaChangeEvent { + default void visit(ColumnSchemaChangeEventVisitorVoid visitor) { + try { + if (this instanceof AddColumnEvent) { + visitor.visit((AddColumnEvent) this); + } else if (this instanceof AlterColumnCommentEvent) { + visitor.visit((AlterColumnCommentEvent) this); + } else if (this instanceof AlterColumnTypeEvent) { + visitor.visit((AlterColumnTypeEvent) this); + } else if (this instanceof DropColumnEvent) { + visitor.visit((DropColumnEvent) this); + } else if (this instanceof RenameColumnEvent) { + visitor.visit((RenameColumnEvent) this); + } else { + throw new IllegalArgumentException("Unknown schema change event type " + getType()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + default T visit(ColumnSchemaChangeEventVisitor visitor) { + try { + if (this instanceof AddColumnEvent) { + return visitor.visit((AddColumnEvent) this); + } else if (this instanceof AlterColumnCommentEvent) { + return visitor.visit((AlterColumnCommentEvent) this); + } else if (this instanceof AlterColumnTypeEvent) { + return visitor.visit((AlterColumnTypeEvent) this); + } else if (this instanceof DropColumnEvent) { + return visitor.visit((DropColumnEvent) this); + } else if (this instanceof RenameColumnEvent) { + return visitor.visit((RenameColumnEvent) this); + } else { + throw new IllegalArgumentException("Unknown schema change event type " + getType()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEventVisitor.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEventVisitor.java new file mode 100644 index 0000000000..fb9393e9aa --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEventVisitor.java @@ -0,0 +1,31 @@ +/* + * 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.cdc.common.event; + +/** Visitor class for all {@link ColumnSchemaChangeEvent}s. */ +public interface ColumnSchemaChangeEventVisitor { + T visit(AddColumnEvent event) throws Exception; + + T visit(AlterColumnCommentEvent event) throws Exception; + + T visit(AlterColumnTypeEvent event) throws Exception; + + T visit(DropColumnEvent event) throws Exception; + + T visit(RenameColumnEvent event) throws Exception; +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEventVisitorVoid.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEventVisitorVoid.java new file mode 100644 index 0000000000..0946cee9af --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEventVisitorVoid.java @@ -0,0 +1,31 @@ +/* + * 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.cdc.common.event; + +/** Visitor class for all {@link ColumnSchemaChangeEvent}s and returns nothing. */ +public interface ColumnSchemaChangeEventVisitorVoid { + void visit(AddColumnEvent event) throws Exception; + + void visit(AlterColumnCommentEvent event) throws Exception; + + void visit(AlterColumnTypeEvent event) throws Exception; + + void visit(DropColumnEvent event) throws Exception; + + void visit(RenameColumnEvent event) throws Exception; +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/CreateTableEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/CreateTableEvent.java index 6d3e547172..ebe704a5ef 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/CreateTableEvent.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/CreateTableEvent.java @@ -28,7 +28,7 @@ * {@link DataSource} before all {@link DataChangeEvent} with the same tableId */ @PublicEvolving -public class CreateTableEvent implements SchemaChangeEvent { +public class CreateTableEvent implements TableSchemaChangeEvent { private static final long serialVersionUID = 1L; diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/DropColumnEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/DropColumnEvent.java index 76cabbde7b..264d731d61 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/DropColumnEvent.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/DropColumnEvent.java @@ -27,7 +27,7 @@ * lenient column type changes. */ @PublicEvolving -public class DropColumnEvent implements SchemaChangeEvent { +public class DropColumnEvent implements ColumnSchemaChangeEvent { private static final long serialVersionUID = 1L; diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/DropTableEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/DropTableEvent.java new file mode 100644 index 0000000000..198d49d67e --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/DropTableEvent.java @@ -0,0 +1,67 @@ +/* + * 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.cdc.common.event; + +import org.apache.flink.cdc.common.source.DataSource; + +import java.util.Objects; + +/** + * A {@link SchemaChangeEvent} that represents an {@code DROP TABLE} DDL. this will be sent by + * {@link DataSource} before all {@link DataChangeEvent} with the same tableId. + */ +public class DropTableEvent implements TableSchemaChangeEvent { + private static final long serialVersionUID = 1L; + private final TableId tableId; + + public DropTableEvent(TableId tableId) { + this.tableId = tableId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof DropTableEvent)) { + return false; + } + DropTableEvent that = (DropTableEvent) o; + return Objects.equals(tableId, that.tableId); + } + + @Override + public int hashCode() { + return Objects.hash(tableId); + } + + @Override + public String toString() { + return "DropTableEvent{" + "tableId=" + tableId + '}'; + } + + @Override + public TableId tableId() { + return tableId; + } + + @Override + public SchemaChangeEventType getType() { + return SchemaChangeEventType.DROP_TABLE; + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/RenameColumnEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/RenameColumnEvent.java index 4558c1508c..5980e55154 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/RenameColumnEvent.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/RenameColumnEvent.java @@ -24,7 +24,7 @@ * A {@link SchemaChangeEvent} that represents an {@code RENAME COLUMN} DDL, which may contain the * lenient column type changes. */ -public class RenameColumnEvent implements SchemaChangeEvent { +public class RenameColumnEvent implements ColumnSchemaChangeEvent { private static final long serialVersionUID = 1L; diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEvent.java index d5596e3b3d..63a5fad805 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEvent.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEvent.java @@ -29,4 +29,60 @@ public interface SchemaChangeEvent extends ChangeEvent, Serializable { /** Returns its {@link SchemaChangeEventType}. */ SchemaChangeEventType getType(); + + default void visit(SchemaChangeEventVisitorVoid visitor) { + try { + if (this instanceof AddColumnEvent) { + visitor.visit((AddColumnEvent) this); + } else if (this instanceof AlterColumnCommentEvent) { + visitor.visit((AlterColumnCommentEvent) this); + } else if (this instanceof AlterColumnTypeEvent) { + visitor.visit((AlterColumnTypeEvent) this); + } else if (this instanceof AlterTableCommentEvent) { + visitor.visit((AlterTableCommentEvent) this); + } else if (this instanceof CreateTableEvent) { + visitor.visit((CreateTableEvent) this); + } else if (this instanceof DropColumnEvent) { + visitor.visit((DropColumnEvent) this); + } else if (this instanceof DropTableEvent) { + visitor.visit((DropTableEvent) this); + } else if (this instanceof RenameColumnEvent) { + visitor.visit((RenameColumnEvent) this); + } else if (this instanceof TruncateTableEvent) { + visitor.visit((TruncateTableEvent) this); + } else { + throw new IllegalArgumentException("Unknown schema change event type " + getType()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + default T visit(SchemaChangeEventVisitor visitor) { + try { + if (this instanceof AddColumnEvent) { + return visitor.visit((AddColumnEvent) this); + } else if (this instanceof AlterColumnCommentEvent) { + return visitor.visit((AlterColumnCommentEvent) this); + } else if (this instanceof AlterColumnTypeEvent) { + return visitor.visit((AlterColumnTypeEvent) this); + } else if (this instanceof AlterTableCommentEvent) { + return visitor.visit((AlterTableCommentEvent) this); + } else if (this instanceof CreateTableEvent) { + return visitor.visit((CreateTableEvent) this); + } else if (this instanceof DropColumnEvent) { + return visitor.visit((DropColumnEvent) this); + } else if (this instanceof DropTableEvent) { + return visitor.visit((DropTableEvent) this); + } else if (this instanceof RenameColumnEvent) { + return visitor.visit((RenameColumnEvent) this); + } else if (this instanceof TruncateTableEvent) { + return visitor.visit((TruncateTableEvent) this); + } else { + throw new IllegalArgumentException("Unknown schema change event type " + getType()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } } diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventType.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventType.java index 668ea76a4d..9883eedb9c 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventType.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventType.java @@ -23,22 +23,34 @@ @PublicEvolving public enum SchemaChangeEventType { ADD_COLUMN, + ALTER_COLUMN_COMMENT, ALTER_COLUMN_TYPE, + ALTER_TABLE_COMMENT, CREATE_TABLE, DROP_COLUMN, - RENAME_COLUMN; + DROP_TABLE, + RENAME_COLUMN, + TRUNCATE_TABLE; public static SchemaChangeEventType ofEvent(SchemaChangeEvent event) { if (event instanceof AddColumnEvent) { return ADD_COLUMN; + } else if (event instanceof AlterColumnCommentEvent) { + return ALTER_COLUMN_COMMENT; } else if (event instanceof AlterColumnTypeEvent) { return ALTER_COLUMN_TYPE; + } else if (event instanceof AlterTableCommentEvent) { + return ALTER_TABLE_COMMENT; } else if (event instanceof CreateTableEvent) { return CREATE_TABLE; } else if (event instanceof DropColumnEvent) { return DROP_COLUMN; + } else if (event instanceof DropTableEvent) { + return DROP_TABLE; } else if (event instanceof RenameColumnEvent) { return RENAME_COLUMN; + } else if (event instanceof TruncateTableEvent) { + return TRUNCATE_TABLE; } else { throw new RuntimeException("Unknown schema change event type: " + event.getClass()); } @@ -48,14 +60,22 @@ public static SchemaChangeEventType ofTag(String tag) { switch (tag) { case "add.column": return ADD_COLUMN; + case "alter.column.comment": + return ALTER_COLUMN_COMMENT; case "alter.column.type": return ALTER_COLUMN_TYPE; + case "alter.table.comment": + return ALTER_TABLE_COMMENT; case "create.table": return CREATE_TABLE; case "drop.column": return DROP_COLUMN; + case "drop.table": + return DROP_TABLE; case "rename.column": return RENAME_COLUMN; + case "truncate.table": + return TRUNCATE_TABLE; default: throw new RuntimeException("Unknown schema change event type: " + tag); } diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventTypeFamily.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventTypeFamily.java index 5ff7187f61..73494e453f 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventTypeFamily.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventTypeFamily.java @@ -19,6 +19,16 @@ import org.apache.flink.cdc.common.annotation.PublicEvolving; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ADD_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ALTER_COLUMN_COMMENT; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ALTER_COLUMN_TYPE; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ALTER_TABLE_COMMENT; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.CREATE_TABLE; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.DROP_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.DROP_TABLE; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.RENAME_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.TRUNCATE_TABLE; + /** * An enumeration of schema change event families for clustering {@link SchemaChangeEvent}s into * categories. @@ -26,31 +36,36 @@ @PublicEvolving public class SchemaChangeEventTypeFamily { - public static final SchemaChangeEventType[] ADD = {SchemaChangeEventType.ADD_COLUMN}; + public static final SchemaChangeEventType[] ADD = {ADD_COLUMN}; - public static final SchemaChangeEventType[] ALTER = {SchemaChangeEventType.ALTER_COLUMN_TYPE}; + public static final SchemaChangeEventType[] ALTER = { + ALTER_COLUMN_COMMENT, ALTER_COLUMN_TYPE, ALTER_TABLE_COMMENT + }; - public static final SchemaChangeEventType[] CREATE = {SchemaChangeEventType.CREATE_TABLE}; + public static final SchemaChangeEventType[] CREATE = {CREATE_TABLE}; - public static final SchemaChangeEventType[] DROP = {SchemaChangeEventType.DROP_COLUMN}; + public static final SchemaChangeEventType[] DROP = {DROP_COLUMN, DROP_TABLE}; - public static final SchemaChangeEventType[] RENAME = {SchemaChangeEventType.RENAME_COLUMN}; + public static final SchemaChangeEventType[] RENAME = {RENAME_COLUMN}; - public static final SchemaChangeEventType[] TABLE = {SchemaChangeEventType.CREATE_TABLE}; + public static final SchemaChangeEventType[] TABLE = { + ALTER_TABLE_COMMENT, CREATE_TABLE, DROP_TABLE, TRUNCATE_TABLE + }; public static final SchemaChangeEventType[] COLUMN = { - SchemaChangeEventType.ADD_COLUMN, - SchemaChangeEventType.ALTER_COLUMN_TYPE, - SchemaChangeEventType.DROP_COLUMN, - SchemaChangeEventType.RENAME_COLUMN + ADD_COLUMN, ALTER_COLUMN_COMMENT, ALTER_COLUMN_TYPE, DROP_COLUMN, RENAME_COLUMN }; public static final SchemaChangeEventType[] ALL = { - SchemaChangeEventType.ADD_COLUMN, - SchemaChangeEventType.CREATE_TABLE, - SchemaChangeEventType.ALTER_COLUMN_TYPE, - SchemaChangeEventType.DROP_COLUMN, - SchemaChangeEventType.RENAME_COLUMN + ADD_COLUMN, + ALTER_COLUMN_COMMENT, + ALTER_COLUMN_TYPE, + ALTER_TABLE_COMMENT, + CREATE_TABLE, + DROP_COLUMN, + DROP_TABLE, + RENAME_COLUMN, + TRUNCATE_TABLE }; public static final SchemaChangeEventType[] NONE = {}; diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventVisitor.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventVisitor.java new file mode 100644 index 0000000000..8d79b72f41 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventVisitor.java @@ -0,0 +1,22 @@ +/* + * 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.cdc.common.event; + +/** Visitor clas for all {@link SchemaChangeEvent}s and returns a {@link T}-typed object. */ +public interface SchemaChangeEventVisitor + extends ColumnSchemaChangeEventVisitor, TableSchemaChangeEventVisitor {} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventVisitorVoid.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventVisitorVoid.java new file mode 100644 index 0000000000..9608a14d6f --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventVisitorVoid.java @@ -0,0 +1,22 @@ +/* + * 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.cdc.common.event; + +/** Visitor clas for all {@link SchemaChangeEvent}s and returns nothing. */ +public interface SchemaChangeEventVisitorVoid + extends ColumnSchemaChangeEventVisitorVoid, TableSchemaChangeEventVisitorVoid {} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventWithPreSchema.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventWithPreSchema.java new file mode 100644 index 0000000000..541ae09029 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventWithPreSchema.java @@ -0,0 +1,35 @@ +/* + * 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.cdc.common.event; + +import org.apache.flink.cdc.common.schema.Schema; + +/** A {@link SchemaChangeEvent} that supports appending schema before change event. */ +public interface SchemaChangeEventWithPreSchema extends SchemaChangeEvent { + + /** Describes if this event already has schema before change info. */ + boolean hasPreSchema(); + + /** Append schema before change info to this event. */ + void fillPreSchema(Schema oldSchema); + + /** Check if this event contains redundant schema change request only. */ + default boolean trimRedundantChanges() { + return false; + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEvent.java new file mode 100644 index 0000000000..7374324300 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEvent.java @@ -0,0 +1,58 @@ +/* + * 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.cdc.common.event; + +/** Schema change events on table-level. */ +public interface TableSchemaChangeEvent extends SchemaChangeEvent { + + default void visit(TableSchemaChangeEventVisitorVoid visitor) { + try { + if (this instanceof AlterTableCommentEvent) { + visitor.visit((AlterTableCommentEvent) this); + } else if (this instanceof CreateTableEvent) { + visitor.visit((CreateTableEvent) this); + } else if (this instanceof DropTableEvent) { + visitor.visit((DropTableEvent) this); + } else if (this instanceof TruncateTableEvent) { + visitor.visit((TruncateTableEvent) this); + } else { + throw new IllegalArgumentException("Unknown schema change event type " + getType()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + default T visit(TableSchemaChangeEventVisitor visitor) { + try { + if (this instanceof AlterTableCommentEvent) { + return visitor.visit((AlterTableCommentEvent) this); + } else if (this instanceof CreateTableEvent) { + return visitor.visit((CreateTableEvent) this); + } else if (this instanceof DropTableEvent) { + return visitor.visit((DropTableEvent) this); + } else if (this instanceof TruncateTableEvent) { + return visitor.visit((TruncateTableEvent) this); + } else { + throw new IllegalArgumentException("Unknown schema change event type " + getType()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEventVisitor.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEventVisitor.java new file mode 100644 index 0000000000..1d62ed4e5f --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEventVisitor.java @@ -0,0 +1,29 @@ +/* + * 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.cdc.common.event; + +/** Visitor class for all {@link TableSchemaChangeEvent}s. */ +public interface TableSchemaChangeEventVisitor { + T visit(AlterTableCommentEvent event) throws Exception; + + T visit(CreateTableEvent event) throws Exception; + + T visit(DropTableEvent event) throws Exception; + + T visit(TruncateTableEvent event) throws Exception; +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEventVisitorVoid.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEventVisitorVoid.java new file mode 100644 index 0000000000..f9fbde3018 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEventVisitorVoid.java @@ -0,0 +1,29 @@ +/* + * 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.cdc.common.event; + +/** Visitor class for all {@link TableSchemaChangeEvent}s and returns nothing. */ +public interface TableSchemaChangeEventVisitorVoid { + void visit(AlterTableCommentEvent event) throws Exception; + + void visit(CreateTableEvent event) throws Exception; + + void visit(DropTableEvent event) throws Exception; + + void visit(TruncateTableEvent event) throws Exception; +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TruncateTableEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TruncateTableEvent.java new file mode 100644 index 0000000000..ef13ea6b98 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TruncateTableEvent.java @@ -0,0 +1,67 @@ +/* + * 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.cdc.common.event; + +import org.apache.flink.cdc.common.source.DataSource; + +import java.util.Objects; + +/** + * A {@link SchemaChangeEvent} that represents an {@code TRUNCATE TABLE} DDL. this will be sent by + * {@link DataSource} before all {@link DataChangeEvent} with the same tableId. + */ +public class TruncateTableEvent implements TableSchemaChangeEvent { + private static final long serialVersionUID = 1L; + private final TableId tableId; + + public TruncateTableEvent(TableId tableId) { + this.tableId = tableId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof TruncateTableEvent)) { + return false; + } + TruncateTableEvent that = (TruncateTableEvent) o; + return Objects.equals(tableId, that.tableId); + } + + @Override + public int hashCode() { + return Objects.hash(tableId); + } + + @Override + public String toString() { + return "TruncateTableEvent{" + "tableId=" + tableId + '}'; + } + + @Override + public TableId tableId() { + return tableId; + } + + @Override + public SchemaChangeEventType getType() { + return SchemaChangeEventType.TRUNCATE_TABLE; + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/Column.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/Column.java index 910032c468..f4bf3f4529 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/Column.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/Column.java @@ -142,6 +142,9 @@ public String asSummaryString() { /** Returns a copy of the column with a replaced name. */ public abstract Column copy(String newName); + /** Returns a copy of the column with a replaced comment. */ + public abstract Column copyComment(String newComment); + @Override public boolean equals(Object o) { if (this == o) { diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/MetadataColumn.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/MetadataColumn.java index 89eecf46f8..b1d0d34016 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/MetadataColumn.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/MetadataColumn.java @@ -58,6 +58,11 @@ public Column copy(String newName) { return new MetadataColumn(newName, type, metadataKey, comment); } + @Override + public Column copyComment(String newComment) { + return new MetadataColumn(name, type, metadataKey, newComment); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/PhysicalColumn.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/PhysicalColumn.java index bf711f791d..760fb29774 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/PhysicalColumn.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/PhysicalColumn.java @@ -51,4 +51,9 @@ public Column copy(DataType newType) { public Column copy(String newName) { return new PhysicalColumn(newName, type, comment, defaultValueExpression); } + + @Override + public Column copyComment(String newComment) { + return new PhysicalColumn(name, type, newComment); + } } diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/Schema.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/Schema.java index 2db509e6e6..7c7deacde8 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/Schema.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/Schema.java @@ -168,6 +168,16 @@ public Schema copy(List columns) { comment); } + /** Returns a copy of the schema with replaced comments. */ + public Schema copyComment(String newComment) { + return new Schema( + columns, + new ArrayList<>(primaryKeys), + new ArrayList<>(partitionKeys), + new HashMap<>(options), + newComment); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/ChangeEventUtils.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/ChangeEventUtils.java index ee72ef6e23..731475ceb8 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/ChangeEventUtils.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/ChangeEventUtils.java @@ -19,15 +19,20 @@ import org.apache.flink.cdc.common.annotation.VisibleForTesting; import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnCommentEvent; import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; +import org.apache.flink.cdc.common.event.AlterTableCommentEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DataChangeEvent; import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.DropTableEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.event.SchemaChangeEventType; import org.apache.flink.cdc.common.event.SchemaChangeEventTypeFamily; +import org.apache.flink.cdc.common.event.SchemaChangeEventVisitor; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; import java.util.ArrayList; import java.util.Arrays; @@ -65,30 +70,56 @@ public static DataChangeEvent recreateDataChangeEvent( public static SchemaChangeEvent recreateSchemaChangeEvent( SchemaChangeEvent schemaChangeEvent, TableId tableId) { - if (schemaChangeEvent instanceof CreateTableEvent) { - CreateTableEvent createTableEvent = (CreateTableEvent) schemaChangeEvent; - return new CreateTableEvent(tableId, createTableEvent.getSchema()); - } - if (schemaChangeEvent instanceof AlterColumnTypeEvent) { - AlterColumnTypeEvent alterColumnTypeEvent = (AlterColumnTypeEvent) schemaChangeEvent; - return new AlterColumnTypeEvent(tableId, alterColumnTypeEvent.getTypeMapping()); - } - if (schemaChangeEvent instanceof RenameColumnEvent) { - RenameColumnEvent renameColumnEvent = (RenameColumnEvent) schemaChangeEvent; - return new RenameColumnEvent(tableId, renameColumnEvent.getNameMapping()); - } - if (schemaChangeEvent instanceof DropColumnEvent) { - DropColumnEvent dropColumnEvent = (DropColumnEvent) schemaChangeEvent; - return new DropColumnEvent(tableId, dropColumnEvent.getDroppedColumnNames()); - } - if (schemaChangeEvent instanceof AddColumnEvent) { - AddColumnEvent addColumnEvent = (AddColumnEvent) schemaChangeEvent; - return new AddColumnEvent(tableId, addColumnEvent.getAddedColumns()); - } - throw new UnsupportedOperationException( - String.format( - "Unsupported schema change event with type \"%s\"", - schemaChangeEvent.getClass().getCanonicalName())); + + return schemaChangeEvent.visit( + new SchemaChangeEventVisitor() { + @Override + public SchemaChangeEvent visit(AddColumnEvent event) { + return new AddColumnEvent(tableId, event.getAddedColumns()); + } + + @Override + public SchemaChangeEvent visit(AlterColumnCommentEvent event) { + return new AlterColumnCommentEvent( + tableId, event.getCommentMapping(), event.getOldCommentMapping()); + } + + @Override + public SchemaChangeEvent visit(AlterColumnTypeEvent event) { + return new AlterColumnTypeEvent( + tableId, event.getTypeMapping(), event.getOldTypeMapping()); + } + + @Override + public SchemaChangeEvent visit(AlterTableCommentEvent event) { + return new AlterTableCommentEvent(tableId, event.getTableComment()); + } + + @Override + public SchemaChangeEvent visit(CreateTableEvent event) { + return new CreateTableEvent(tableId, event.getSchema()); + } + + @Override + public SchemaChangeEvent visit(DropColumnEvent event) { + return new DropColumnEvent(tableId, event.getDroppedColumnNames()); + } + + @Override + public SchemaChangeEvent visit(DropTableEvent event) { + return new DropTableEvent(tableId); + } + + @Override + public SchemaChangeEvent visit(RenameColumnEvent event) { + return new RenameColumnEvent(tableId, event.getNameMapping()); + } + + @Override + public SchemaChangeEvent visit(TruncateTableEvent event) { + return new TruncateTableEvent(tableId); + } + }); } public static Set resolveSchemaEvolutionOptions( diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/SchemaUtils.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/SchemaUtils.java index 68cdcd0e1d..0cfbb23838 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/SchemaUtils.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/utils/SchemaUtils.java @@ -21,10 +21,16 @@ import org.apache.flink.cdc.common.annotation.VisibleForTesting; import org.apache.flink.cdc.common.data.RecordData; import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnCommentEvent; import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; +import org.apache.flink.cdc.common.event.AlterTableCommentEvent; +import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.DropTableEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventVisitor; +import org.apache.flink.cdc.common.event.TruncateTableEvent; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.types.DataType; @@ -244,20 +250,53 @@ public static int getNumericPrecision(DataType dataType) { /** apply SchemaChangeEvent to the old schema and return the schema after changing. */ public static Schema applySchemaChangeEvent(Schema schema, SchemaChangeEvent event) { - if (event instanceof AddColumnEvent) { - return applyAddColumnEvent((AddColumnEvent) event, schema); - } else if (event instanceof DropColumnEvent) { - return applyDropColumnEvent((DropColumnEvent) event, schema); - } else if (event instanceof RenameColumnEvent) { - return applyRenameColumnEvent((RenameColumnEvent) event, schema); - } else if (event instanceof AlterColumnTypeEvent) { - return applyAlterColumnTypeEvent((AlterColumnTypeEvent) event, schema); - } else { - throw new UnsupportedOperationException( - String.format( - "Unsupported schema change event type \"%s\"", - event.getClass().getCanonicalName())); - } + return event.visit( + new SchemaChangeEventVisitor() { + @Override + public Schema visit(AddColumnEvent event) { + return applyAddColumnEvent(event, schema); + } + + @Override + public Schema visit(AlterColumnCommentEvent event) { + return applyAlterColumnCommentEvent(event, schema); + } + + @Override + public Schema visit(AlterColumnTypeEvent event) { + return applyAlterColumnTypeEvent(event, schema); + } + + @Override + public Schema visit(AlterTableCommentEvent event) { + return schema.copyComment(event.getTableComment()); + } + + @Override + public Schema visit(CreateTableEvent event) { + return event.getSchema(); + } + + @Override + public Schema visit(DropColumnEvent event) { + return applyDropColumnEvent(event, schema); + } + + @Override + public Schema visit(DropTableEvent event) { + return null; + } + + @Override + public Schema visit(RenameColumnEvent event) { + return applyRenameColumnEvent(event, schema); + } + + @Override + public Schema visit(TruncateTableEvent event) { + return schema; + } + }); } private static Schema applyAddColumnEvent(AddColumnEvent event, Schema oldSchema) { @@ -352,4 +391,22 @@ private static Schema applyAlterColumnTypeEvent(AlterColumnTypeEvent event, Sche }); return oldSchema.copy(columns); } + + private static Schema applyAlterColumnCommentEvent( + AlterColumnCommentEvent event, Schema oldSchema) { + List columns = new ArrayList<>(); + oldSchema + .getColumns() + .forEach( + column -> { + if (event.getCommentMapping().containsKey(column.getName())) { + columns.add( + column.copyComment( + event.getCommentMapping().get(column.getName()))); + } else { + columns.add(column); + } + }); + return oldSchema.copy(columns); + } } diff --git a/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/ChangeEventUtilsTest.java b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/ChangeEventUtilsTest.java index 0a34a75018..49d3873d67 100644 --- a/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/ChangeEventUtilsTest.java +++ b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/ChangeEventUtilsTest.java @@ -17,102 +17,140 @@ package org.apache.flink.cdc.common.utils; -import org.assertj.core.api.Assertions; import org.assertj.core.util.Sets; -import org.junit.jupiter.api.Test; +import org.junit.Assert; +import org.junit.Test; import java.util.Arrays; import java.util.Collections; import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ADD_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ALTER_COLUMN_COMMENT; import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ALTER_COLUMN_TYPE; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ALTER_TABLE_COMMENT; import static org.apache.flink.cdc.common.event.SchemaChangeEventType.CREATE_TABLE; import static org.apache.flink.cdc.common.event.SchemaChangeEventType.DROP_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.DROP_TABLE; import static org.apache.flink.cdc.common.event.SchemaChangeEventType.RENAME_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.TRUNCATE_TABLE; /** A test for the {@link org.apache.flink.cdc.common.utils.ChangeEventUtils}. */ public class ChangeEventUtilsTest { @Test public void testResolveSchemaEvolutionOptions() { - Assertions.assertThat( - ChangeEventUtils.resolveSchemaEvolutionOptions( - Collections.emptyList(), Collections.emptyList())) - .isEqualTo( - Sets.set( - CREATE_TABLE, - ADD_COLUMN, - ALTER_COLUMN_TYPE, - DROP_COLUMN, - RENAME_COLUMN)); - - Assertions.assertThat( - ChangeEventUtils.resolveSchemaEvolutionOptions( - Collections.emptyList(), Collections.singletonList("drop"))) - .isEqualTo(Sets.set(CREATE_TABLE, ADD_COLUMN, ALTER_COLUMN_TYPE, RENAME_COLUMN)); - - Assertions.assertThat( - ChangeEventUtils.resolveSchemaEvolutionOptions( - Arrays.asList("create", "add"), Collections.emptyList())) - .isEqualTo(Sets.set(CREATE_TABLE, ADD_COLUMN)); - - Assertions.assertThat( - ChangeEventUtils.resolveSchemaEvolutionOptions( - Collections.singletonList("column"), - Collections.singletonList("drop.column"))) - .isEqualTo(Sets.set(ADD_COLUMN, ALTER_COLUMN_TYPE, RENAME_COLUMN)); - - Assertions.assertThat( - ChangeEventUtils.resolveSchemaEvolutionOptions( - Collections.emptyList(), Collections.singletonList("drop.column"))) - .isEqualTo(Sets.set(CREATE_TABLE, ADD_COLUMN, ALTER_COLUMN_TYPE, RENAME_COLUMN)); + Assert.assertEquals( + Sets.set( + ALTER_COLUMN_COMMENT, + TRUNCATE_TABLE, + RENAME_COLUMN, + CREATE_TABLE, + DROP_TABLE, + ALTER_COLUMN_TYPE, + ALTER_TABLE_COMMENT, + ADD_COLUMN, + DROP_COLUMN), + ChangeEventUtils.resolveSchemaEvolutionOptions( + Collections.emptyList(), Collections.emptyList())); + + Assert.assertEquals( + Sets.set( + ADD_COLUMN, + ALTER_TABLE_COMMENT, + ALTER_COLUMN_COMMENT, + ALTER_COLUMN_TYPE, + RENAME_COLUMN, + CREATE_TABLE, + TRUNCATE_TABLE), + ChangeEventUtils.resolveSchemaEvolutionOptions( + Collections.emptyList(), Collections.singletonList("drop"))); + + Assert.assertEquals( + Sets.set(ADD_COLUMN, CREATE_TABLE), + ChangeEventUtils.resolveSchemaEvolutionOptions( + Arrays.asList("create", "add"), Collections.emptyList())); + + Assert.assertEquals( + Sets.set(ADD_COLUMN, ALTER_COLUMN_COMMENT, ALTER_COLUMN_TYPE, RENAME_COLUMN), + ChangeEventUtils.resolveSchemaEvolutionOptions( + Collections.singletonList("column"), + Collections.singletonList("drop.column"))); + + Assert.assertEquals( + Sets.set( + ADD_COLUMN, + ALTER_COLUMN_COMMENT, + DROP_TABLE, + TRUNCATE_TABLE, + RENAME_COLUMN, + ALTER_COLUMN_TYPE, + ALTER_TABLE_COMMENT, + CREATE_TABLE), + ChangeEventUtils.resolveSchemaEvolutionOptions( + Collections.emptyList(), Collections.singletonList("drop.column"))); } @Test public void testResolveSchemaEvolutionTag() { - Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("all")) - .isEqualTo( - Arrays.asList( - ADD_COLUMN, - CREATE_TABLE, - ALTER_COLUMN_TYPE, - DROP_COLUMN, - RENAME_COLUMN)); - - Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("column")) - .isEqualTo( - Arrays.asList(ADD_COLUMN, ALTER_COLUMN_TYPE, DROP_COLUMN, RENAME_COLUMN)); - - Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("table")) - .isEqualTo(Collections.singletonList(CREATE_TABLE)); - - Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("rename")) - .isEqualTo(Collections.singletonList(RENAME_COLUMN)); - - Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("rename.column")) - .isEqualTo(Collections.singletonList(RENAME_COLUMN)); - - Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("drop")) - .isEqualTo(Collections.singletonList(DROP_COLUMN)); - - Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("drop.column")) - .isEqualTo(Collections.singletonList(DROP_COLUMN)); - - Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("create")) - .isEqualTo(Collections.singletonList(CREATE_TABLE)); - - Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("create.table")) - .isEqualTo(Collections.singletonList(CREATE_TABLE)); - - Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("alter")) - .isEqualTo(Collections.singletonList(ALTER_COLUMN_TYPE)); - - Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("alter.column.type")) - .isEqualTo(Collections.singletonList(ALTER_COLUMN_TYPE)); - - Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("add")) - .isEqualTo(Collections.singletonList(ADD_COLUMN)); - - Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("add.column")) - .isEqualTo(Collections.singletonList(ADD_COLUMN)); + Assert.assertEquals( + Arrays.asList( + ADD_COLUMN, + ALTER_COLUMN_COMMENT, + ALTER_COLUMN_TYPE, + ALTER_TABLE_COMMENT, + CREATE_TABLE, + DROP_COLUMN, + DROP_TABLE, + RENAME_COLUMN, + TRUNCATE_TABLE), + ChangeEventUtils.resolveSchemaEvolutionTag("all")); + + Assert.assertEquals( + Arrays.asList( + ADD_COLUMN, + ALTER_COLUMN_COMMENT, + ALTER_COLUMN_TYPE, + DROP_COLUMN, + RENAME_COLUMN), + ChangeEventUtils.resolveSchemaEvolutionTag("column")); + + Assert.assertEquals( + Arrays.asList(ALTER_TABLE_COMMENT, CREATE_TABLE, DROP_TABLE, TRUNCATE_TABLE), + ChangeEventUtils.resolveSchemaEvolutionTag("table")); + + Assert.assertEquals( + Collections.singletonList(RENAME_COLUMN), + ChangeEventUtils.resolveSchemaEvolutionTag("rename.column")); + + Assert.assertEquals( + Arrays.asList(DROP_COLUMN, DROP_TABLE), + ChangeEventUtils.resolveSchemaEvolutionTag("drop")); + + Assert.assertEquals( + Collections.singletonList(DROP_COLUMN), + ChangeEventUtils.resolveSchemaEvolutionTag("drop.column")); + + Assert.assertEquals( + Collections.singletonList(CREATE_TABLE), + ChangeEventUtils.resolveSchemaEvolutionTag("create")); + + Assert.assertEquals( + Collections.singletonList(CREATE_TABLE), + ChangeEventUtils.resolveSchemaEvolutionTag("create.table")); + + Assert.assertEquals( + Arrays.asList(ALTER_COLUMN_COMMENT, ALTER_COLUMN_TYPE, ALTER_TABLE_COMMENT), + ChangeEventUtils.resolveSchemaEvolutionTag("alter")); + + Assert.assertEquals( + Collections.singletonList(ALTER_COLUMN_TYPE), + ChangeEventUtils.resolveSchemaEvolutionTag("alter.column.type")); + + Assert.assertEquals( + Collections.singletonList(ADD_COLUMN), + ChangeEventUtils.resolveSchemaEvolutionTag("add")); + + Assert.assertEquals( + Collections.singletonList(ADD_COLUMN), + ChangeEventUtils.resolveSchemaEvolutionTag("add.column")); } } diff --git a/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/SchemaUtilsTest.java b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/SchemaUtilsTest.java index 8a508a8908..96f3385f65 100644 --- a/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/SchemaUtilsTest.java +++ b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/SchemaUtilsTest.java @@ -28,7 +28,7 @@ import org.apache.flink.cdc.common.types.DataTypes; import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.Test; +import org.junit.Test; import java.util.ArrayList; import java.util.Arrays; @@ -40,7 +40,7 @@ public class SchemaUtilsTest { @Test - public void testApplySchemaChangeEvent() { + public void testApplyColumnSchemaChangeEvent() { TableId tableId = TableId.parse("default.default.table1"); Schema schema = Schema.newBuilder() diff --git a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineComposerITCase.java b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineComposerITCase.java index 39c55a6f86..bc005c551c 100644 --- a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineComposerITCase.java +++ b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/FlinkPipelineComposerITCase.java @@ -718,7 +718,7 @@ void testMergingWithRoute() throws Exception { "DataChangeEvent{tableId=default_namespace.default_schema.merged, before=[], after=[2, Bob, 20], op=INSERT, meta=()}", "DataChangeEvent{tableId=default_namespace.default_schema.merged, before=[2, Bob, 20], after=[2, Bob, 30], op=UPDATE, meta=()}", "AddColumnEvent{tableId=default_namespace.default_schema.merged, addedColumns=[ColumnWithPosition{column=`description` STRING, position=LAST, existedColumnName=null}]}", - "AlterColumnTypeEvent{tableId=default_namespace.default_schema.merged, nameMapping={age=BIGINT, id=BIGINT}}", + "AlterColumnTypeEvent{tableId=default_namespace.default_schema.merged, typeMapping={age=BIGINT, id=BIGINT}, oldTypeMapping={age=INT, id=INT}}", "DataChangeEvent{tableId=default_namespace.default_schema.merged, before=[], after=[3, Charlie, 15, student], op=INSERT, meta=()}", "DataChangeEvent{tableId=default_namespace.default_schema.merged, before=[], after=[4, Donald, 25, student], op=INSERT, meta=()}", "DataChangeEvent{tableId=default_namespace.default_schema.merged, before=[4, Donald, 25, student], after=[], op=DELETE, meta=()}", @@ -918,7 +918,7 @@ void testTransformMergingWithRoute() throws Exception { "DataChangeEvent{tableId=default_namespace.default_schema.merged, before=[], after=[2, Bob, 20, last_name], op=INSERT, meta=()}", "DataChangeEvent{tableId=default_namespace.default_schema.merged, before=[2, Bob, 20, last_name], after=[2, Bob, 30, last_name], op=UPDATE, meta=()}", "AddColumnEvent{tableId=default_namespace.default_schema.merged, addedColumns=[ColumnWithPosition{column=`description` STRING, position=LAST, existedColumnName=null}]}", - "AlterColumnTypeEvent{tableId=default_namespace.default_schema.merged, nameMapping={age=BIGINT, id=BIGINT}}", + "AlterColumnTypeEvent{tableId=default_namespace.default_schema.merged, typeMapping={age=BIGINT, id=BIGINT}, oldTypeMapping={age=INT, id=INT}}", "DataChangeEvent{tableId=default_namespace.default_schema.merged, before=[], after=[3, Charlie, 15, last_name, student], op=INSERT, meta=()}", "DataChangeEvent{tableId=default_namespace.default_schema.merged, before=[], after=[4, Donald, 25, last_name, student], op=INSERT, meta=()}", "DataChangeEvent{tableId=default_namespace.default_schema.merged, before=[4, Donald, 25, last_name, student], after=[], op=DELETE, meta=()}", diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisEventSerializer.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisEventSerializer.java index a1cb9182d1..7b681cf222 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisEventSerializer.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisEventSerializer.java @@ -27,7 +27,6 @@ import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.utils.Preconditions; -import org.apache.flink.cdc.common.utils.SchemaUtils; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; @@ -75,13 +74,7 @@ public DorisRecord serialize(Event event) throws IOException { if (event instanceof CreateTableEvent) { schemaMaps.put(tableId, ((CreateTableEvent) event).getSchema()); } else { - if (!schemaMaps.containsKey(tableId)) { - throw new RuntimeException("schema of " + tableId + " is not existed."); - } - schemaMaps.put( - tableId, - SchemaUtils.applySchemaChangeEvent( - schemaMaps.get(tableId), schemaChangeEvent)); + schemaMaps.remove(tableId); } } return null; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisMetadataApplier.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisMetadataApplier.java index 811d03298d..00fb9be998 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisMetadataApplier.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisMetadataApplier.java @@ -19,15 +19,18 @@ import org.apache.flink.cdc.common.configuration.Configuration; import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnCommentEvent; import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; +import org.apache.flink.cdc.common.event.AlterTableCommentEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.DropTableEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.event.SchemaChangeEventType; +import org.apache.flink.cdc.common.event.SchemaChangeEventVisitorVoid; import org.apache.flink.cdc.common.event.TableId; -import org.apache.flink.cdc.common.exceptions.SchemaEvolveException; -import org.apache.flink.cdc.common.exceptions.UnsupportedSchemaChangeEventException; +import org.apache.flink.cdc.common.event.TruncateTableEvent; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.sink.MetadataApplier; @@ -60,6 +63,7 @@ import java.util.Set; import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ADD_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ALTER_COLUMN_TYPE; import static org.apache.flink.cdc.common.event.SchemaChangeEventType.DROP_COLUMN; import static org.apache.flink.cdc.common.event.SchemaChangeEventType.RENAME_COLUMN; import static org.apache.flink.cdc.connectors.doris.sink.DorisDataSinkOptions.TABLE_CREATE_PROPERTIES_PREFIX; @@ -93,29 +97,63 @@ public boolean acceptsSchemaEvolutionType(SchemaChangeEventType schemaChangeEven @Override public Set getSupportedSchemaEvolutionTypes() { - return Sets.newHashSet(ADD_COLUMN, DROP_COLUMN, RENAME_COLUMN); + return Sets.newHashSet(ADD_COLUMN, ALTER_COLUMN_TYPE, DROP_COLUMN, RENAME_COLUMN); } @Override - public void applySchemaChange(SchemaChangeEvent event) throws SchemaEvolveException { - try { - // send schema change op to doris - if (event instanceof CreateTableEvent) { - applyCreateTableEvent((CreateTableEvent) event); - } else if (event instanceof AddColumnEvent) { - applyAddColumnEvent((AddColumnEvent) event); - } else if (event instanceof DropColumnEvent) { - applyDropColumnEvent((DropColumnEvent) event); - } else if (event instanceof RenameColumnEvent) { - applyRenameColumnEvent((RenameColumnEvent) event); - } else if (event instanceof AlterColumnTypeEvent) { - applyAlterColumnTypeEvent((AlterColumnTypeEvent) event); - } else { - throw new UnsupportedSchemaChangeEventException(event); - } - } catch (Exception ex) { - throw new SchemaEvolveException(event, ex.getMessage(), null); - } + public void applySchemaChange(SchemaChangeEvent event) { + event.visit( + new SchemaChangeEventVisitorVoid() { + + @Override + public void visit(AddColumnEvent event) throws Exception { + applyAddColumnEvent(event); + } + + @Override + public void visit(AlterColumnCommentEvent event) { + throw new UnsupportedOperationException( + "Unsupported schema change event: " + event); + } + + @Override + public void visit(AlterColumnTypeEvent event) throws Exception { + applyAlterColumnTypeEvent(event); + } + + @Override + public void visit(AlterTableCommentEvent event) { + throw new UnsupportedOperationException( + "Unsupported schema change event: " + event); + } + + @Override + public void visit(CreateTableEvent event) throws Exception { + applyCreateTableEvent(event); + } + + @Override + public void visit(DropColumnEvent event) throws Exception { + applyDropColumnEvent(event); + } + + @Override + public void visit(DropTableEvent event) { + throw new UnsupportedOperationException( + "Unsupported schema change event: " + event); + } + + @Override + public void visit(RenameColumnEvent event) throws Exception { + applyRenameColumnEvent(event); + } + + @Override + public void visit(TruncateTableEvent event) { + throw new UnsupportedOperationException( + "Unsupported schema change event: " + event); + } + }); } private void applyCreateTableEvent(CreateTableEvent event) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/canal/CanalJsonSerializationSchema.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/canal/CanalJsonSerializationSchema.java index 0a145cab73..964cb2221b 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/canal/CanalJsonSerializationSchema.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/canal/CanalJsonSerializationSchema.java @@ -18,7 +18,7 @@ package org.apache.flink.cdc.connectors.kafka.json.canal; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.event.DataChangeEvent; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.SchemaChangeEvent; @@ -42,6 +42,7 @@ import java.time.ZoneId; import java.util.HashMap; import java.util.Map; +import java.util.Optional; import static java.lang.String.format; import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; @@ -100,34 +101,37 @@ public void open(InitializationContext context) { @Override public byte[] serialize(Event event) { if (event instanceof SchemaChangeEvent) { - Schema schema; SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; - if (event instanceof CreateTableEvent) { - CreateTableEvent createTableEvent = (CreateTableEvent) event; - schema = createTableEvent.getSchema(); + Tuple2 appliedSchema = + SchemaUtils.applySchemaChangeEvent( + schemaChangeEvent, + Optional.ofNullable( + jsonSerializers.getOrDefault( + schemaChangeEvent.tableId(), null)) + .map(TableSchemaInfo::getSchema) + .orElse(null)); + if (appliedSchema.f1 != null) { + LogicalType rowType = + DataTypeUtils.toFlinkDataType(appliedSchema.f1.toRowDataType()) + .getLogicalType(); + JsonRowDataSerializationSchema jsonSerializer = + new JsonRowDataSerializationSchema( + createJsonRowType(fromLogicalToDataType(rowType)), + timestampFormat, + mapNullKeyMode, + mapNullKeyLiteral, + encodeDecimalAsPlainNumber); + try { + jsonSerializer.open(context); + } catch (Exception e) { + throw new RuntimeException(e); + } + jsonSerializers.put( + appliedSchema.f0, + new TableSchemaInfo(appliedSchema.f1, jsonSerializer, zoneId)); } else { - schema = - SchemaUtils.applySchemaChangeEvent( - jsonSerializers.get(schemaChangeEvent.tableId()).getSchema(), - schemaChangeEvent); + jsonSerializers.remove(appliedSchema.f0); } - LogicalType rowType = - DataTypeUtils.toFlinkDataType(schema.toRowDataType()).getLogicalType(); - JsonRowDataSerializationSchema jsonSerializer = - new JsonRowDataSerializationSchema( - createJsonRowType(fromLogicalToDataType(rowType)), - timestampFormat, - mapNullKeyMode, - mapNullKeyLiteral, - encodeDecimalAsPlainNumber); - try { - jsonSerializer.open(context); - } catch (Exception e) { - throw new RuntimeException(e); - } - jsonSerializers.put( - schemaChangeEvent.tableId(), - new TableSchemaInfo(schema, jsonSerializer, zoneId)); return null; } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/debezium/DebeziumJsonSerializationSchema.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/debezium/DebeziumJsonSerializationSchema.java index ce8afc0dbe..284f886764 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/debezium/DebeziumJsonSerializationSchema.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/debezium/DebeziumJsonSerializationSchema.java @@ -18,7 +18,7 @@ package org.apache.flink.cdc.connectors.kafka.json.debezium; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.event.DataChangeEvent; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.SchemaChangeEvent; @@ -40,6 +40,7 @@ import java.time.ZoneId; import java.util.HashMap; import java.util.Map; +import java.util.Optional; import static java.lang.String.format; import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; @@ -99,34 +100,38 @@ public void open(InitializationContext context) { @Override public byte[] serialize(Event event) { if (event instanceof SchemaChangeEvent) { - Schema schema; SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; - if (event instanceof CreateTableEvent) { - CreateTableEvent createTableEvent = (CreateTableEvent) event; - schema = createTableEvent.getSchema(); + Tuple2 appliedSchema = + SchemaUtils.applySchemaChangeEvent( + schemaChangeEvent, + Optional.ofNullable( + jsonSerializers.getOrDefault( + schemaChangeEvent.tableId(), null)) + .map(TableSchemaInfo::getSchema) + .orElse(null)); + + if (appliedSchema.f1 != null) { + LogicalType rowType = + DataTypeUtils.toFlinkDataType(appliedSchema.f1.toRowDataType()) + .getLogicalType(); + JsonRowDataSerializationSchema jsonSerializer = + new JsonRowDataSerializationSchema( + createJsonRowType(fromLogicalToDataType(rowType)), + timestampFormat, + mapNullKeyMode, + mapNullKeyLiteral, + encodeDecimalAsPlainNumber); + try { + jsonSerializer.open(context); + } catch (Exception e) { + throw new RuntimeException(e); + } + jsonSerializers.put( + appliedSchema.f0, + new TableSchemaInfo(appliedSchema.f1, jsonSerializer, zoneId)); } else { - schema = - SchemaUtils.applySchemaChangeEvent( - jsonSerializers.get(schemaChangeEvent.tableId()).getSchema(), - schemaChangeEvent); + jsonSerializers.remove(appliedSchema.f0, null); } - LogicalType rowType = - DataTypeUtils.toFlinkDataType(schema.toRowDataType()).getLogicalType(); - JsonRowDataSerializationSchema jsonSerializer = - new JsonRowDataSerializationSchema( - createJsonRowType(fromLogicalToDataType(rowType)), - timestampFormat, - mapNullKeyMode, - mapNullKeyLiteral, - encodeDecimalAsPlainNumber); - try { - jsonSerializer.open(context); - } catch (Exception e) { - throw new RuntimeException(e); - } - jsonSerializers.put( - schemaChangeEvent.tableId(), - new TableSchemaInfo(schema, jsonSerializer, zoneId)); return null; } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/source/parser/CustomAlterTableParserListener.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/source/parser/CustomAlterTableParserListener.java index 79583d83e0..87a384857b 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/source/parser/CustomAlterTableParserListener.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/source/parser/CustomAlterTableParserListener.java @@ -18,10 +18,13 @@ package org.apache.flink.cdc.connectors.mysql.source.parser; import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnCommentEvent; import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.DropTableEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.TruncateTableEvent; import org.apache.flink.cdc.common.types.DataType; import io.debezium.connector.mysql.antlr.MySqlAntlrDdlParser; @@ -206,6 +209,10 @@ public void exitAlterByChangeColumn(MySqlParser.AlterByChangeColumnContext ctx) typeMapping.put(column.name(), fromDbzColumn(column)); changes.add(new AlterColumnTypeEvent(currentTable, typeMapping)); + Map commentMapping = new HashMap<>(); + commentMapping.put(column.name(), column.comment()); + changes.add(new AlterColumnCommentEvent(currentTable, commentMapping)); + if (newColumnName != null && !column.name().equalsIgnoreCase(newColumnName)) { Map renameMap = new HashMap<>(); renameMap.put(column.name(), newColumnName); @@ -251,9 +258,15 @@ public void exitAlterByModifyColumn(MySqlParser.AlterByModifyColumnContext ctx) parser.runIfNotNull( () -> { Column column = columnDefinitionListener.getColumn(); + Map typeMapping = new HashMap<>(); typeMapping.put(column.name(), fromDbzColumn(column)); changes.add(new AlterColumnTypeEvent(currentTable, typeMapping)); + + Map commentMapping = new HashMap<>(); + commentMapping.put(column.name(), column.comment()); + changes.add(new AlterColumnCommentEvent(currentTable, commentMapping)); + listeners.remove(columnDefinitionListener); }, columnDefinitionListener); @@ -277,6 +290,25 @@ public void exitAlterByRenameColumn(MySqlParser.AlterByRenameColumnContext ctx) super.exitAlterByRenameColumn(ctx); } + @Override + public void exitTruncateTable(MySqlParser.TruncateTableContext ctx) { + TableId tableId = parser.parseQualifiedTableId(ctx.tableName().fullId()); + changes.add(new TruncateTableEvent(toCdcTableId(tableId))); + super.exitTruncateTable(ctx); + } + + @Override + public void exitDropTable(MySqlParser.DropTableContext ctx) { + ctx.tables() + .tableName() + .forEach( + evt -> { + TableId tableId = parser.parseQualifiedTableId(evt.fullId()); + changes.add(new DropTableEvent(toCdcTableId(tableId))); + }); + super.exitDropTable(ctx); + } + private org.apache.flink.cdc.common.schema.Column toCdcColumn(Column dbzColumn) { return org.apache.flink.cdc.common.schema.Column.physicalColumn( dbzColumn.name(), diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/source/parser/CustomMySqlAntlrDdlParser.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/source/parser/CustomMySqlAntlrDdlParser.java index 1264aa8d68..ffe4b2079a 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/source/parser/CustomMySqlAntlrDdlParser.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/source/parser/CustomMySqlAntlrDdlParser.java @@ -23,6 +23,7 @@ import io.debezium.antlr.DataTypeResolver; import io.debezium.connector.mysql.antlr.MySqlAntlrDdlParser; import io.debezium.ddl.parser.mysql.generated.MySqlParser; +import io.debezium.relational.Tables; import java.sql.Types; import java.util.ArrayList; @@ -36,7 +37,7 @@ public class CustomMySqlAntlrDdlParser extends MySqlAntlrDdlParser { private final LinkedList parsedEvents; public CustomMySqlAntlrDdlParser() { - super(); + super(true, false, true, null, Tables.TableFilter.includeAll()); this.parsedEvents = new LinkedList<>(); } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlPipelineITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlPipelineITCase.java index bd2c059bbf..348946acca 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlPipelineITCase.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlPipelineITCase.java @@ -21,13 +21,16 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.cdc.common.data.binary.BinaryStringData; import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnCommentEvent; import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DataChangeEvent; import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.DropTableEvent; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.source.FlinkSourceProvider; @@ -61,6 +64,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.stream.Collectors; import java.util.stream.Stream; import static org.apache.flink.cdc.connectors.mysql.source.MySqlDataSourceOptions.SCHEMA_CHANGE_ENABLED; @@ -350,6 +354,154 @@ public void testParseAlterStatement() throws Exception { assertThat(actual).isEqualTo(expected); } + @Test + public void testSchemaChangeEvents() throws Exception { + env.setParallelism(1); + inventoryDatabase.createAndInitialize(); + MySqlSourceConfigFactory configFactory = + new MySqlSourceConfigFactory() + .hostname(MYSQL8_CONTAINER.getHost()) + .port(MYSQL8_CONTAINER.getDatabasePort()) + .username(TEST_USER) + .password(TEST_PASSWORD) + .databaseList(inventoryDatabase.getDatabaseName()) + .tableList(inventoryDatabase.getDatabaseName() + ".*") + .startupOptions(StartupOptions.latest()) + .serverId(getServerId(env.getParallelism())) + .serverTimeZone("UTC") + .includeSchemaChanges(SCHEMA_CHANGE_ENABLED.defaultValue()); + + FlinkSourceProvider sourceProvider = + (FlinkSourceProvider) new MySqlDataSource(configFactory).getEventSourceProvider(); + CloseableIterator events = + env.fromSource( + sourceProvider.getSource(), + WatermarkStrategy.noWatermarks(), + MySqlDataSourceFactory.IDENTIFIER, + new EventTypeInfo()) + .executeAndCollect(); + Thread.sleep(5_000); + + List expected = + new ArrayList<>( + getInventoryCreateAllTableEvents(inventoryDatabase.getDatabaseName())); + + try (Connection connection = inventoryDatabase.getJdbcConnection(); + Statement statement = connection.createStatement()) { + + statement.execute( + String.format( + "ALTER TABLE `%s`.`customers` ADD COLUMN `newcol1` INT NULL;", + inventoryDatabase.getDatabaseName())); + expected.add( + new AddColumnEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "customers"), + Collections.singletonList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn("newcol1", DataTypes.INT()))))); + + // Test MODIFY COLUMN DDL + statement.execute( + String.format( + "ALTER TABLE `%s`.`customers` MODIFY COLUMN `newcol1` DOUBLE;", + inventoryDatabase.getDatabaseName())); + + // MySQL MODIFY COLUMN DDL always emits Comment / Type change event at the same time + expected.add( + new AlterColumnCommentEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "customers"), + Collections.singletonMap("newcol1", null))); + + expected.add( + new AlterColumnTypeEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "customers"), + Collections.singletonMap("newcol1", DataTypes.DOUBLE()))); + + statement.execute( + String.format( + "ALTER TABLE `%s`.`customers` MODIFY COLUMN `newcol1` DOUBLE COMMENT 'SomeDescriptiveDescription';", + inventoryDatabase.getDatabaseName())); + + expected.add( + new AlterColumnCommentEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "customers"), + Collections.singletonMap("newcol1", "SomeDescriptiveDescription"))); + + expected.add( + new AlterColumnTypeEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "customers"), + Collections.singletonMap("newcol1", DataTypes.DOUBLE()))); + + // Test CHANGE COLUMN DDL + statement.execute( + String.format( + "ALTER TABLE `%s`.`customers` CHANGE COLUMN `newcol1` `newcol2` INT;", + inventoryDatabase.getDatabaseName())); + + expected.add( + new AlterColumnCommentEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "customers"), + Collections.singletonMap("newcol1", null))); + + expected.add( + new AlterColumnTypeEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "customers"), + Collections.singletonMap("newcol1", DataTypes.INT()))); + + expected.add( + new RenameColumnEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "customers"), + Collections.singletonMap("newcol1", "newcol2"))); + + statement.execute( + String.format( + "ALTER TABLE `%s`.`customers` CHANGE COLUMN `newcol2` `newcol1` DOUBLE COMMENT 'SomeDescriptiveDescription';", + inventoryDatabase.getDatabaseName())); + + expected.add( + new AlterColumnCommentEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "customers"), + Collections.singletonMap("newcol2", "SomeDescriptiveDescription"))); + + expected.add( + new AlterColumnTypeEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "customers"), + Collections.singletonMap("newcol2", DataTypes.DOUBLE()))); + + expected.add( + new RenameColumnEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "customers"), + Collections.singletonMap("newcol2", "newcol1"))); + + // Test truncate table DDL + statement.execute( + String.format( + "TRUNCATE TABLE `%s`.`orders`;", inventoryDatabase.getDatabaseName())); + + expected.add( + new TruncateTableEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "orders"))); + + // Test drop table DDL + statement.execute( + String.format( + "DROP TABLE `%s`.`orders`, `%s`.`customers`;", + inventoryDatabase.getDatabaseName(), + inventoryDatabase.getDatabaseName())); + + expected.add( + new DropTableEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "orders"))); + expected.add( + new DropTableEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "customers"))); + } + List actual = fetchResults(events, expected.size()); + assertEqualsInAnyOrder( + expected.stream().map(Object::toString).collect(Collectors.toList()), + actual.stream().map(Object::toString).collect(Collectors.toList())); + } + private CreateTableEvent getProductsCreateTableEvent(TableId tableId) { return new CreateTableEvent( tableId, @@ -362,6 +514,38 @@ private CreateTableEvent getProductsCreateTableEvent(TableId tableId) { .build()); } + private List getInventoryCreateAllTableEvents(String databaseName) { + return Arrays.asList( + new CreateTableEvent( + TableId.tableId(databaseName, "products"), + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT().notNull()) + .physicalColumn("name", DataTypes.VARCHAR(255).notNull(), "flink") + .physicalColumn("description", DataTypes.VARCHAR(512)) + .physicalColumn("weight", DataTypes.FLOAT()) + .primaryKey(Collections.singletonList("id")) + .build()), + new CreateTableEvent( + TableId.tableId(databaseName, "customers"), + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT().notNull()) + .physicalColumn("first_name", DataTypes.VARCHAR(255).notNull()) + .physicalColumn("last_name", DataTypes.VARCHAR(255).notNull()) + .physicalColumn("email", DataTypes.VARCHAR(255).notNull()) + .primaryKey(Collections.singletonList("id")) + .build()), + new CreateTableEvent( + TableId.tableId(databaseName, "orders"), + Schema.newBuilder() + .physicalColumn("order_number", DataTypes.INT().notNull()) + .physicalColumn("order_date", DataTypes.DATE().notNull()) + .physicalColumn("purchaser", DataTypes.INT().notNull()) + .physicalColumn("quantity", DataTypes.INT().notNull()) + .physicalColumn("product_id", DataTypes.INT().notNull()) + .primaryKey(Collections.singletonList("order_number")) + .build())); + } + private List getSnapshotExpected(TableId tableId) { RowType rowType = RowType.of( @@ -492,16 +676,22 @@ private List executeAlterAndProvideExpected(TableId tableId, Statement st expected.add( new AlterColumnTypeEvent( tableId, Collections.singletonMap("description", DataTypes.VARCHAR(255)))); + expected.add( + new AlterColumnCommentEvent( + tableId, Collections.singletonMap("description", null))); expected.add( new RenameColumnEvent(tableId, Collections.singletonMap("description", "desc"))); statement.execute( String.format( - "ALTER TABLE `%s`.`products` CHANGE COLUMN `desc` `desc2` VARCHAR(400) NULL DEFAULT NULL;", + "ALTER TABLE `%s`.`products` CHANGE COLUMN `desc` `desc2` VARCHAR(400) NULL DEFAULT NULL COMMENT 'JustSomeDesc';", inventoryDatabase.getDatabaseName())); expected.add( new AlterColumnTypeEvent( tableId, Collections.singletonMap("desc", DataTypes.VARCHAR(400)))); + expected.add( + new AlterColumnCommentEvent( + tableId, Collections.singletonMap("desc", "JustSomeDesc"))); expected.add(new RenameColumnEvent(tableId, Collections.singletonMap("desc", "desc2"))); statement.execute( @@ -540,12 +730,15 @@ private List executeAlterAndProvideExpected(TableId tableId, Statement st statement.execute( String.format( - "ALTER TABLE `%s`.`products` DROP COLUMN `desc2`, CHANGE COLUMN `desc1` `desc1` VARCHAR(65) NULL DEFAULT NULL;", + "ALTER TABLE `%s`.`products` DROP COLUMN `desc2`, CHANGE COLUMN `desc1` `desc1` VARCHAR(65) NULL DEFAULT NULL COMMENT 'NewDescription';", inventoryDatabase.getDatabaseName())); expected.add(new DropColumnEvent(tableId, Collections.singletonList("desc2"))); expected.add( new AlterColumnTypeEvent( tableId, Collections.singletonMap("desc1", DataTypes.VARCHAR(65)))); + expected.add( + new AlterColumnCommentEvent( + tableId, Collections.singletonMap("desc1", "NewDescription"))); // Only available in mysql 8.0 statement.execute( @@ -561,6 +754,7 @@ private List executeAlterAndProvideExpected(TableId tableId, Statement st expected.add( new AlterColumnTypeEvent( tableId, Collections.singletonMap("DESC3", DataTypes.VARCHAR(255)))); + expected.add(new AlterColumnCommentEvent(tableId, Collections.singletonMap("DESC3", null))); statement.execute( String.format( diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/PaimonMetadataApplier.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/PaimonMetadataApplier.java index 3ae3c9c4c7..dc918a4103 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/PaimonMetadataApplier.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/PaimonMetadataApplier.java @@ -18,15 +18,20 @@ package org.apache.flink.cdc.connectors.paimon.sink; import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnCommentEvent; import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; +import org.apache.flink.cdc.common.event.AlterTableCommentEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.DropTableEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; +import org.apache.flink.cdc.common.event.RenameTableEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.event.SchemaChangeEventType; +import org.apache.flink.cdc.common.event.SchemaChangeEventVisitorVoid; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; import org.apache.flink.cdc.common.exceptions.SchemaEvolveException; -import org.apache.flink.cdc.common.exceptions.UnsupportedSchemaChangeEventException; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.sink.MetadataApplier; import org.apache.flink.cdc.common.types.utils.DataTypeUtils; @@ -117,23 +122,69 @@ public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) if (catalog == null) { catalog = FlinkCatalogFactory.createPaimonCatalog(catalogOptions); } - try { - if (schemaChangeEvent instanceof CreateTableEvent) { - applyCreateTable((CreateTableEvent) schemaChangeEvent); - } else if (schemaChangeEvent instanceof AddColumnEvent) { - applyAddColumn((AddColumnEvent) schemaChangeEvent); - } else if (schemaChangeEvent instanceof DropColumnEvent) { - applyDropColumn((DropColumnEvent) schemaChangeEvent); - } else if (schemaChangeEvent instanceof RenameColumnEvent) { - applyRenameColumn((RenameColumnEvent) schemaChangeEvent); - } else if (schemaChangeEvent instanceof AlterColumnTypeEvent) { - applyAlterColumn((AlterColumnTypeEvent) schemaChangeEvent); - } else { - throw new UnsupportedSchemaChangeEventException(schemaChangeEvent); - } - } catch (Exception e) { - throw new SchemaEvolveException(schemaChangeEvent, "schema change applying failure", e); - } + schemaChangeEvent.visit( + new SchemaChangeEventVisitorVoid() { + + @Override + public void visit(AddColumnEvent event) throws Exception { + applyAddColumn(event); + } + + @Override + public void visit(AlterColumnCommentEvent event) throws Exception { + throw new UnsupportedOperationException( + "PaimonDataSink doesn't support schema change event " + + schemaChangeEvent); + } + + @Override + public void visit(AlterColumnTypeEvent event) throws Exception { + applyAlterColumnType(event); + } + + @Override + public void visit(AlterTableCommentEvent event) throws Exception { + throw new UnsupportedOperationException( + "PaimonDataSink doesn't support schema change event " + + schemaChangeEvent); + } + + @Override + public void visit(CreateTableEvent event) throws Exception { + applyCreateTable(event); + } + + @Override + public void visit(DropColumnEvent event) throws Exception { + applyDropColumn(event); + } + + @Override + public void visit(DropTableEvent event) throws Exception { + throw new UnsupportedOperationException( + "PaimonDataSink doesn't support schema change event " + + schemaChangeEvent); + } + + @Override + public void visit(RenameColumnEvent event) throws Exception { + applyRenameColumn(event); + } + + @Override + public void visit(RenameTableEvent event) throws Exception { + throw new UnsupportedOperationException( + "PaimonDataSink doesn't support schema change event " + + schemaChangeEvent); + } + + @Override + public void visit(TruncateTableEvent event) throws Exception { + throw new UnsupportedOperationException( + "PaimonDataSink doesn't support schema change event " + + schemaChangeEvent); + } + }); } private void applyCreateTable(CreateTableEvent event) @@ -274,7 +325,7 @@ private void applyRenameColumn(RenameColumnEvent event) true); } - private void applyAlterColumn(AlterColumnTypeEvent event) + private void applyAlterColumnType(AlterColumnTypeEvent event) throws Catalog.ColumnAlreadyExistException, Catalog.TableNotExistException, Catalog.ColumnNotExistException { List tableChangeList = new ArrayList<>(); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonRecordEventSerializer.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonRecordEventSerializer.java index 53b63f3b59..95e90038d0 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonRecordEventSerializer.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonRecordEventSerializer.java @@ -17,12 +17,13 @@ package org.apache.flink.cdc.connectors.paimon.sink.v2; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.event.ChangeEvent; -import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DataChangeEvent; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.utils.SchemaUtils; import org.apache.paimon.catalog.Identifier; @@ -31,6 +32,7 @@ import java.time.ZoneId; import java.util.HashMap; import java.util.Map; +import java.util.Optional; /** * A {@link PaimonRecordSerializer} for converting {@link Event} into {@link PaimonEvent} for {@link @@ -56,20 +58,19 @@ public PaimonEvent serialize(Event event) { ((ChangeEvent) event).tableId().getSchemaName(), ((ChangeEvent) event).tableId().getTableName()); if (event instanceof SchemaChangeEvent) { - if (event instanceof CreateTableEvent) { - CreateTableEvent createTableEvent = (CreateTableEvent) event; - schemaMaps.put( - createTableEvent.tableId(), - new TableSchemaInfo(createTableEvent.getSchema(), zoneId)); + SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; + Tuple2 appliedSchema = + SchemaUtils.applySchemaChangeEvent( + schemaChangeEvent, + Optional.ofNullable( + schemaMaps.getOrDefault( + schemaChangeEvent.tableId(), null)) + .map(TableSchemaInfo::getSchema) + .orElse(null)); + if (appliedSchema.f1 != null) { + schemaMaps.put(appliedSchema.f0, new TableSchemaInfo(appliedSchema.f1, zoneId)); } else { - SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; - schemaMaps.put( - schemaChangeEvent.tableId(), - new TableSchemaInfo( - SchemaUtils.applySchemaChangeEvent( - schemaMaps.get(schemaChangeEvent.tableId()).getSchema(), - schemaChangeEvent), - zoneId)); + schemaMaps.remove(appliedSchema.f0, null); } return new PaimonEvent(tableId, null, true); } else if (event instanceof DataChangeEvent) { diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/EventRecordSerializationSchema.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/EventRecordSerializationSchema.java index 76b55aafc9..05e8b7f7cf 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/EventRecordSerializationSchema.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/EventRecordSerializationSchema.java @@ -92,15 +92,19 @@ private void applySchemaChangeEvent(SchemaChangeEvent event) { } newSchema = SchemaUtils.applySchemaChangeEvent(tableInfo.schema, event); } - TableInfo tableInfo = new TableInfo(); - tableInfo.schema = newSchema; - tableInfo.fieldGetters = new RecordData.FieldGetter[newSchema.getColumnCount()]; - for (int i = 0; i < newSchema.getColumnCount(); i++) { - tableInfo.fieldGetters[i] = - StarRocksUtils.createFieldGetter( - newSchema.getColumns().get(i).getType(), i, zoneId); + if (newSchema != null) { + TableInfo tableInfo = new TableInfo(); + tableInfo.schema = newSchema; + tableInfo.fieldGetters = new RecordData.FieldGetter[newSchema.getColumnCount()]; + for (int i = 0; i < newSchema.getColumnCount(); i++) { + tableInfo.fieldGetters[i] = + StarRocksUtils.createFieldGetter( + newSchema.getColumns().get(i).getType(), i, zoneId); + } + tableInfoMap.put(tableId, tableInfo); + } else { + tableInfoMap.remove(tableId); } - tableInfoMap.put(tableId, tableInfo); } private StarRocksRowData applyDataChangeEvent(DataChangeEvent event) { diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksMetadataApplier.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksMetadataApplier.java index 865613eda5..1b8dc962a5 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksMetadataApplier.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksMetadataApplier.java @@ -18,13 +18,18 @@ package org.apache.flink.cdc.connectors.starrocks.sink; import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnCommentEvent; import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; +import org.apache.flink.cdc.common.event.AlterTableCommentEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.DropTableEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.event.SchemaChangeEventType; +import org.apache.flink.cdc.common.event.SchemaChangeEventVisitorVoid; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; import org.apache.flink.cdc.common.exceptions.SchemaEvolveException; import org.apache.flink.cdc.common.exceptions.UnsupportedSchemaChangeEventException; import org.apache.flink.cdc.common.schema.Column; @@ -97,19 +102,62 @@ public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) catalog.open(); } - if (schemaChangeEvent instanceof CreateTableEvent) { - applyCreateTable((CreateTableEvent) schemaChangeEvent); - } else if (schemaChangeEvent instanceof AddColumnEvent) { - applyAddColumn((AddColumnEvent) schemaChangeEvent); - } else if (schemaChangeEvent instanceof DropColumnEvent) { - applyDropColumn((DropColumnEvent) schemaChangeEvent); - } else if (schemaChangeEvent instanceof RenameColumnEvent) { - applyRenameColumn((RenameColumnEvent) schemaChangeEvent); - } else if (schemaChangeEvent instanceof AlterColumnTypeEvent) { - applyAlterColumn((AlterColumnTypeEvent) schemaChangeEvent); - } else { - throw new UnsupportedSchemaChangeEventException(schemaChangeEvent); - } + schemaChangeEvent.visit( + new SchemaChangeEventVisitorVoid() { + + @Override + public void visit(AddColumnEvent event) { + applyAddColumn(event); + } + + @Override + public void visit(AlterColumnCommentEvent event) { + throw new UnsupportedOperationException( + "StarRocksDataSink doesn't support schema change event " + + schemaChangeEvent); + } + + @Override + public void visit(AlterColumnTypeEvent event) { + applyAlterColumnType(event); + } + + @Override + public void visit(AlterTableCommentEvent event) { + throw new UnsupportedOperationException( + "StarRocksDataSink doesn't support schema change event " + + schemaChangeEvent); + } + + @Override + public void visit(CreateTableEvent event) { + applyCreateTable(event); + } + + @Override + public void visit(DropColumnEvent event) { + applyDropColumn(event); + } + + @Override + public void visit(DropTableEvent event) { + throw new UnsupportedOperationException( + "StarRocksDataSink doesn't support schema change event " + + schemaChangeEvent); + } + + @Override + public void visit(RenameColumnEvent event) { + applyRenameColumn(event); + } + + @Override + public void visit(TruncateTableEvent event) { + throw new UnsupportedOperationException( + "StarRocksDataSink doesn't support schema change event " + + schemaChangeEvent); + } + }); } private void applyCreateTable(CreateTableEvent createTableEvent) throws SchemaEvolveException { @@ -289,8 +337,7 @@ private void applyRenameColumn(RenameColumnEvent renameColumnEvent) throw new UnsupportedSchemaChangeEventException(renameColumnEvent); } - private void applyAlterColumn(AlterColumnTypeEvent alterColumnTypeEvent) - throws SchemaEvolveException { + private void applyAlterColumnType(AlterColumnTypeEvent alterColumnTypeEvent) { // TODO There are limitations for data type conversions. We should know the data types // before and after changing so that we can make a validation. But the event only contains // data diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/ValuesDatabase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/ValuesDatabase.java index e19c4a844b..f09fb9e203 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/ValuesDatabase.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/ValuesDatabase.java @@ -20,14 +20,17 @@ import org.apache.flink.cdc.common.annotation.Internal; import org.apache.flink.cdc.common.data.RecordData; import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnCommentEvent; import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DataChangeEvent; import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.DropTableEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.event.SchemaChangeEventType; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; import org.apache.flink.cdc.common.exceptions.SchemaEvolveException; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; @@ -223,6 +226,13 @@ public static Schema getTableSchema(TableId tableId) { return builder.primaryKey(table.primaryKeys).build(); } + public static void applyTruncateTableEvent(TruncateTableEvent event) { + ValuesTable table = globalTables.get(event.tableId()); + Preconditions.checkNotNull(table, event.tableId() + " is not existed"); + table.applyTruncateTableEvent(event); + LOG.info("apply TruncateTableEvent: " + event); + } + public static void applyDataChangeEvent(DataChangeEvent event) { ValuesTable table = globalTables.get(event.tableId()); Preconditions.checkNotNull(table, event.tableId() + " is not existed"); @@ -237,6 +247,13 @@ public static void applySchemaChangeEvent(SchemaChangeEvent event) { globalTables.put( tableId, new ValuesTable(tableId, ((CreateTableEvent) event).getSchema())); } + } else if (event instanceof DropTableEvent) { + globalTables.remove(tableId); + } else if (event instanceof TruncateTableEvent) { + if (globalTables.containsKey(tableId)) { + ValuesTable table = globalTables.get(event.tableId()); + table.applyTruncateTableEvent((TruncateTableEvent) event); + } } else { ValuesTable table = globalTables.get(event.tableId()); Preconditions.checkNotNull(table, event.tableId() + " is not existed"); @@ -407,6 +424,24 @@ private void applyAlterColumnTypeEvent(AlterColumnTypeEvent event) { }); } + private void applyAlterColumnCommentEvent(AlterColumnCommentEvent event) { + event.getCommentMapping() + .forEach( + (columnName, columnComment) -> { + for (int i = 0; i < columns.size(); i++) { + Column column = columns.get(i); + if (column.getName().equals(columnName)) { + columns.set( + i, + Column.physicalColumn( + columnName, + column.getType(), + columnComment)); + } + } + }); + } + private void applyAddColumnEvent(AddColumnEvent event) { for (AddColumnEvent.ColumnWithPosition columnWithPosition : event.getAddedColumns()) { if (columns.contains(columnWithPosition.getAddColumn())) { @@ -489,5 +524,9 @@ private void applyRenameColumnEvent(RenameColumnEvent event) { }); }); } + + private void applyTruncateTableEvent(TruncateTableEvent event) { + records.clear(); + } } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/sink/ValuesDataSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/sink/ValuesDataSink.java index 7d4ee3dad8..18726cea02 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/sink/ValuesDataSink.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/sink/ValuesDataSink.java @@ -22,7 +22,6 @@ import org.apache.flink.cdc.common.annotation.Internal; import org.apache.flink.cdc.common.data.RecordData; import org.apache.flink.cdc.common.event.ChangeEvent; -import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DataChangeEvent; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.SchemaChangeEvent; @@ -143,19 +142,16 @@ public void write(Event event, Context context) { if (event instanceof SchemaChangeEvent) { SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; TableId tableId = schemaChangeEvent.tableId(); - if (event instanceof CreateTableEvent) { - Schema schema = ((CreateTableEvent) event).getSchema(); - schemaMaps.put(tableId, schema); - fieldGetterMaps.put(tableId, SchemaUtils.createFieldGetters(schema)); + Schema appliedSchema = + SchemaUtils.applySchemaChangeEvent( + schemaMaps.get(tableId), schemaChangeEvent); + + if (appliedSchema != null) { + schemaMaps.put(tableId, appliedSchema); + fieldGetterMaps.put(tableId, SchemaUtils.createFieldGetters(appliedSchema)); } else { - if (!schemaMaps.containsKey(tableId)) { - throw new RuntimeException("schema of " + tableId + " is not existed."); - } - Schema schema = - SchemaUtils.applySchemaChangeEvent( - schemaMaps.get(tableId), schemaChangeEvent); - schemaMaps.put(tableId, schema); - fieldGetterMaps.put(tableId, SchemaUtils.createFieldGetters(schema)); + schemaMaps.remove(tableId); + fieldGetterMaps.remove(tableId); } } else if (materializedInMemory && event instanceof DataChangeEvent) { ValuesDatabase.applyDataChangeEvent((DataChangeEvent) event); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/sink/ValuesDataSinkFunction.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/sink/ValuesDataSinkFunction.java index f4876d5e04..8b49e139f1 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/sink/ValuesDataSinkFunction.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/sink/ValuesDataSinkFunction.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.eventtime.Watermark; import org.apache.flink.cdc.common.data.RecordData; import org.apache.flink.cdc.common.event.ChangeEvent; -import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DataChangeEvent; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.SchemaChangeEvent; @@ -60,19 +59,16 @@ public void invoke(Event event, Context context) throws Exception { if (event instanceof SchemaChangeEvent) { SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; TableId tableId = schemaChangeEvent.tableId(); - if (event instanceof CreateTableEvent) { - Schema schema = ((CreateTableEvent) event).getSchema(); - schemaMaps.put(tableId, schema); - fieldGetterMaps.put(tableId, SchemaUtils.createFieldGetters(schema)); + Schema appliedSchema = + SchemaUtils.applySchemaChangeEvent( + schemaMaps.get(schemaChangeEvent.tableId()), schemaChangeEvent); + + if (appliedSchema != null) { + schemaMaps.put(tableId, appliedSchema); + fieldGetterMaps.put(tableId, SchemaUtils.createFieldGetters(appliedSchema)); } else { - if (!schemaMaps.containsKey(tableId)) { - throw new RuntimeException("schema of " + tableId + " is not existed."); - } - Schema schema = - SchemaUtils.applySchemaChangeEvent( - schemaMaps.get(tableId), schemaChangeEvent); - schemaMaps.put(tableId, schema); - fieldGetterMaps.put(tableId, SchemaUtils.createFieldGetters(schema)); + schemaMaps.remove(tableId); + fieldGetterMaps.remove(tableId); } } else if (materializedInMemory && event instanceof DataChangeEvent) { ValuesDatabase.applyDataChangeEvent((DataChangeEvent) event); diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MysqlE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MysqlE2eITCase.java index ada384e070..d1f3d07802 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MysqlE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MysqlE2eITCase.java @@ -197,9 +197,15 @@ public void testSyncWholeDatabase() throws Exception { stat.execute( "INSERT INTO products VALUES (default,'jacket','water resistent white wind breaker',0.2, null, null, null, 1);"); // 110 stat.execute( - "INSERT INTO products VALUES (default,'scooter','Big 2-wheel scooter ',5.18, null, null, null, 1);"); // 111 + "INSERT INTO products VALUES (default,'scooter','Big 2-wheel scooter ',5.17, null, null, null, 1);"); // 111 stat.execute( "UPDATE products SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110;"); + stat.execute("ALTER TABLE products MODIFY COLUMN weight INT COMMENT 'mass effect';"); + stat.execute("ALTER TABLE products RENAME COLUMN weight TO weight_tmp;"); + stat.execute("ALTER TABLE products RENAME COLUMN weight_tmp TO weight;"); + stat.execute("ALTER TABLE products MODIFY COLUMN weight DOUBLE COMMENT 'mass effect';"); + stat.execute("ALTER TABLE products MODIFY COLUMN weight DOUBLE;"); + stat.execute("ALTER TABLE products MODIFY COLUMN weight FLOAT;"); stat.execute("UPDATE products SET weight='5.17' WHERE id=111;"); stat.execute("DELETE FROM products WHERE id=111;"); } catch (SQLException e) { @@ -227,16 +233,34 @@ public void testSyncWholeDatabase() throws Exception { "DataChangeEvent{tableId=%s.products, before=[], after=[110, jacket, water resistent white wind breaker, 0.2, null, null, null, 1], op=INSERT, meta=()}", mysqlInventoryDatabase.getDatabaseName()), String.format( - "DataChangeEvent{tableId=%s.products, before=[], after=[111, scooter, Big 2-wheel scooter , 5.18, null, null, null, 1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[], after=[111, scooter, Big 2-wheel scooter , 5.17, null, null, null, 1], op=INSERT, meta=()}", mysqlInventoryDatabase.getDatabaseName()), String.format( "DataChangeEvent{tableId=%s.products, before=[110, jacket, water resistent white wind breaker, 0.2, null, null, null, 1], after=[110, jacket, new water resistent white wind breaker, 0.5, null, null, null, 1], op=UPDATE, meta=()}", mysqlInventoryDatabase.getDatabaseName()), String.format( - "DataChangeEvent{tableId=%s.products, before=[111, scooter, Big 2-wheel scooter , 5.18, null, null, null, 1], after=[111, scooter, Big 2-wheel scooter , 5.17, null, null, null, 1], op=UPDATE, meta=()}", + "DataChangeEvent{tableId=%s.products, before=[111, scooter, Big 2-wheel scooter , 5.0, null, null, null, 1], after=[111, scooter, Big 2-wheel scooter , 5.17, null, null, null, 1], op=UPDATE, meta=()}", mysqlInventoryDatabase.getDatabaseName()), String.format( "DataChangeEvent{tableId=%s.products, before=[111, scooter, Big 2-wheel scooter , 5.17, null, null, null, 1], after=[], op=DELETE, meta=()}", + mysqlInventoryDatabase.getDatabaseName()), + String.format( + "AlterColumnTypeEvent{tableId=%s.products, typeMapping={weight=INT}, oldTypeMapping={weight=FLOAT}}", + mysqlInventoryDatabase.getDatabaseName()), + String.format( + "RenameColumnEvent{tableId=%s.products, nameMapping={weight=weight_tmp}}", + mysqlInventoryDatabase.getDatabaseName()), + String.format( + "RenameColumnEvent{tableId=%s.products, nameMapping={weight_tmp=weight}}", + mysqlInventoryDatabase.getDatabaseName()), + String.format( + "AlterColumnTypeEvent{tableId=%s.products, typeMapping={weight=DOUBLE}, oldTypeMapping={weight=INT}}", + mysqlInventoryDatabase.getDatabaseName()), + String.format( + "AlterColumnCommentEvent{tableId=%s.products, commentMapping={weight=mass effect}, oldCommentMapping={weight=null}}", + mysqlInventoryDatabase.getDatabaseName()), + String.format( + "AlterColumnCommentEvent{tableId=%s.products, commentMapping={weight=null}, oldCommentMapping={weight=mass effect}}", mysqlInventoryDatabase.getDatabaseName())); validateResult(expectedEvents); } diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/RouteE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/RouteE2eITCase.java index ab7ed178e6..8b3c907b89 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/RouteE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/RouteE2eITCase.java @@ -221,7 +221,7 @@ public void testDefaultRoute() throws Exception { "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[10001, 12, Derrida], op=INSERT, meta=()}", "RenameColumnEvent{tableId=%s.TABLEBETA, nameMapping={VERSION=VERSION_EX}}", "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[10002, 15], op=INSERT, meta=()}", - "AlterColumnTypeEvent{tableId=%s.TABLEGAMMA, nameMapping={VERSION=VARCHAR(19)}}", + "AlterColumnTypeEvent{tableId=%s.TABLEGAMMA, typeMapping={VERSION=VARCHAR(19)}, oldTypeMapping={VERSION=VARCHAR(17)}}", "RenameColumnEvent{tableId=%s.TABLEGAMMA, nameMapping={VERSION=VERSION_EX}}", "DataChangeEvent{tableId=%s.TABLEGAMMA, before=[], after=[10003, Fluorite], op=INSERT, meta=()}", "DropColumnEvent{tableId=%s.TABLEDELTA, droppedColumnNames=[VERSION]}"); @@ -305,7 +305,7 @@ public void testMergeTableRoute() throws Exception { "DataChangeEvent{tableId=%s.ALL, before=[], after=[10001, 12, Derrida], op=INSERT, meta=()}", "AddColumnEvent{tableId=%s.ALL, addedColumns=[ColumnWithPosition{column=`VERSION_EX` VARCHAR(17), position=LAST, existedColumnName=null}]}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[10002, null, null, 15], op=INSERT, meta=()}", - "AlterColumnTypeEvent{tableId=%s.ALL, nameMapping={VERSION=STRING}}", + "AlterColumnTypeEvent{tableId=%s.ALL, typeMapping={VERSION=STRING}, oldTypeMapping={VERSION=VARCHAR(17)}}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[10003, null, null, Fluorite], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[10004, null, null, null], op=INSERT, meta=()}"); } @@ -398,7 +398,7 @@ public void testPartialRoute() throws Exception { "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[], after=[10001, 12, Derrida], op=INSERT, meta=()}", "AddColumnEvent{tableId=NEW_%s.ALPHABET, addedColumns=[ColumnWithPosition{column=`VERSION_EX` VARCHAR(17), position=LAST, existedColumnName=null}]}", "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[], after=[10002, null, null, 15], op=INSERT, meta=()}", - "AlterColumnTypeEvent{tableId=%s.TABLEGAMMA, nameMapping={VERSION=VARCHAR(19)}}", + "AlterColumnTypeEvent{tableId=%s.TABLEGAMMA, typeMapping={VERSION=VARCHAR(19)}, oldTypeMapping={VERSION=VARCHAR(17)}}", "RenameColumnEvent{tableId=%s.TABLEGAMMA, nameMapping={VERSION=VERSION_EX}}", "DataChangeEvent{tableId=%s.TABLEGAMMA, before=[], after=[10003, Fluorite], op=INSERT, meta=()}", "DropColumnEvent{tableId=%s.TABLEDELTA, droppedColumnNames=[VERSION]}", @@ -504,7 +504,7 @@ public void testMultipleRoute() throws Exception { "AddColumnEvent{tableId=NEW_%s.BETAGAMM, addedColumns=[ColumnWithPosition{column=`VERSION_EX` VARCHAR(17), position=LAST, existedColumnName=null}]}", "DataChangeEvent{tableId=NEW_%s.ALPHABET, before=[], after=[10002, null, null, 15], op=INSERT, meta=()}", "DataChangeEvent{tableId=NEW_%s.BETAGAMM, before=[], after=[10002, null, 15], op=INSERT, meta=()}", - "AlterColumnTypeEvent{tableId=NEW_%s.BETAGAMM, nameMapping={VERSION=STRING}}", + "AlterColumnTypeEvent{tableId=NEW_%s.BETAGAMM, typeMapping={VERSION=STRING}, oldTypeMapping={VERSION=VARCHAR(17)}}", "DataChangeEvent{tableId=NEW_%s.BETAGAMM, before=[], after=[10003, null, Fluorite], op=INSERT, meta=()}", "DropColumnEvent{tableId=%s.TABLEDELTA, droppedColumnNames=[VERSION]}", "DataChangeEvent{tableId=%s.TABLEDELTA, before=[], after=[10004], op=INSERT, meta=()}"); @@ -616,7 +616,7 @@ public void testOneToManyRoute() throws Exception { "DataChangeEvent{tableId=NEW_%s.TABLEC, before=[], after=[10001, 12, Derrida], op=INSERT, meta=()}", "RenameColumnEvent{tableId=%s.TABLEBETA, nameMapping={VERSION=VERSION_EX}}", "DataChangeEvent{tableId=%s.TABLEBETA, before=[], after=[10002, 15], op=INSERT, meta=()}", - "AlterColumnTypeEvent{tableId=%s.TABLEGAMMA, nameMapping={VERSION=VARCHAR(19)}}", + "AlterColumnTypeEvent{tableId=%s.TABLEGAMMA, typeMapping={VERSION=VARCHAR(19)}, oldTypeMapping={VERSION=VARCHAR(17)}}", "RenameColumnEvent{tableId=%s.TABLEGAMMA, nameMapping={VERSION=VERSION_EX}}", "DataChangeEvent{tableId=%s.TABLEGAMMA, before=[], after=[10003, Fluorite], op=INSERT, meta=()}", "DropColumnEvent{tableId=%s.TABLEDELTA, droppedColumnNames=[VERSION]}", @@ -706,7 +706,7 @@ public void testMergeTableRouteWithTransform() throws Exception { "DataChangeEvent{tableId=%s.ALL, before=[], after=[10001, 12, Derrida, extras], op=INSERT, meta=()}", "AddColumnEvent{tableId=%s.ALL, addedColumns=[ColumnWithPosition{column=`VERSION_EX` VARCHAR(17), position=LAST, existedColumnName=null}]}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[10002, null, extras, null, 15], op=INSERT, meta=()}", - "AlterColumnTypeEvent{tableId=%s.ALL, nameMapping={VERSION=STRING}}", + "AlterColumnTypeEvent{tableId=%s.ALL, typeMapping={VERSION=STRING}, oldTypeMapping={VERSION=VARCHAR(17)}}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[10003, null, extras, null, Fluorite], op=INSERT, meta=()}", "DataChangeEvent{tableId=%s.ALL, before=[], after=[10004, null, extras, null, null], op=INSERT, meta=()}"); } @@ -800,7 +800,7 @@ public void testReplacementSymbol() throws Exception { "DataChangeEvent{tableId=NEW_%s.NEW_TABLEALPHA, before=[], after=[10001, 12, Derrida], op=INSERT, meta=()}", "RenameColumnEvent{tableId=NEW_%s.NEW_TABLEBETA, nameMapping={VERSION=VERSION_EX}}", "DataChangeEvent{tableId=NEW_%s.NEW_TABLEBETA, before=[], after=[10002, 15], op=INSERT, meta=()}", - "AlterColumnTypeEvent{tableId=NEW_%s.NEW_TABLEGAMMA, nameMapping={VERSION=VARCHAR(19)}}", + "AlterColumnTypeEvent{tableId=NEW_%s.NEW_TABLEGAMMA, typeMapping={VERSION=VARCHAR(19)}, oldTypeMapping={VERSION=VARCHAR(17)}", "RenameColumnEvent{tableId=NEW_%s.NEW_TABLEGAMMA, nameMapping={VERSION=VERSION_EX}}", "DataChangeEvent{tableId=NEW_%s.NEW_TABLEGAMMA, before=[], after=[10003, Fluorite], op=INSERT, meta=()}", "DropColumnEvent{tableId=NEW_%s.NEW_TABLEDELTA, droppedColumnNames=[VERSION]}", diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/SchemaEvolveE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/SchemaEvolveE2eITCase.java index d187853daf..27fbeecba6 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/SchemaEvolveE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/SchemaEvolveE2eITCase.java @@ -96,7 +96,7 @@ public void testSchemaEvolve() throws Exception { Arrays.asList( "AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`gender` TINYINT, position=AFTER, existedColumnName=age}]}", "DataChangeEvent{tableId=%s.members, before=[], after=[1012, Eve, 17, 0], op=INSERT, meta=()}", - "AlterColumnTypeEvent{tableId=%s.members, nameMapping={age=DOUBLE}}", + "AlterColumnTypeEvent{tableId=%s.members, typeMapping={age=DOUBLE}, oldTypeMapping={age=INT}}", "RenameColumnEvent{tableId=%s.members, nameMapping={age=precise_age}}", "RenameColumnEvent{tableId=%s.members, nameMapping={gender=biological_sex}}", "DropColumnEvent{tableId=%s.members, droppedColumnNames=[biological_sex]}", @@ -183,7 +183,7 @@ public void testLenientSchemaEvolution() throws Exception { Arrays.asList( "AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`gender` TINYINT, position=AFTER, existedColumnName=age}]}", "DataChangeEvent{tableId=%s.members, before=[], after=[1012, Eve, 17, 0], op=INSERT, meta=()}", - "AlterColumnTypeEvent{tableId=%s.members, nameMapping={age=DOUBLE}}", + "AlterColumnTypeEvent{tableId=%s.members, typeMapping={age=DOUBLE}, oldTypeMapping={age=INT}}", "AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`precise_age` DOUBLE, position=LAST, existedColumnName=null}]}", "AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`biological_sex` TINYINT, position=LAST, existedColumnName=null}]}", "DataChangeEvent{tableId=%s.members, before=[], after=[1013, Fiona, null, null, 16.0, null], op=INSERT, meta=()}")); @@ -200,7 +200,7 @@ public void testFineGrainedSchemaEvolution() throws Exception { Arrays.asList( "AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`gender` TINYINT, position=AFTER, existedColumnName=age}]}", "DataChangeEvent{tableId=%s.members, before=[], after=[1012, Eve, 17, 0], op=INSERT, meta=()}", - "AlterColumnTypeEvent{tableId=%s.members, nameMapping={age=DOUBLE}}", + "AlterColumnTypeEvent{tableId=%s.members, typeMapping={age=DOUBLE}, oldTypeMapping={age=INT}}", "RenameColumnEvent{tableId=%s.members, nameMapping={age=precise_age}}", "RenameColumnEvent{tableId=%s.members, nameMapping={gender=biological_sex}}", "DataChangeEvent{tableId=%s.members, before=[], after=[1013, Fiona, 16.0, null], op=INSERT, meta=()}"), diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaDerivation.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaDerivation.java index e4b547b216..dc0bb3bdb5 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaDerivation.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaDerivation.java @@ -19,12 +19,17 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnCommentEvent; import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; +import org.apache.flink.cdc.common.event.AlterTableCommentEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.DropTableEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventVisitor; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; import org.apache.flink.cdc.common.route.RouteRule; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.PhysicalColumn; @@ -111,37 +116,64 @@ public List applySchemaChange(SchemaChangeEvent schemaChangeE // multiple source mapping (merging tables) Schema derivedTableSchema = schemaManager.getLatestEvolvedSchema(derivedTable).get(); - if (schemaChangeEvent instanceof CreateTableEvent) { - events.addAll( - handleCreateTableEvent( - (CreateTableEvent) schemaChangeEvent, - derivedTableSchema, - derivedTable)); - } else if (schemaChangeEvent instanceof AddColumnEvent) { - events.addAll( - handleAddColumnEvent( - (AddColumnEvent) schemaChangeEvent, - derivedTableSchema, - derivedTable)); - } else if (schemaChangeEvent instanceof AlterColumnTypeEvent) { - events.addAll( - handleAlterColumnTypeEvent( - (AlterColumnTypeEvent) schemaChangeEvent, - derivedTableSchema, - derivedTable)); - } else if (schemaChangeEvent instanceof DropColumnEvent) { - // Do nothing: drop column event should not be sent to downstream - } else if (schemaChangeEvent instanceof RenameColumnEvent) { - events.addAll( - handleRenameColumnEvent( - (RenameColumnEvent) schemaChangeEvent, - derivedTableSchema, - derivedTable)); - } else { - throw new IllegalStateException( - String.format( - "Unrecognized SchemaChangeEvent type: %s", schemaChangeEvent)); - } + events.addAll( + schemaChangeEvent.visit( + new SchemaChangeEventVisitor>() { + + @Override + public List visit(AddColumnEvent event) { + return handleAddColumnEvent( + event, derivedTableSchema, derivedTable); + } + + @Override + public List visit( + AlterColumnCommentEvent event) { + return Collections.emptyList(); + } + + @Override + public List visit( + AlterColumnTypeEvent event) { + return handleAlterColumnTypeEvent( + event, derivedTableSchema, derivedTable); + } + + @Override + public List visit( + AlterTableCommentEvent event) { + return Collections.emptyList(); + } + + @Override + public List visit(CreateTableEvent event) { + return handleCreateTableEvent( + event, derivedTableSchema, derivedTable); + } + + @Override + public List visit(DropColumnEvent event) { + // Column drop shouldn't be spread to route destination. + return Collections.emptyList(); + } + + @Override + public List visit(DropTableEvent event) { + // Column drop shouldn't be spread to route destination. + return Collections.emptyList(); + } + + @Override + public List visit(RenameColumnEvent event) { + return handleRenameColumnEvent( + event, derivedTableSchema, derivedTable); + } + + @Override + public List visit(TruncateTableEvent event) { + return Collections.emptyList(); + } + })); } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaManager.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaManager.java index b87ef5b128..fa38fd11f8 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaManager.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaManager.java @@ -150,39 +150,48 @@ public Schema getOriginalSchema(TableId tableId, int version) { /** Apply schema change to a table. */ public void applyOriginalSchemaChange(SchemaChangeEvent schemaChangeEvent) { - if (schemaChangeEvent instanceof CreateTableEvent) { - handleCreateTableEvent(originalSchemas, ((CreateTableEvent) schemaChangeEvent)); - } else { - Optional optionalSchema = getLatestOriginalSchema(schemaChangeEvent.tableId()); + Optional optionalSchema = getLatestOriginalSchema(schemaChangeEvent.tableId()); + if (!(schemaChangeEvent instanceof CreateTableEvent)) { checkArgument( optionalSchema.isPresent(), "Unable to apply SchemaChangeEvent for table \"%s\" without existing schema", schemaChangeEvent.tableId()); + } + + LOG.info("Handling upstream schema change event: {}", schemaChangeEvent); + Schema appliedSchema = + SchemaUtils.applySchemaChangeEvent(optionalSchema.orElse(null), schemaChangeEvent); - LOG.info("Handling original schema change event: {}", schemaChangeEvent); - registerNewSchema( - originalSchemas, - schemaChangeEvent.tableId(), - SchemaUtils.applySchemaChangeEvent(optionalSchema.get(), schemaChangeEvent)); + if (appliedSchema != null) { + registerNewSchema(originalSchemas, schemaChangeEvent.tableId(), appliedSchema); + } else { + dropSchema(originalSchemas, schemaChangeEvent.tableId()); } } /** Apply schema change to a table. */ public void applyEvolvedSchemaChange(SchemaChangeEvent schemaChangeEvent) { - if (schemaChangeEvent instanceof CreateTableEvent) { - handleCreateTableEvent(evolvedSchemas, ((CreateTableEvent) schemaChangeEvent)); - } else { - Optional optionalSchema = getLatestEvolvedSchema(schemaChangeEvent.tableId()); + Optional optionalSchema = getLatestEvolvedSchema(schemaChangeEvent.tableId()); + if (!(schemaChangeEvent instanceof CreateTableEvent)) { checkArgument( optionalSchema.isPresent(), "Unable to apply SchemaChangeEvent for table \"%s\" without existing schema", schemaChangeEvent.tableId()); + } else { + checkArgument( + !optionalSchema.isPresent(), + "Unable to apply CreateTableEvent to an existing schema for table \"%s\"", + schemaChangeEvent.tableId()); + } - LOG.info("Handling evolved schema change event: {}", schemaChangeEvent); - registerNewSchema( - evolvedSchemas, - schemaChangeEvent.tableId(), - SchemaUtils.applySchemaChangeEvent(optionalSchema.get(), schemaChangeEvent)); + LOG.info("Handling evolved schema change event: {}", schemaChangeEvent); + Schema appliedSchema = + SchemaUtils.applySchemaChangeEvent(optionalSchema.orElse(null), schemaChangeEvent); + + if (appliedSchema != null) { + registerNewSchema(evolvedSchemas, schemaChangeEvent.tableId(), appliedSchema); + } else { + dropSchema(evolvedSchemas, schemaChangeEvent.tableId()); } } @@ -218,16 +227,6 @@ private Optional getLatestSchemaVersion( } } - private void handleCreateTableEvent( - final Map> schemaMap, CreateTableEvent event) { - checkArgument( - !schemaExists(schemaMap, event.tableId()), - "Unable to apply CreateTableEvent to an existing schema for table \"%s\"", - event.tableId()); - LOG.info("Handling schema change event: {}", event); - registerNewSchema(schemaMap, event.tableId(), event.getSchema()); - } - private void registerNewSchema( final Map> schemaMap, TableId tableId, @@ -246,6 +245,11 @@ private void registerNewSchema( } } + private void dropSchema( + final Map> schemaMap, TableId tableId) { + schemaMap.remove(tableId, null); + } + /** Serializer for {@link SchemaManager}. */ public static class Serializer implements SimpleVersionedSerializer { diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaRegistryRequestHandler.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaRegistryRequestHandler.java index 77360169e2..aedf75c181 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaRegistryRequestHandler.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaRegistryRequestHandler.java @@ -26,6 +26,7 @@ import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.event.SchemaChangeEventType; +import org.apache.flink.cdc.common.event.SchemaChangeEventWithPreSchema; import org.apache.flink.cdc.common.event.TableId; import org.apache.flink.cdc.common.exceptions.SchemaEvolveException; import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; @@ -175,14 +176,26 @@ public CompletableFuture handleSchemaChangeRequest( LOG.info( "Received schema change event request from table {}. Start to buffer requests for others.", request.getTableId().toString()); - if (request.getSchemaChangeEvent() instanceof CreateTableEvent + SchemaChangeEvent event = request.getSchemaChangeEvent(); + if (event instanceof CreateTableEvent && schemaManager.originalSchemaExists(request.getTableId())) { return CompletableFuture.completedFuture( wrap(new SchemaChangeResponse(Collections.emptyList()))); } - schemaManager.applyOriginalSchemaChange(request.getSchemaChangeEvent()); + schemaManager.applyOriginalSchemaChange(event); List derivedSchemaChangeEvents = calculateDerivedSchemaChangeEvents(request.getSchemaChangeEvent()); + derivedSchemaChangeEvents.forEach( + e -> { + if (e instanceof SchemaChangeEventWithPreSchema) { + SchemaChangeEventWithPreSchema pe = (SchemaChangeEventWithPreSchema) e; + if (!pe.hasPreSchema()) { + schemaManager + .getLatestEvolvedSchema(pe.tableId()) + .ifPresent(pe::fillPreSchema); + } + } + }); CompletableFuture response = CompletableFuture.completedFuture( wrap(new SchemaChangeResponse(derivedSchemaChangeEvents))); diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperator.java index b280bf9016..6430ac50b2 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PostTransformOperator.java @@ -246,7 +246,7 @@ private Schema transformSchema(TableId tableId, Schema schema) throws Exception private Optional processDataChangeEvent(DataChangeEvent dataChangeEvent) throws Exception { TableId tableId = dataChangeEvent.tableId(); - PostTransformChangeInfo tableInfo = getPostTransformChangeInfo(tableId); + PostTransformChangeInfo tableInfo = postTransformChangeInfoMap.get(tableId); List> transformedDataChangeEventOptionalList = new ArrayList<>(); long epochTime = System.currentTimeMillis(); for (PostTransformer transform : transforms) { diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperator.java index 7dbba91d1b..32927b55c1 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperator.java @@ -22,11 +22,17 @@ import org.apache.flink.api.common.state.OperatorStateStore; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.data.binary.BinaryRecordData; +import org.apache.flink.cdc.common.event.AlterTableCommentEvent; +import org.apache.flink.cdc.common.event.ColumnSchemaChangeEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DataChangeEvent; +import org.apache.flink.cdc.common.event.DropTableEvent; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TableSchemaChangeEvent; +import org.apache.flink.cdc.common.event.TableSchemaChangeEventVisitorVoid; +import org.apache.flink.cdc.common.event.TruncateTableEvent; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.schema.Selectors; import org.apache.flink.cdc.common.utils.SchemaUtils; @@ -193,19 +199,38 @@ public void close() throws Exception { @Override public void processElement(StreamRecord element) throws Exception { Event event = element.getValue(); - if (event instanceof CreateTableEvent) { - CreateTableEvent createTableEvent = (CreateTableEvent) event; - preTransformProcessorMap.remove(createTableEvent.tableId()); - event = cacheCreateTable(createTableEvent); - output.collect(new StreamRecord<>(event)); - } else if (event instanceof SchemaChangeEvent) { + if (event instanceof TableSchemaChangeEvent) { + ((TableSchemaChangeEvent) event) + .visit( + new TableSchemaChangeEventVisitorVoid() { + @Override + public void visit(AlterTableCommentEvent event) { + output.collect(new StreamRecord<>(event)); + } + + @Override + public void visit(CreateTableEvent event) { + preTransformProcessorMap.remove(event.tableId()); + output.collect(new StreamRecord<>(cacheCreateTable(event))); + } + + @Override + public void visit(DropTableEvent event) { + output.collect(new StreamRecord<>(event)); + } + + @Override + public void visit(TruncateTableEvent event) { + output.collect(new StreamRecord<>(event)); + } + }); + } else if (event instanceof ColumnSchemaChangeEvent) { SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; preTransformProcessorMap.remove(schemaChangeEvent.tableId()); - event = cacheChangeSchema(schemaChangeEvent); + cacheChangeSchema(schemaChangeEvent); output.collect(new StreamRecord<>(event)); } else if (event instanceof DataChangeEvent) { - DataChangeEvent dataChangeEvent = processDataChangeEvent(((DataChangeEvent) event)); - output.collect(new StreamRecord<>(dataChangeEvent)); + output.collect(new StreamRecord<>(processDataChangeEvent(((DataChangeEvent) event)))); } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/AlterColumnCommentEventSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/AlterColumnCommentEventSerializer.java new file mode 100644 index 0000000000..4c669b85de --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/AlterColumnCommentEventSerializer.java @@ -0,0 +1,119 @@ +/* + * 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.cdc.runtime.serializer.event; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.cdc.common.event.AlterColumnCommentEvent; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.runtime.serializer.MapSerializer; +import org.apache.flink.cdc.runtime.serializer.StringSerializer; +import org.apache.flink.cdc.runtime.serializer.TableIdSerializer; +import org.apache.flink.cdc.runtime.serializer.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; +import java.util.Collections; + +/** A {@link TypeSerializer} for {@link AlterColumnCommentEvent}. */ +public class AlterColumnCommentEventSerializer + extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + /** Sharable instance of the TableIdSerializer. */ + public static final AlterColumnCommentEventSerializer INSTANCE = + new AlterColumnCommentEventSerializer(); + + private final TableIdSerializer tableIdSerializer = TableIdSerializer.INSTANCE; + private final MapSerializer commentMapSerializer = + new MapSerializer<>(StringSerializer.INSTANCE, StringSerializer.INSTANCE); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public AlterColumnCommentEvent createInstance() { + return new AlterColumnCommentEvent(TableId.tableId("unknown"), Collections.emptyMap()); + } + + @Override + public AlterColumnCommentEvent copy(AlterColumnCommentEvent from) { + return new AlterColumnCommentEvent( + from.tableId(), + commentMapSerializer.copy(from.getCommentMapping()), + commentMapSerializer.copy(from.getOldCommentMapping())); + } + + @Override + public AlterColumnCommentEvent copy( + AlterColumnCommentEvent from, AlterColumnCommentEvent reuse) { + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(AlterColumnCommentEvent record, DataOutputView target) + throws IOException { + tableIdSerializer.serialize(record.tableId(), target); + commentMapSerializer.serialize(record.getCommentMapping(), target); + commentMapSerializer.serialize(record.getOldCommentMapping(), target); + } + + @Override + public AlterColumnCommentEvent deserialize(DataInputView source) throws IOException { + return new AlterColumnCommentEvent( + tableIdSerializer.deserialize(source), + commentMapSerializer.deserialize(source), + commentMapSerializer.deserialize(source)); + } + + @Override + public AlterColumnCommentEvent deserialize(AlterColumnCommentEvent reuse, DataInputView source) + throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new AlterColumnCommentEventSerializer.AlterColumnCommentEventSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class AlterColumnCommentEventSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public AlterColumnCommentEventSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/AlterColumnTypeEventSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/AlterColumnTypeEventSerializer.java index ab228e4d6b..978b9b87ec 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/AlterColumnTypeEventSerializer.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/AlterColumnTypeEventSerializer.java @@ -60,7 +60,9 @@ public AlterColumnTypeEvent createInstance() { @Override public AlterColumnTypeEvent copy(AlterColumnTypeEvent from) { return new AlterColumnTypeEvent( - from.tableId(), typeMapSerializer.copy(from.getTypeMapping())); + from.tableId(), + typeMapSerializer.copy(from.getTypeMapping()), + typeMapSerializer.copy(from.getOldTypeMapping())); } @Override @@ -77,12 +79,15 @@ public int getLength() { public void serialize(AlterColumnTypeEvent record, DataOutputView target) throws IOException { tableIdSerializer.serialize(record.tableId(), target); typeMapSerializer.serialize(record.getTypeMapping(), target); + typeMapSerializer.serialize(record.getOldTypeMapping(), target); } @Override public AlterColumnTypeEvent deserialize(DataInputView source) throws IOException { return new AlterColumnTypeEvent( - tableIdSerializer.deserialize(source), typeMapSerializer.deserialize(source)); + tableIdSerializer.deserialize(source), + typeMapSerializer.deserialize(source), + typeMapSerializer.deserialize(source)); } @Override diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/AlterTableCommentEventSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/AlterTableCommentEventSerializer.java new file mode 100644 index 0000000000..c066896758 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/AlterTableCommentEventSerializer.java @@ -0,0 +1,108 @@ +/* + * 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.cdc.runtime.serializer.event; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.cdc.common.event.AlterTableCommentEvent; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.runtime.serializer.StringSerializer; +import org.apache.flink.cdc.runtime.serializer.TableIdSerializer; +import org.apache.flink.cdc.runtime.serializer.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; + +/** A {@link TypeSerializer} for {@link AlterTableCommentEvent}. */ +public class AlterTableCommentEventSerializer + extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + /** Sharable instance of the TableIdSerializer. */ + public static final AlterTableCommentEventSerializer INSTANCE = + new AlterTableCommentEventSerializer(); + + private final TableIdSerializer tableIdSerializer = TableIdSerializer.INSTANCE; + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public AlterTableCommentEvent createInstance() { + return new AlterTableCommentEvent(TableId.tableId("unknown"), "unknown"); + } + + @Override + public AlterTableCommentEvent copy(AlterTableCommentEvent from) { + return new AlterTableCommentEvent(from.tableId(), from.getTableComment()); + } + + @Override + public AlterTableCommentEvent copy(AlterTableCommentEvent from, AlterTableCommentEvent reuse) { + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(AlterTableCommentEvent record, DataOutputView target) throws IOException { + tableIdSerializer.serialize(record.tableId(), target); + StringSerializer.INSTANCE.serialize(record.getTableComment(), target); + } + + @Override + public AlterTableCommentEvent deserialize(DataInputView source) throws IOException { + return new AlterTableCommentEvent( + tableIdSerializer.deserialize(source), + StringSerializer.INSTANCE.deserialize(source)); + } + + @Override + public AlterTableCommentEvent deserialize(AlterTableCommentEvent reuse, DataInputView source) + throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new AlterTableCommentEventSerializer.AlterTableCommentEventSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class AlterTableCommentEventSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public AlterTableCommentEventSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/DropTableEventSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/DropTableEventSerializer.java new file mode 100644 index 0000000000..b5f8a0d339 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/DropTableEventSerializer.java @@ -0,0 +1,108 @@ +/* + * 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.cdc.runtime.serializer.event; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.cdc.common.event.DropTableEvent; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.runtime.serializer.MapSerializer; +import org.apache.flink.cdc.runtime.serializer.StringSerializer; +import org.apache.flink.cdc.runtime.serializer.TableIdSerializer; +import org.apache.flink.cdc.runtime.serializer.TypeSerializerSingleton; +import org.apache.flink.cdc.runtime.serializer.schema.DataTypeSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; + +/** A {@link TypeSerializer} for {@link DropTableEvent}. */ +public class DropTableEventSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + /** Sharable instance of the TableIdSerializer. */ + public static final DropTableEventSerializer INSTANCE = new DropTableEventSerializer(); + + private final TableIdSerializer tableIdSerializer = TableIdSerializer.INSTANCE; + private final MapSerializer typeMapSerializer = + new MapSerializer<>(StringSerializer.INSTANCE, new DataTypeSerializer()); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public DropTableEvent createInstance() { + return new DropTableEvent(TableId.tableId("unknown")); + } + + @Override + public DropTableEvent copy(DropTableEvent from) { + return new DropTableEvent(from.tableId()); + } + + @Override + public DropTableEvent copy(DropTableEvent from, DropTableEvent reuse) { + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(DropTableEvent record, DataOutputView target) throws IOException { + tableIdSerializer.serialize(record.tableId(), target); + } + + @Override + public DropTableEvent deserialize(DataInputView source) throws IOException { + return new DropTableEvent(tableIdSerializer.deserialize(source)); + } + + @Override + public DropTableEvent deserialize(DropTableEvent reuse, DataInputView source) + throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new DropTableEventSerializer.DropTableEventSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class DropTableEventSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public DropTableEventSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/SchemaChangeEventSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/SchemaChangeEventSerializer.java index 85de2fd7e0..f93e7ef7c0 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/SchemaChangeEventSerializer.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/SchemaChangeEventSerializer.java @@ -21,13 +21,19 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnCommentEvent; import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; +import org.apache.flink.cdc.common.event.AlterTableCommentEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.DropTableEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.event.SchemaChangeEventType; +import org.apache.flink.cdc.common.event.SchemaChangeEventVisitor; +import org.apache.flink.cdc.common.event.SchemaChangeEventVisitorVoid; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; import org.apache.flink.cdc.runtime.serializer.EnumSerializer; import org.apache.flink.cdc.runtime.serializer.TypeSerializerSingleton; import org.apache.flink.core.memory.DataInputView; @@ -35,6 +41,16 @@ import java.io.IOException; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ADD_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ALTER_COLUMN_COMMENT; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ALTER_COLUMN_TYPE; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ALTER_TABLE_COMMENT; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.CREATE_TABLE; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.DROP_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.DROP_TABLE; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.RENAME_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.TRUNCATE_TABLE; + /** A {@link TypeSerializer} for {@link SchemaChangeEvent}. */ public final class SchemaChangeEventSerializer extends TypeSerializerSingleton { @@ -55,32 +71,67 @@ public boolean isImmutableType() { public SchemaChangeEvent createInstance() { return new SchemaChangeEvent() { @Override - public TableId tableId() { - return TableId.tableId("unknown", "unknown", "unknown"); + public SchemaChangeEventType getType() { + return null; } @Override - public SchemaChangeEventType getType() { - return null; + public TableId tableId() { + return TableId.tableId("unknown", "unknown", "unknown"); } }; } @Override public SchemaChangeEvent copy(SchemaChangeEvent from) { - if (from instanceof AlterColumnTypeEvent) { - return AlterColumnTypeEventSerializer.INSTANCE.copy((AlterColumnTypeEvent) from); - } else if (from instanceof CreateTableEvent) { - return CreateTableEventSerializer.INSTANCE.copy((CreateTableEvent) from); - } else if (from instanceof RenameColumnEvent) { - return RenameColumnEventSerializer.INSTANCE.copy((RenameColumnEvent) from); - } else if (from instanceof AddColumnEvent) { - return AddColumnEventSerializer.INSTANCE.copy((AddColumnEvent) from); - } else if (from instanceof DropColumnEvent) { - return DropColumnEventSerializer.INSTANCE.copy((DropColumnEvent) from); - } else { - throw new IllegalArgumentException("Unknown schema change event: " + from); - } + return from.visit( + new SchemaChangeEventVisitor() { + + @Override + public SchemaChangeEvent visit(AddColumnEvent event) { + return AddColumnEventSerializer.INSTANCE.copy(event); + } + + @Override + public SchemaChangeEvent visit(AlterColumnCommentEvent event) { + return AlterColumnCommentEventSerializer.INSTANCE.copy(event); + } + + @Override + public SchemaChangeEvent visit(AlterColumnTypeEvent event) { + return AlterColumnTypeEventSerializer.INSTANCE.copy(event); + } + + @Override + public SchemaChangeEvent visit(AlterTableCommentEvent event) { + return AlterTableCommentEventSerializer.INSTANCE.copy(event); + } + + @Override + public SchemaChangeEvent visit(CreateTableEvent event) { + return CreateTableEventSerializer.INSTANCE.copy(event); + } + + @Override + public SchemaChangeEvent visit(DropColumnEvent event) { + return DropColumnEventSerializer.INSTANCE.copy(event); + } + + @Override + public SchemaChangeEvent visit(DropTableEvent event) { + return DropTableEventSerializer.INSTANCE.copy(event); + } + + @Override + public SchemaChangeEvent visit(RenameColumnEvent event) { + return RenameColumnEventSerializer.INSTANCE.copy(event); + } + + @Override + public SchemaChangeEvent visit(TruncateTableEvent event) { + return TruncateTableEventSerializer.INSTANCE.copy(event); + } + }); } @Override @@ -95,25 +146,64 @@ public int getLength() { @Override public void serialize(SchemaChangeEvent record, DataOutputView target) throws IOException { - if (record instanceof AlterColumnTypeEvent) { - enumSerializer.serialize(SchemaChangeEventType.ALTER_COLUMN_TYPE, target); - AlterColumnTypeEventSerializer.INSTANCE.serialize( - (AlterColumnTypeEvent) record, target); - } else if (record instanceof CreateTableEvent) { - enumSerializer.serialize(SchemaChangeEventType.CREATE_TABLE, target); - CreateTableEventSerializer.INSTANCE.serialize((CreateTableEvent) record, target); - } else if (record instanceof RenameColumnEvent) { - enumSerializer.serialize(SchemaChangeEventType.RENAME_COLUMN, target); - RenameColumnEventSerializer.INSTANCE.serialize((RenameColumnEvent) record, target); - } else if (record instanceof AddColumnEvent) { - enumSerializer.serialize(SchemaChangeEventType.ADD_COLUMN, target); - AddColumnEventSerializer.INSTANCE.serialize((AddColumnEvent) record, target); - } else if (record instanceof DropColumnEvent) { - enumSerializer.serialize(SchemaChangeEventType.DROP_COLUMN, target); - DropColumnEventSerializer.INSTANCE.serialize((DropColumnEvent) record, target); - } else { - throw new IllegalArgumentException("Unknown schema change event: " + record); - } + + record.visit( + new SchemaChangeEventVisitorVoid() { + + @Override + public void visit(AddColumnEvent event) throws Exception { + enumSerializer.serialize(ADD_COLUMN, target); + AddColumnEventSerializer.INSTANCE.serialize(event, target); + } + + @Override + public void visit(AlterColumnCommentEvent event) throws Exception { + enumSerializer.serialize(ALTER_COLUMN_COMMENT, target); + AlterColumnCommentEventSerializer.INSTANCE.serialize(event, target); + } + + @Override + public void visit(AlterColumnTypeEvent event) throws Exception { + enumSerializer.serialize(ALTER_COLUMN_TYPE, target); + AlterColumnTypeEventSerializer.INSTANCE.serialize(event, target); + } + + @Override + public void visit(AlterTableCommentEvent event) throws Exception { + enumSerializer.serialize(ALTER_TABLE_COMMENT, target); + AlterTableCommentEventSerializer.INSTANCE.serialize(event, target); + } + + @Override + public void visit(CreateTableEvent event) throws Exception { + enumSerializer.serialize(CREATE_TABLE, target); + CreateTableEventSerializer.INSTANCE.serialize(event, target); + } + + @Override + public void visit(DropColumnEvent event) throws Exception { + enumSerializer.serialize(DROP_COLUMN, target); + DropColumnEventSerializer.INSTANCE.serialize(event, target); + } + + @Override + public void visit(DropTableEvent event) throws Exception { + enumSerializer.serialize(DROP_TABLE, target); + DropTableEventSerializer.INSTANCE.serialize(event, target); + } + + @Override + public void visit(RenameColumnEvent event) throws Exception { + enumSerializer.serialize(RENAME_COLUMN, target); + RenameColumnEventSerializer.INSTANCE.serialize(event, target); + } + + @Override + public void visit(TruncateTableEvent event) throws Exception { + enumSerializer.serialize(TRUNCATE_TABLE, target); + TruncateTableEventSerializer.INSTANCE.serialize(event, target); + } + }); } @Override @@ -130,6 +220,12 @@ public SchemaChangeEvent deserialize(DataInputView source) throws IOException { return RenameColumnEventSerializer.INSTANCE.deserialize(source); case ALTER_COLUMN_TYPE: return AlterColumnTypeEventSerializer.INSTANCE.deserialize(source); + case ALTER_COLUMN_COMMENT: + return AlterColumnCommentEventSerializer.INSTANCE.deserialize(source); + case DROP_TABLE: + return DropTableEventSerializer.INSTANCE.deserialize(source); + case TRUNCATE_TABLE: + return TruncateTableEventSerializer.INSTANCE.deserialize(source); default: throw new IllegalArgumentException( "Unknown schema change event class: " + schemaChangeEventType); diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/TruncateTableEventSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/TruncateTableEventSerializer.java new file mode 100644 index 0000000000..26eb6e744b --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/TruncateTableEventSerializer.java @@ -0,0 +1,108 @@ +/* + * 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.cdc.runtime.serializer.event; + +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; +import org.apache.flink.cdc.common.types.DataType; +import org.apache.flink.cdc.runtime.serializer.MapSerializer; +import org.apache.flink.cdc.runtime.serializer.StringSerializer; +import org.apache.flink.cdc.runtime.serializer.TableIdSerializer; +import org.apache.flink.cdc.runtime.serializer.TypeSerializerSingleton; +import org.apache.flink.cdc.runtime.serializer.schema.DataTypeSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; + +/** A {@link TypeSerializer} for {@link TruncateTableEvent}. */ +public class TruncateTableEventSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + /** Sharable instance of the TableIdSerializer. */ + public static final TruncateTableEventSerializer INSTANCE = new TruncateTableEventSerializer(); + + private final TableIdSerializer tableIdSerializer = TableIdSerializer.INSTANCE; + private final MapSerializer typeMapSerializer = + new MapSerializer<>(StringSerializer.INSTANCE, new DataTypeSerializer()); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TruncateTableEvent createInstance() { + return new TruncateTableEvent(TableId.tableId("unknown")); + } + + @Override + public TruncateTableEvent copy(TruncateTableEvent from) { + return new TruncateTableEvent(from.tableId()); + } + + @Override + public TruncateTableEvent copy(TruncateTableEvent from, TruncateTableEvent reuse) { + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(TruncateTableEvent record, DataOutputView target) throws IOException { + tableIdSerializer.serialize(record.tableId(), target); + } + + @Override + public TruncateTableEvent deserialize(DataInputView source) throws IOException { + return new TruncateTableEvent(tableIdSerializer.deserialize(source)); + } + + @Override + public TruncateTableEvent deserialize(TruncateTableEvent reuse, DataInputView source) + throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new TruncateTableEventSerializer.TruncateTableEventSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class TruncateTableEventSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public TruncateTableEventSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/SchemaEvolveTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/SchemaEvolveTest.java index 26a6276c05..351ecd7a0b 100644 --- a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/SchemaEvolveTest.java +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/SchemaEvolveTest.java @@ -249,7 +249,9 @@ public void testEvolveSchema() throws Exception { List alterColumnTypeEvents = Arrays.asList( new AlterColumnTypeEvent( - tableId, ImmutableMap.of("score", BIGINT, "toshi", FLOAT)), + tableId, + ImmutableMap.of("score", BIGINT, "toshi", FLOAT), + ImmutableMap.of("score", INT, "toshi", SMALLINT)), DataChangeEvent.insertEvent( tableId, buildRecord( @@ -504,7 +506,9 @@ public void testTryEvolveSchema() throws Exception { List alterColumnTypeEvents = Arrays.asList( new AlterColumnTypeEvent( - tableId, ImmutableMap.of("score", BIGINT, "toshi", FLOAT)), + tableId, + ImmutableMap.of("score", BIGINT, "toshi", FLOAT), + ImmutableMap.of("score", INT, "toshi", SMALLINT)), DataChangeEvent.insertEvent( tableId, buildRecord( @@ -1828,7 +1832,9 @@ public void testLenientSchemaEvolves() throws Exception { Column.physicalColumn( "toshi", SMALLINT, null)))), new AlterColumnTypeEvent( - tableId, Collections.singletonMap("name", STRING)), + tableId, + Collections.singletonMap("name", STRING), + Collections.singletonMap("name", STRING.notNull())), DataChangeEvent.insertEvent( tableId, buildRecord( @@ -1906,7 +1912,9 @@ public void testLenientSchemaEvolves() throws Exception { List alterColumnTypeEvents = Arrays.asList( new AlterColumnTypeEvent( - tableId, ImmutableMap.of("score", BIGINT, "toshi", FLOAT)), + tableId, + ImmutableMap.of("score", BIGINT, "toshi", FLOAT), + ImmutableMap.of("score", INT, "toshi", SMALLINT)), DataChangeEvent.insertEvent( tableId, buildRecord( @@ -1923,7 +1931,9 @@ public void testLenientSchemaEvolves() throws Exception { List lenientAlterColumnTypeEvents = Arrays.asList( new AlterColumnTypeEvent( - tableId, ImmutableMap.of("score", BIGINT, "toshi", FLOAT)), + tableId, + ImmutableMap.of("score", BIGINT, "toshi", FLOAT), + ImmutableMap.of("score", INT, "toshi", SMALLINT)), DataChangeEvent.insertEvent( tableId, buildRecord( @@ -2106,7 +2116,9 @@ tableId, buildRecord(INT, 12, INT, 0, SMALLINT, (short) 11)), List lenientDropColumnEvents = Arrays.asList( new AlterColumnTypeEvent( - tableId, Collections.singletonMap("name", STRING)), + tableId, + Collections.singletonMap("name", STRING), + Collections.singletonMap("name", STRING.notNull())), DataChangeEvent.insertEvent( tableId, buildRecord( @@ -2311,7 +2323,9 @@ tableId, buildRecord(INT, 12, INT, 0, SMALLINT, (short) 11)), new AddColumnEvent.ColumnWithPosition( Column.physicalColumn("yina", INT)))), new AlterColumnTypeEvent( - tableId, Collections.singletonMap("iina", INT)), + tableId, + Collections.singletonMap("iina", INT), + Collections.singletonMap("iina", INT.notNull())), DataChangeEvent.insertEvent( tableId, buildRecord( diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaDerivationTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaDerivationTest.java index 9a2d1cfb4f..445f6e3fd7 100644 --- a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaDerivationTest.java +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaDerivationTest.java @@ -380,8 +380,8 @@ void testIncompatibleTypes() { () -> schemaDerivation.applySchemaChange( new CreateTableEvent(TABLE_2, INCOMPATIBLE_SCHEMA))) - .isInstanceOf(IllegalStateException.class) - .hasMessage("Incompatible types: \"INT\" and \"STRING\""); + .hasRootCauseInstanceOf(IllegalStateException.class) + .hasRootCauseMessage("Incompatible types: \"INT\" and \"STRING\""); } @Test diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/AlterColumnCommentEventSerializerTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/AlterColumnCommentEventSerializerTest.java new file mode 100644 index 0000000000..346a6b580c --- /dev/null +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/AlterColumnCommentEventSerializerTest.java @@ -0,0 +1,64 @@ +/* + * 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.cdc.runtime.serializer.event; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.cdc.common.event.AlterColumnCommentEvent; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.runtime.serializer.SerializerTestBase; + +import java.util.HashMap; +import java.util.Map; + +/** A test for the {@link AlterColumnCommentEventSerializer}. */ +public class AlterColumnCommentEventSerializerTest + extends SerializerTestBase { + @Override + protected TypeSerializer createSerializer() { + return AlterColumnCommentEventSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return AlterColumnCommentEvent.class; + } + + @Override + protected AlterColumnCommentEvent[] getTestData() { + Map map = new HashMap<>(); + map.put("col1", "Comments of Column One"); + map.put("col2", "Comments of Column Two"); + Map oldMap = new HashMap<>(); + oldMap.put("col1", "Old Comments of Column One"); + oldMap.put("col2", "Old Comments of Column Two"); + return new AlterColumnCommentEvent[] { + new AlterColumnCommentEvent(TableId.tableId("table"), map), + new AlterColumnCommentEvent(TableId.tableId("schema", "table"), map), + new AlterColumnCommentEvent(TableId.tableId("namespace", "schema", "table"), map), + new AlterColumnCommentEvent(TableId.tableId("table"), map, oldMap), + new AlterColumnCommentEvent(TableId.tableId("schema", "table"), map, oldMap), + new AlterColumnCommentEvent( + TableId.tableId("namespace", "schema", "table"), map, oldMap) + }; + } +} diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/AlterColumnTypeEventSerializerTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/AlterColumnTypeEventSerializerTest.java index baddd0bf20..1c45d71771 100644 --- a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/AlterColumnTypeEventSerializerTest.java +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/AlterColumnTypeEventSerializerTest.java @@ -49,10 +49,17 @@ protected AlterColumnTypeEvent[] getTestData() { Map map = new HashMap<>(); map.put("col1", DataTypes.BYTES()); map.put("col2", DataTypes.TIME()); + + Map oldMap = new HashMap<>(); + oldMap.put("col1", DataTypes.TIME()); + oldMap.put("col2", DataTypes.BYTES()); return new AlterColumnTypeEvent[] { new AlterColumnTypeEvent(TableId.tableId("table"), map), new AlterColumnTypeEvent(TableId.tableId("schema", "table"), map), - new AlterColumnTypeEvent(TableId.tableId("namespace", "schema", "table"), map) + new AlterColumnTypeEvent(TableId.tableId("namespace", "schema", "table"), map), + new AlterColumnTypeEvent(TableId.tableId("table"), map, oldMap), + new AlterColumnTypeEvent(TableId.tableId("schema", "table"), map, oldMap), + new AlterColumnTypeEvent(TableId.tableId("namespace", "schema", "table"), map, oldMap) }; } } diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/AlterTableCommentEventSerializerTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/AlterTableCommentEventSerializerTest.java new file mode 100644 index 0000000000..897b17a72a --- /dev/null +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/AlterTableCommentEventSerializerTest.java @@ -0,0 +1,52 @@ +/* + * 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.cdc.runtime.serializer.event; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.cdc.common.event.AlterTableCommentEvent; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.runtime.serializer.SerializerTestBase; + +/** A test for the {@link AlterColumnCommentEventSerializer}. */ +public class AlterTableCommentEventSerializerTest + extends SerializerTestBase { + @Override + protected TypeSerializer createSerializer() { + return AlterTableCommentEventSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return AlterTableCommentEvent.class; + } + + @Override + protected AlterTableCommentEvent[] getTestData() { + return new AlterTableCommentEvent[] { + new AlterTableCommentEvent(TableId.tableId("table"), "No Comments"), + new AlterTableCommentEvent(TableId.tableId("schema", "table"), "No more comments"), + new AlterTableCommentEvent( + TableId.tableId("namespace", "schema", "table"), "No any comments"), + }; + } +} diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/DropTableEventSerializerTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/DropTableEventSerializerTest.java new file mode 100644 index 0000000000..bbeb92c413 --- /dev/null +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/DropTableEventSerializerTest.java @@ -0,0 +1,50 @@ +/* + * 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.cdc.runtime.serializer.event; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.cdc.common.event.DropTableEvent; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.runtime.serializer.SerializerTestBase; + +/** A test for the {@link RenameTableEventSerializer}. */ +public class DropTableEventSerializerTest extends SerializerTestBase { + @Override + protected TypeSerializer createSerializer() { + return DropTableEventSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return DropTableEvent.class; + } + + @Override + protected DropTableEvent[] getTestData() { + return new DropTableEvent[] { + new DropTableEvent(TableId.tableId("table")), + new DropTableEvent(TableId.tableId("schema", "table")), + new DropTableEvent(TableId.tableId("namespace", "schema", "table")) + }; + } +} diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/TruncateTableEventSerializerTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/TruncateTableEventSerializerTest.java new file mode 100644 index 0000000000..ba2f439b19 --- /dev/null +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/TruncateTableEventSerializerTest.java @@ -0,0 +1,51 @@ +/* + * 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.cdc.runtime.serializer.event; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; +import org.apache.flink.cdc.runtime.serializer.SerializerTestBase; + +/** A test for the {@link RenameTableEventSerializer}. */ +public class TruncateTableEventSerializerTest extends SerializerTestBase { + @Override + protected TypeSerializer createSerializer() { + + return TruncateTableEventSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return TruncateTableEvent.class; + } + + @Override + protected TruncateTableEvent[] getTestData() { + return new TruncateTableEvent[] { + new TruncateTableEvent(TableId.tableId("table")), + new TruncateTableEvent(TableId.tableId("schema", "table")), + new TruncateTableEvent(TableId.tableId("namespace", "schema", "table")) + }; + } +}