From f22c3a4b54f6bcffc78dcb28aa6de6acdc628f80 Mon Sep 17 00:00:00 2001 From: yuxiqian <34335406+yuxiqian@users.noreply.github.com> Date: Wed, 8 May 2024 17:15:42 +0800 Subject: [PATCH] [FLINK-35243][base] Support more schema change events & PreSchema backfill --- .../common/event/AlterColumnCommentEvent.java | 143 ++++++++ .../common/event/AlterColumnTypeEvent.java | 80 ++++- .../common/event/AlterTableCommentEvent.java | 72 ++++ .../cdc/common/event/DropTableEvent.java | 62 ++++ .../cdc/common/event/RenameTableEvent.java | 70 ++++ .../cdc/common/event/SchemaChangeEvent.java | 11 +- .../event/SchemaChangeEventHandler.java | 308 ++++++++++++++++++ .../event/SchemaChangeEventVoidHandler.java | 298 +++++++++++++++++ .../event/SchemaChangeEventWithPreSchema.java | 35 ++ .../cdc/common/event/TruncateTableEvent.java | 62 ++++ .../cdc/common/utils/ChangeEventUtils.java | 57 ++-- .../flink/cdc/common/utils/SchemaUtils.java | 56 +++- .../cdc/common/utils/SchemaUtilsTest.java | 16 +- .../flink/FlinkPipelineComposerITCase.java | 2 +- .../doris/sink/DorisEventSerializer.java | 14 +- .../doris/sink/DorisMetadataApplier.java | 22 +- .../canal/CanalJsonSerializationSchema.java | 2 +- .../DebeziumJsonSerializationSchema.java | 2 +- .../CustomAlterTableParserListener.java | 73 +++++ .../parser/CustomMySqlAntlrDdlParser.java | 3 +- .../mysql/source/MySqlPipelineITCase.java | 224 ++++++++++++- .../paimon/sink/PaimonMetadataApplier.java | 115 ++++--- .../sink/v2/PaimonRecordEventSerializer.java | 2 +- .../sink/EventRecordSerializationSchema.java | 2 +- .../sink/StarRocksMetadataApplier.java | 28 +- .../EventRecordSerializationSchemaTest.java | 4 +- .../cdc/connectors/values/ValuesDatabase.java | 71 +++- .../values/sink/ValuesDataSink.java | 20 +- .../values/sink/ValuesDataSinkFunction.java | 23 +- .../connectors/values/ValuesDatabaseTest.java | 22 ++ .../cdc/pipeline/tests/MysqlE2eITCase.java | 55 ++-- .../operators/schema/SchemaOperator.java | 22 ++ .../schema/coordinator/SchemaDerivation.java | 132 ++++++-- .../schema/coordinator/SchemaManager.java | 7 +- .../transform/TransformDataOperator.java | 2 +- .../transform/TransformSchemaOperator.java | 6 +- .../AlterColumnCommentEventSerializer.java | 119 +++++++ .../event/AlterColumnTypeEventSerializer.java | 9 +- .../AlterTableCommentEventSerializer.java | 108 ++++++ .../event/DropTableEventSerializer.java | 108 ++++++ .../event/RenameTableEventSerializer.java | 110 +++++++ .../event/SchemaChangeEventSerializer.java | 131 +++++--- .../event/TruncateTableEventSerializer.java | 108 ++++++ .../coordinator/SchemaDerivationTest.java | 8 +- ...AlterColumnCommentEventSerializerTest.java | 64 ++++ .../AlterColumnTypeEventSerializerTest.java | 9 +- .../AlterTableCommentEventSerializerTest.java | 52 +++ .../event/DropTableEventSerializerTest.java | 50 +++ .../event/RenameTableEventSerializerTest.java | 53 +++ .../TruncateTableEventSerializerTest.java | 51 +++ 50 files changed, 2857 insertions(+), 246 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/DropTableEvent.java create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/RenameTableEvent.java create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventHandler.java create mode 100644 flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventVoidHandler.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/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/RenameTableEventSerializer.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/RenameTableEventSerializerTest.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/AlterColumnCommentEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterColumnCommentEvent.java new file mode 100644 index 00000000000..0c61dd36f6d --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterColumnCommentEvent.java @@ -0,0 +1,143 @@ +/* + * 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 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(); + } +} 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 5eb5da0d121..1446cda7992 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,21 @@ 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 SchemaChangeEventWithPreSchema { private static final long serialVersionUID = 1L; @@ -35,9 +40,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,26 +72,71 @@ 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 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(); + } } 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 00000000000..44e4d9535ac --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterTableCommentEvent.java @@ -0,0 +1,72 @@ +/* + * 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 SchemaChangeEvent { + 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; + } +} 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 00000000000..e3738ca091e --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/DropTableEvent.java @@ -0,0 +1,62 @@ +/* + * 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 SchemaChangeEvent { + 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; + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/RenameTableEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/RenameTableEvent.java new file mode 100644 index 00000000000..a8cfe9c5e5e --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/RenameTableEvent.java @@ -0,0 +1,70 @@ +/* + * 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 RENAME TABLE} DDL. this will be sent by + * {@link DataSource} before all {@link DataChangeEvent} with the same tableId. + */ +public class RenameTableEvent implements SchemaChangeEvent { + private static final long serialVersionUID = 1L; + + private final TableId tableId; + private final TableId newTableId; + + public RenameTableEvent(TableId tableId, TableId newTableId) { + this.tableId = tableId; + this.newTableId = newTableId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof RenameTableEvent)) { + return false; + } + RenameTableEvent that = (RenameTableEvent) o; + return Objects.equals(tableId, that.tableId) && Objects.equals(newTableId, that.newTableId); + } + + @Override + public int hashCode() { + return Objects.hash(tableId, newTableId); + } + + @Override + public String toString() { + return "RenameTableEvent{" + "tableId=" + tableId + ", newTableId=" + newTableId + '}'; + } + + @Override + public TableId tableId() { + return tableId; + } + + /** Returns the new table name. */ + public TableId newTableId() { + return newTableId; + } +} 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 1d37860fb5d..f1558326452 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 @@ -26,4 +26,13 @@ * system, such as CREATE, DROP, RENAME and so on. */ @PublicEvolving -public interface SchemaChangeEvent extends ChangeEvent, Serializable {} +public interface SchemaChangeEvent extends ChangeEvent, Serializable { + + static SchemaChangeEventHandler handle(SchemaChangeEvent event) { + return new SchemaChangeEventHandler<>(event); + } + + static SchemaChangeEventVoidHandler handleWithoutResult(SchemaChangeEvent event) { + return new SchemaChangeEventVoidHandler(event); + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventHandler.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventHandler.java new file mode 100644 index 00000000000..c4e79a3989f --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventHandler.java @@ -0,0 +1,308 @@ +/* + * 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.util.FlinkRuntimeException; + +import java.util.Optional; +import java.util.function.Supplier; + +/** Handles {@code SchemaChangeEvent} of various types and yields a result. */ +public class SchemaChangeEventHandler { + + private final SchemaChangeEvent event; + private boolean handled; + private T result; + + SchemaChangeEventHandler(SchemaChangeEvent event) { + this.event = event; + this.handled = false; + this.result = null; + } + + // Beginning of event handlers + + /** Handles a {@code AddColumnEvent} and yields result. */ + @FunctionalInterface + public interface AddColumnEventHandler { + U handleThrows(AddColumnEvent event) throws Exception; + + default U handle(AddColumnEvent event) { + try { + return handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventHandler onAddColumn(AddColumnEventHandler handler) { + if (event instanceof AddColumnEvent) { + this.result = handler.handle((AddColumnEvent) event); + this.handled = true; + } + return this; + } + + /** Handles a {@code AlterColumnCommentEvent} and yields result. */ + @FunctionalInterface + public interface AlterColumnCommentEventHandler { + U handleThrows(AlterColumnCommentEvent event) throws Exception; + + default U handle(AlterColumnCommentEvent event) { + try { + return handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventHandler onAlterColumnComment( + AlterColumnCommentEventHandler handler) { + if (event instanceof AlterColumnCommentEvent) { + this.result = handler.handle((AlterColumnCommentEvent) event); + this.handled = true; + } + return this; + } + + /** Handles a {@code AlterColumnTypeEvent} and yields result. */ + @FunctionalInterface + public interface AlterColumnTypeEventHandler { + U handleThrows(AlterColumnTypeEvent event) throws Exception; + + default U handle(AlterColumnTypeEvent event) { + try { + return handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventHandler onAlterColumnType(AlterColumnTypeEventHandler handler) { + if (event instanceof AlterColumnTypeEvent) { + this.result = handler.handle((AlterColumnTypeEvent) event); + this.handled = true; + } + return this; + } + + /** Handles a {@code AlterTableCommentEvent} and yields result. */ + @FunctionalInterface + public interface AlterTableCommentEventHandler { + U handleThrows(AlterTableCommentEvent event) throws Exception; + + default U handle(AlterTableCommentEvent event) { + try { + return handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventHandler onAlterTableComment( + AlterTableCommentEventHandler handler) { + if (event instanceof AlterTableCommentEvent) { + this.result = handler.handle((AlterTableCommentEvent) event); + this.handled = true; + } + return this; + } + + /** Handles a {@code CreateTableEvent} and yields result. */ + @FunctionalInterface + public interface CreateTableEventHandler { + U handleThrows(CreateTableEvent event) throws Exception; + + default U handle(CreateTableEvent event) { + try { + return handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventHandler onCreateTable(CreateTableEventHandler handler) { + if (event instanceof CreateTableEvent) { + this.result = handler.handle((CreateTableEvent) event); + this.handled = true; + } + return this; + } + + /** Handles a {@code DropColumnEvent} and yields result. */ + @FunctionalInterface + public interface DropColumnEventHandler { + U handleThrows(DropColumnEvent event) throws Exception; + + default U handle(DropColumnEvent event) { + try { + return handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventHandler onDropColumn(DropColumnEventHandler handler) { + if (event instanceof DropColumnEvent) { + this.result = handler.handle((DropColumnEvent) event); + this.handled = true; + } + return this; + } + + /** Handles a {@code DropTableEvent} and yields result. */ + @FunctionalInterface + public interface DropTableEventHandler { + U handleThrows(DropTableEvent event) throws Exception; + + default U handle(DropTableEvent event) { + try { + return handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventHandler onDropTable(DropTableEventHandler handler) { + if (event instanceof DropTableEvent) { + this.result = handler.handle((DropTableEvent) event); + this.handled = true; + } + return this; + } + + /** Handles a {@code RenameColumnEvent} and yields result. */ + @FunctionalInterface + public interface RenameColumnEventHandler { + U handleThrows(RenameColumnEvent event) throws Exception; + + default U handle(RenameColumnEvent event) { + try { + return handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventHandler onRenameColumn(RenameColumnEventHandler handler) { + if (event instanceof RenameColumnEvent) { + this.result = handler.handle((RenameColumnEvent) event); + this.handled = true; + } + return this; + } + + /** Handles a {@code RenameTableEvent} and yields result. */ + @FunctionalInterface + public interface RenameTableEventHandler { + U handleThrows(RenameTableEvent event) throws Exception; + + default U handle(RenameTableEvent event) { + try { + return handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventHandler onRenameTable(RenameTableEventHandler handler) { + if (event instanceof RenameTableEvent) { + this.result = handler.handle((RenameTableEvent) event); + this.handled = true; + } + return this; + } + + /** Handles a {@code TruncateTableEvent} and yields result. */ + @FunctionalInterface + public interface TruncateTableEventHandler { + U handleThrows(TruncateTableEvent event) throws Exception; + + default U handle(TruncateTableEvent event) { + try { + return handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventHandler onTruncateTable(TruncateTableEventHandler handler) { + if (event instanceof TruncateTableEvent) { + this.result = handler.handle((TruncateTableEvent) event); + this.handled = true; + } + return this; + } + + // End of event handlers + + /** Handles any {@code SchemaChangeEvent} and yields result. */ + @FunctionalInterface + public interface FallbackEventHandler { + U handleThrows(SchemaChangeEvent event) throws Exception; + + default U handle(SchemaChangeEvent event) { + try { + return handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public Optional get() { + if (handled) { + return Optional.of(result); + } + return Optional.empty(); + } + + public SchemaChangeEventHandler orElse(FallbackEventHandler handler) { + this.result = handler.handle(event); + this.handled = true; + return this; + } + + public void orElseThrow(Supplier exceptionSupplier) + throws X { + if (!handled) { + throw exceptionSupplier.get(); + } + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventVoidHandler.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventVoidHandler.java new file mode 100644 index 00000000000..d78d979856c --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventVoidHandler.java @@ -0,0 +1,298 @@ +/* + * 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.util.FlinkRuntimeException; + +import java.util.function.Supplier; + +/** Handles {@code SchemaChangeEvent} of various types but yields no result. */ +public class SchemaChangeEventVoidHandler { + + private final SchemaChangeEvent event; + private boolean handled; + + SchemaChangeEventVoidHandler(SchemaChangeEvent event) { + this.event = event; + this.handled = false; + } + + // Beginning of event handlers + + /** Handles a {@code AddColumnEvent}. */ + @FunctionalInterface + public interface AddColumnEventHandlerVoid { + void handleThrows(AddColumnEvent event) throws Exception; + + default void handle(AddColumnEvent event) { + try { + handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventVoidHandler onAddColumn(AddColumnEventHandlerVoid handler) { + if (event instanceof AddColumnEvent) { + handler.handle((AddColumnEvent) event); + this.handled = true; + } + return this; + } + + /** Handles a {@code AlterColumnCommentEvent}. */ + @FunctionalInterface + public interface AlterColumnCommentEventHandlerVoid { + void handleThrows(AlterColumnCommentEvent event) throws Exception; + + default void handle(AlterColumnCommentEvent event) { + try { + handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventVoidHandler onAlterColumnComment( + AlterColumnCommentEventHandlerVoid handler) { + if (event instanceof AlterColumnCommentEvent) { + handler.handle((AlterColumnCommentEvent) event); + this.handled = true; + } + return this; + } + + /** Handles a {@code AlterColumnTypeEvent}. */ + @FunctionalInterface + public interface AlterColumnTypeEventHandlerVoid { + void handleThrows(AlterColumnTypeEvent event) throws Exception; + + default void handle(AlterColumnTypeEvent event) { + try { + handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventVoidHandler onAlterColumnType(AlterColumnTypeEventHandlerVoid handler) { + if (event instanceof AlterColumnTypeEvent) { + handler.handle((AlterColumnTypeEvent) event); + this.handled = true; + } + return this; + } + + /** Handles a {@code AlterTableCommentEvent}. */ + @FunctionalInterface + public interface AlterTableCommentEventHandlerVoid { + void handleThrows(AlterTableCommentEvent event) throws Exception; + + default void handle(AlterTableCommentEvent event) { + try { + handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventVoidHandler onAlterTableComment( + AlterTableCommentEventHandlerVoid handler) { + if (event instanceof AlterTableCommentEvent) { + handler.handle((AlterTableCommentEvent) event); + this.handled = true; + } + return this; + } + + /** Handles a {@code CreateTableEvent}. */ + @FunctionalInterface + public interface CreateTableEventHandlerVoid { + void handleThrows(CreateTableEvent event) throws Exception; + + default void handle(CreateTableEvent event) { + try { + handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventVoidHandler onCreateTable(CreateTableEventHandlerVoid handler) { + if (event instanceof CreateTableEvent) { + handler.handle((CreateTableEvent) event); + this.handled = true; + } + return this; + } + + /** Handles a {@code DropColumnEvent}. */ + @FunctionalInterface + public interface DropColumnEventHandlerVoid { + void handleThrows(DropColumnEvent event) throws Exception; + + default void handle(DropColumnEvent event) { + try { + handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventVoidHandler onDropColumn(DropColumnEventHandlerVoid handler) { + if (event instanceof DropColumnEvent) { + handler.handle((DropColumnEvent) event); + this.handled = true; + } + return this; + } + + /** Handles a {@code DropTableEvent}. */ + @FunctionalInterface + public interface DropTableEventHandlerVoid { + void handleThrows(DropTableEvent event) throws Exception; + + default void handle(DropTableEvent event) { + try { + handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventVoidHandler onDropTable(DropTableEventHandlerVoid handler) { + if (event instanceof DropTableEvent) { + handler.handle((DropTableEvent) event); + this.handled = true; + } + return this; + } + + /** Handles a {@code RenameColumnEvent}. */ + @FunctionalInterface + public interface RenameColumnEventHandlerVoid { + void handleThrows(RenameColumnEvent event) throws Exception; + + default void handle(RenameColumnEvent event) { + try { + handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventVoidHandler onRenameColumn(RenameColumnEventHandlerVoid handler) { + if (event instanceof RenameColumnEvent) { + handler.handle((RenameColumnEvent) event); + this.handled = true; + } + return this; + } + + /** Handles a {@code RenameTableEvent}. */ + @FunctionalInterface + public interface RenameTableEventHandlerVoid { + void handleThrows(RenameTableEvent event) throws Exception; + + default void handle(RenameTableEvent event) { + try { + handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventVoidHandler onRenameTable(RenameTableEventHandlerVoid handler) { + if (event instanceof RenameTableEvent) { + handler.handle((RenameTableEvent) event); + this.handled = true; + } + return this; + } + + /** Handles a {@code TruncateTableEvent}. */ + @FunctionalInterface + public interface TruncateTableEventHandlerVoid { + void handleThrows(TruncateTableEvent event) throws Exception; + + default void handle(TruncateTableEvent event) { + try { + handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventVoidHandler onTruncateTable(TruncateTableEventHandlerVoid handler) { + if (event instanceof TruncateTableEvent) { + handler.handle((TruncateTableEvent) event); + this.handled = true; + } + return this; + } + + // End of event handlers + + /** Handles any {@code SchemaChangeEvent} yields no result. */ + @FunctionalInterface + public interface FallbackEventHandler { + void handleThrows(SchemaChangeEvent event) throws Exception; + + default void handle(SchemaChangeEvent event) { + try { + handleThrows(event); + } catch (final Exception ex) { + throw new FlinkRuntimeException( + "Exception occurred during handling event " + event, ex); + } + } + } + + public SchemaChangeEventVoidHandler orElse(FallbackEventHandler handler) { + handler.handle(event); + this.handled = true; + return this; + } + + public void orElseThrow(Supplier exceptionSupplier) + throws X { + if (!handled) { + throw exceptionSupplier.get(); + } + } +} 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 00000000000..541ae090298 --- /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/TruncateTableEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TruncateTableEvent.java new file mode 100644 index 00000000000..74340450687 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TruncateTableEvent.java @@ -0,0 +1,62 @@ +/* + * 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 SchemaChangeEvent { + 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; + } +} 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 1825dd26233..c2dbc6f5d73 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 @@ -18,13 +18,18 @@ package org.apache.flink.cdc.common.utils; 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.RenameTableEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; /** Utilities for handling {@link org.apache.flink.cdc.common.event.ChangeEvent}s. */ public class ChangeEventUtils { @@ -56,29 +61,33 @@ 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.handle(schemaChangeEvent) + .onAddColumn(event -> new AddColumnEvent(tableId, event.getAddedColumns())) + .onAlterColumnComment( + event -> + new AlterColumnCommentEvent( + tableId, + event.getCommentMapping(), + event.getOldCommentMapping())) + .onAlterColumnType( + event -> + new AlterColumnTypeEvent( + tableId, event.getTypeMapping(), event.getOldTypeMapping())) + .onAlterTableComment( + event -> new AlterTableCommentEvent(tableId, event.getTableComment())) + .onCreateTable(event -> new CreateTableEvent(tableId, event.getSchema())) + .onDropColumn(event -> new DropColumnEvent(tableId, event.getDroppedColumnNames())) + .onDropTable(event -> new DropTableEvent(tableId)) + .onRenameColumn(event -> new RenameColumnEvent(tableId, event.getNameMapping())) + .onRenameTable(event -> new RenameTableEvent(tableId, event.newTableId())) + .onTruncateTable(event -> new TruncateTableEvent(tableId)) + .get() + .orElseThrow( + () -> + new UnsupportedOperationException( + String.format( + "Unsupported schema change event with type \"%s\"", + schemaChangeEvent.getClass().getCanonicalName()))); } } 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 107e63d821c..eeea4707905 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 @@ -20,7 +20,9 @@ import org.apache.flink.cdc.common.annotation.PublicEvolving; 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.DropColumnEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; @@ -57,21 +59,25 @@ public static List createFieldGetters(List colum } /** 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())); - } + public static Schema applyColumnSchemaChangeEvent(Schema schema, SchemaChangeEvent event) { + return SchemaChangeEvent.handle(event) + .onAddColumn(e -> applyAddColumnEvent(e, schema)) + .onAlterColumnComment(e -> applyAlterColumnCommentEvent(e, schema)) + .onAlterColumnType(e -> applyAlterColumnTypeEvent(e, schema)) + .onAlterTableComment(e -> schema) + .onCreateTable(CreateTableEvent::getSchema) + .onDropColumn(e -> applyDropColumnEvent(e, schema)) + .onDropTable(e -> schema) + .onRenameColumn(e -> applyRenameColumnEvent(e, schema)) + .onRenameTable(e -> schema) + .onTruncateTable(e -> schema) + .get() + .orElseThrow( + () -> + new UnsupportedOperationException( + String.format( + "Unsupported schema change event type \"%s\"", + event.getClass().getCanonicalName()))); } private static Schema applyAddColumnEvent(AddColumnEvent event, Schema oldSchema) { @@ -166,4 +172,24 @@ 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.physicalColumn( + column.getName(), + column.getType(), + 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/SchemaUtilsTest.java b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/SchemaUtilsTest.java index 83fb358b2ce..0517e76f887 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 @@ -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() @@ -54,7 +54,7 @@ public void testApplySchemaChangeEvent() { new AddColumnEvent.ColumnWithPosition( Column.physicalColumn("col3", DataTypes.STRING()))); AddColumnEvent addColumnEvent = new AddColumnEvent(tableId, addedColumns); - schema = SchemaUtils.applySchemaChangeEvent(schema, addColumnEvent); + schema = SchemaUtils.applyColumnSchemaChangeEvent(schema, addColumnEvent); Assert.assertEquals( schema, Schema.newBuilder() @@ -71,7 +71,7 @@ public void testApplySchemaChangeEvent() { AddColumnEvent.ColumnPosition.BEFORE, "col3")); addColumnEvent = new AddColumnEvent(tableId, addedColumns); - schema = SchemaUtils.applySchemaChangeEvent(schema, addColumnEvent); + schema = SchemaUtils.applyColumnSchemaChangeEvent(schema, addColumnEvent); Assert.assertEquals( schema, Schema.newBuilder() @@ -89,7 +89,7 @@ public void testApplySchemaChangeEvent() { AddColumnEvent.ColumnPosition.AFTER, "col4")); addColumnEvent = new AddColumnEvent(tableId, addedColumns); - schema = SchemaUtils.applySchemaChangeEvent(schema, addColumnEvent); + schema = SchemaUtils.applyColumnSchemaChangeEvent(schema, addColumnEvent); Assert.assertEquals( schema, Schema.newBuilder() @@ -108,7 +108,7 @@ public void testApplySchemaChangeEvent() { AddColumnEvent.ColumnPosition.FIRST, null)); addColumnEvent = new AddColumnEvent(tableId, addedColumns); - schema = SchemaUtils.applySchemaChangeEvent(schema, addColumnEvent); + schema = SchemaUtils.applyColumnSchemaChangeEvent(schema, addColumnEvent); Assert.assertEquals( schema, Schema.newBuilder() @@ -123,7 +123,7 @@ public void testApplySchemaChangeEvent() { // drop columns DropColumnEvent dropColumnEvent = new DropColumnEvent(tableId, Arrays.asList("col3", "col5")); - schema = SchemaUtils.applySchemaChangeEvent(schema, dropColumnEvent); + schema = SchemaUtils.applyColumnSchemaChangeEvent(schema, dropColumnEvent); Assert.assertEquals( schema, Schema.newBuilder() @@ -138,7 +138,7 @@ public void testApplySchemaChangeEvent() { nameMapping.put("col2", "newCol2"); nameMapping.put("col4", "newCol4"); RenameColumnEvent renameColumnEvent = new RenameColumnEvent(tableId, nameMapping); - schema = SchemaUtils.applySchemaChangeEvent(schema, renameColumnEvent); + schema = SchemaUtils.applyColumnSchemaChangeEvent(schema, renameColumnEvent); Assert.assertEquals( schema, Schema.newBuilder() @@ -153,7 +153,7 @@ public void testApplySchemaChangeEvent() { typeMapping.put("newCol2", DataTypes.VARCHAR(10)); typeMapping.put("newCol4", DataTypes.VARCHAR(10)); AlterColumnTypeEvent alterColumnTypeEvent = new AlterColumnTypeEvent(tableId, typeMapping); - schema = SchemaUtils.applySchemaChangeEvent(schema, alterColumnTypeEvent); + schema = SchemaUtils.applyColumnSchemaChangeEvent(schema, alterColumnTypeEvent); Assert.assertEquals( 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 b59f4ead641..0cac14fe5e2 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 @@ -648,7 +648,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=()}", 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 a3d6f21d0cc..2c4621a8fa0 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 @@ -20,10 +20,13 @@ import org.apache.flink.cdc.common.data.RecordData; 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.Event; import org.apache.flink.cdc.common.event.OperationType; +import org.apache.flink.cdc.common.event.RenameTableEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; 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.utils.Preconditions; @@ -74,13 +77,22 @@ public DorisRecord serialize(Event event) throws IOException { TableId tableId = schemaChangeEvent.tableId(); if (event instanceof CreateTableEvent) { schemaMaps.put(tableId, ((CreateTableEvent) event).getSchema()); + } else if (event instanceof RenameTableEvent) { + RenameTableEvent renameTableEvent = (RenameTableEvent) event; + schemaMaps.put( + renameTableEvent.newTableId(), + schemaMaps.remove(renameTableEvent.tableId())); + } else if (event instanceof TruncateTableEvent) { + // Truncate table does not affect the schema + } else if (event instanceof DropColumnEvent) { + schemaMaps.remove(((DropColumnEvent) event).tableId()); } else { if (!schemaMaps.containsKey(tableId)) { throw new RuntimeException("schema of " + tableId + " is not existed."); } schemaMaps.put( tableId, - SchemaUtils.applySchemaChangeEvent( + SchemaUtils.applyColumnSchemaChangeEvent( schemaMaps.get(tableId), schemaChangeEvent)); } } 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 7e107eac17f..5c49ce0dfee 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,7 +19,6 @@ import org.apache.flink.cdc.common.configuration.Configuration; import org.apache.flink.cdc.common.event.AddColumnEvent; -import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DropColumnEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; @@ -71,17 +70,16 @@ public DorisMetadataApplier(DorisOptions dorisOptions, Configuration config) { public void applySchemaChange(SchemaChangeEvent event) { 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) { - throw new RuntimeException("Unsupported schema change event, " + event); - } + SchemaChangeEvent.handleWithoutResult(event) + .onAddColumn(this::applyAddColumnEvent) + .onCreateTable(this::applyCreateTableEvent) + .onDropColumn(this::applyDropColumnEvent) + .onRenameColumn(this::applyRenameColumnEvent) + .orElseThrow( + () -> + new UnsupportedOperationException( + "Doris data sink doesn't support schema change event " + + event)); } catch (Exception ex) { throw new RuntimeException( "Failed to schema change, " + event + ", reason: " + ex.getMessage()); 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 eecd2b80197..9d9a3c0c362 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 @@ -106,7 +106,7 @@ public byte[] serialize(Event event) { schema = createTableEvent.getSchema(); } else { schema = - SchemaUtils.applySchemaChangeEvent( + SchemaUtils.applyColumnSchemaChangeEvent( jsonSerializers.get(schemaChangeEvent.tableId()).getSchema(), schemaChangeEvent); } 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 3fa1d4ce19f..37e541e64be 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 @@ -105,7 +105,7 @@ public byte[] serialize(Event event) { schema = createTableEvent.getSchema(); } else { schema = - SchemaUtils.applySchemaChangeEvent( + SchemaUtils.applyColumnSchemaChangeEvent( jsonSerializers.get(schemaChangeEvent.tableId()).getSchema(), schemaChangeEvent); } 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 6f3fb9a8ccc..2419bb36252 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,14 @@ 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.RenameTableEvent; 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 +210,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); @@ -217,6 +225,36 @@ public void exitAlterByChangeColumn(MySqlParser.AlterByChangeColumnContext ctx) super.exitAlterByChangeColumn(ctx); } + @Override + public void enterAlterByModifyColumn(MySqlParser.AlterByModifyColumnContext ctx) { + String columnName = parser.parseName(ctx.uid(0)); + ColumnEditor columnEditor = Column.editor().name(columnName); + columnDefinitionListener = + new CustomColumnDefinitionParserListener(columnEditor, parser, listeners); + listeners.add(columnDefinitionListener); + super.enterAlterByModifyColumn(ctx); + } + + @Override + 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); + super.exitAlterByModifyColumn(ctx); + } + @Override public void enterAlterByDropColumn(MySqlParser.AlterByDropColumnContext ctx) { String removedColName = parser.parseName(ctx.uid()); @@ -251,6 +289,41 @@ public void exitAlterByRenameColumn(MySqlParser.AlterByRenameColumnContext ctx) super.exitAlterByRenameColumn(ctx); } + @Override + public void enterRenameTable(MySqlParser.RenameTableContext ctx) { + ctx.renameTableClause() + .forEach( + (clause -> { + TableId oldTableId = + parser.parseQualifiedTableId(clause.tableName(0).fullId()); + TableId newTableId = + parser.parseQualifiedTableId(clause.tableName(1).fullId()); + changes.add( + new RenameTableEvent( + toCdcTableId(oldTableId), toCdcTableId(newTableId))); + })); + super.enterRenameTable(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(), fromDbzColumn(dbzColumn), dbzColumn.comment()); 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 1264aa8d683..ffe4b2079a7 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 5eb6ce0e598..d55908c5ef0 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,17 @@ 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.RenameTableEvent; 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 +65,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 +355,180 @@ 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 rename clause DDL + statement.execute( + String.format( + "RENAME TABLE `%s`.`customers` TO `consumers`;", + inventoryDatabase.getDatabaseName())); + expected.add( + new RenameTableEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "customers"), + TableId.tableId(inventoryDatabase.getDatabaseName(), "consumers"))); + + // Test multiple rename clauses support + statement.execute( + String.format( + "RENAME TABLE `%s`.`products` TO `goods`, `%s`.`orders` TO `bills`;", + inventoryDatabase.getDatabaseName(), + inventoryDatabase.getDatabaseName())); + expected.add( + new RenameTableEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "products"), + TableId.tableId(inventoryDatabase.getDatabaseName(), "goods"))); + + expected.add( + new RenameTableEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "orders"), + TableId.tableId(inventoryDatabase.getDatabaseName(), "bills"))); + + // Test truncate table DDL + statement.execute( + String.format( + "TRUNCATE TABLE `%s`.`bills`;", inventoryDatabase.getDatabaseName())); + + expected.add( + new TruncateTableEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "bills"))); + + // Test drop table DDL + statement.execute( + String.format( + "DROP TABLE `%s`.`bills`, `%s`.`consumers`;", + inventoryDatabase.getDatabaseName(), + inventoryDatabase.getDatabaseName())); + + expected.add( + new DropTableEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "bills"))); + expected.add( + new DropTableEvent( + TableId.tableId(inventoryDatabase.getDatabaseName(), "consumers"))); + } + 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 +541,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()) + .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 +703,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 +757,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( 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 9253108ca12..936e0c19f92 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,12 +18,17 @@ 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.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; 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; @@ -76,24 +81,23 @@ 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 UnsupportedOperationException( - "PaimonDataSink doesn't support schema change event " + schemaChangeEvent); - } - } catch (Exception e) { - throw new RuntimeException(e); - } + + SchemaChangeEvent.handleWithoutResult(schemaChangeEvent) + .onAddColumn(this::applyAddColumn) + .onAlterColumnComment(this::applyAlterColumnComment) + .onAlterColumnType(this::applyAlterColumnType) + .onAlterTableComment(this::applyAlterTableComment) + .onCreateTable(this::applyCreateTable) + .onDropColumn(this::applyDropColumn) + .onDropTable(this::applyDropTable) + .onRenameColumn(this::applyRenameColumn) + .onRenameTable(this::applyRenameTable) + .onTruncateTable(this::applyTruncateTable) + .orElseThrow( + () -> + new UnsupportedOperationException( + "PaimonDataSink doesn't support schema change event " + + schemaChangeEvent)); } private void applyCreateTable(CreateTableEvent event) @@ -120,10 +124,7 @@ private void applyCreateTable(CreateTableEvent event) builder.partitionKeys(schema.partitionKeys()); } builder.options(tableOptions); - catalog.createTable( - new Identifier(event.tableId().getSchemaName(), event.tableId().getTableName()), - builder.build(), - true); + catalog.createTable(tableIdToPaimonIdentifier(event.tableId()), builder.build(), true); } private void applyAddColumn(AddColumnEvent event) @@ -142,10 +143,7 @@ private void applyAddColumn(AddColumnEvent event) .getLogicalType())); tableChangeList.add(tableChange); }); - catalog.alterTable( - new Identifier(event.tableId().getSchemaName(), event.tableId().getTableName()), - tableChangeList, - true); + catalog.alterTable(tableIdToPaimonIdentifier(event.tableId()), tableChangeList, true); } private void applyDropColumn(DropColumnEvent event) @@ -158,10 +156,7 @@ private void applyDropColumn(DropColumnEvent event) SchemaChange tableChange = SchemaChange.dropColumn(column); tableChangeList.add(tableChange); }); - catalog.alterTable( - new Identifier(event.tableId().getSchemaName(), event.tableId().getTableName()), - tableChangeList, - true); + catalog.alterTable(tableIdToPaimonIdentifier(event.tableId()), tableChangeList, true); } private void applyRenameColumn(RenameColumnEvent event) @@ -174,13 +169,24 @@ private void applyRenameColumn(RenameColumnEvent event) SchemaChange tableChange = SchemaChange.renameColumn(oldName, newName); tableChangeList.add(tableChange); }); - catalog.alterTable( - new Identifier(event.tableId().getSchemaName(), event.tableId().getTableName()), - tableChangeList, - true); + catalog.alterTable(tableIdToPaimonIdentifier(event.tableId()), tableChangeList, true); + } + + private void applyAlterColumnComment(AlterColumnCommentEvent event) + throws Catalog.ColumnAlreadyExistException, Catalog.TableNotExistException, + Catalog.ColumnNotExistException { + List tableChangeList = new ArrayList<>(); + event.getCommentMapping() + .forEach( + (oldName, newComment) -> { + SchemaChange tableChange = + SchemaChange.updateColumnComment(oldName, newComment); + tableChangeList.add(tableChange); + }); + catalog.alterTable(tableIdToPaimonIdentifier(event.tableId()), tableChangeList, true); } - private void applyAlterColumn(AlterColumnTypeEvent event) + private void applyAlterColumnType(AlterColumnTypeEvent event) throws Catalog.ColumnAlreadyExistException, Catalog.TableNotExistException, Catalog.ColumnNotExistException { List tableChangeList = new ArrayList<>(); @@ -195,9 +201,42 @@ private void applyAlterColumn(AlterColumnTypeEvent event) .getLogicalType())); tableChangeList.add(tableChange); }); + catalog.alterTable(tableIdToPaimonIdentifier(event.tableId()), tableChangeList, true); + } + + private void applyRenameTable(RenameTableEvent renameTableEvent) { + try { + catalog.renameTable( + tableIdToPaimonIdentifier(renameTableEvent.tableId()), + tableIdToPaimonIdentifier(renameTableEvent.newTableId()), + false); + } catch (Catalog.TableNotExistException | Catalog.TableAlreadyExistException e) { + throw new RuntimeException("Failed to apply rename table event " + renameTableEvent, e); + } + } + + private void applyTruncateTable(TruncateTableEvent truncateTableEvent) { + throw new UnsupportedOperationException("Truncate table is not supported currently"); + } + + private void applyDropTable(DropTableEvent dropTableEvent) { + try { + catalog.dropTable(tableIdToPaimonIdentifier(dropTableEvent.tableId()), false); + } catch (Catalog.TableNotExistException e) { + throw new RuntimeException("Failed to apply drop table event " + dropTableEvent, e); + } + } + + private void applyAlterTableComment(AlterTableCommentEvent alterTableCommentEvent) + throws Catalog.TableNotExistException, Catalog.ColumnAlreadyExistException, + Catalog.ColumnNotExistException { catalog.alterTable( - new Identifier(event.tableId().getSchemaName(), event.tableId().getTableName()), - tableChangeList, - true); + tableIdToPaimonIdentifier(alterTableCommentEvent.tableId()), + SchemaChange.updateComment(alterTableCommentEvent.getTableComment()), + false); + } + + private Identifier tableIdToPaimonIdentifier(TableId tableId) { + return new Identifier(tableId.getSchemaName(), tableId.getTableName()); } } 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 53b63f3b599..77e9a19edf6 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 @@ -66,7 +66,7 @@ public PaimonEvent serialize(Event event) { schemaMaps.put( schemaChangeEvent.tableId(), new TableSchemaInfo( - SchemaUtils.applySchemaChangeEvent( + SchemaUtils.applyColumnSchemaChangeEvent( schemaMaps.get(schemaChangeEvent.tableId()).getSchema(), schemaChangeEvent), zoneId)); 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 76b55aafc98..08dee530744 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 @@ -90,7 +90,7 @@ private void applySchemaChangeEvent(SchemaChangeEvent event) { if (tableInfo == null) { throw new RuntimeException("schema of " + tableId + " is not existed."); } - newSchema = SchemaUtils.applySchemaChangeEvent(tableInfo.schema, event); + newSchema = SchemaUtils.applyColumnSchemaChangeEvent(tableInfo.schema, event); } TableInfo tableInfo = new TableInfo(); tableInfo.schema = newSchema; 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 5ee93ff62e3..cf1deea0654 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 @@ -66,20 +66,17 @@ 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 UnsupportedOperationException( - "StarRocksDataSink doesn't support schema change event " + schemaChangeEvent); - } + SchemaChangeEvent.handleWithoutResult(schemaChangeEvent) + .onAddColumn(this::applyAddColumn) + .onAlterColumnType(this::applyAlterColumnType) + .onCreateTable(this::applyCreateTable) + .onDropColumn(this::applyDropColumn) + .onRenameColumn(this::applyRenameColumn) + .orElseThrow( + () -> + new UnsupportedOperationException( + "StarRocksDataSink doesn't support schema change event " + + schemaChangeEvent)); } private void applyCreateTable(CreateTableEvent createTableEvent) { @@ -229,7 +226,6 @@ private void applyDropColumn(DropColumnEvent dropColumnEvent) { dropColumnEvent, alterException); } - return; } if (alterException != null) { @@ -257,7 +253,7 @@ private void applyRenameColumn(RenameColumnEvent renameColumnEvent) { throw new UnsupportedOperationException("Rename column is not supported currently"); } - private void applyAlterColumn(AlterColumnTypeEvent alterColumnTypeEvent) { + 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-starrocks/src/test/java/org/apache/flink/cdc/connectors/starrocks/sink/EventRecordSerializationSchemaTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/test/java/org/apache/flink/cdc/connectors/starrocks/sink/EventRecordSerializationSchemaTest.java index d6622e3e09a..4b47d8e92e3 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/test/java/org/apache/flink/cdc/connectors/starrocks/sink/EventRecordSerializationSchemaTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/test/java/org/apache/flink/cdc/connectors/starrocks/sink/EventRecordSerializationSchemaTest.java @@ -213,7 +213,7 @@ public void testMixedSchemaAndDataChanges() throws Exception { new AddColumnEvent.ColumnWithPosition( Column.physicalColumn( "col6", new LocalZonedTimestampType())))); - Schema newSchema1 = SchemaUtils.applySchemaChangeEvent(schema1, addColumnEvent); + Schema newSchema1 = SchemaUtils.applyColumnSchemaChangeEvent(schema1, addColumnEvent); BinaryRecordDataGenerator newGenerator1 = new BinaryRecordDataGenerator( newSchema1.getColumnDataTypes().toArray(new DataType[0])); @@ -242,7 +242,7 @@ public void testMixedSchemaAndDataChanges() throws Exception { // 4. drop columns from table2, and insert data DropColumnEvent dropColumnEvent = new DropColumnEvent(table2, Arrays.asList("col2", "col3")); - Schema newSchema2 = SchemaUtils.applySchemaChangeEvent(schema2, dropColumnEvent); + Schema newSchema2 = SchemaUtils.applyColumnSchemaChangeEvent(schema2, dropColumnEvent); BinaryRecordDataGenerator newGenerator2 = new BinaryRecordDataGenerator( newSchema2.getColumnDataTypes().toArray(new DataType[0])); 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 36edab6d862..8c5abf63f45 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,13 +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.RenameTableEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; 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.sink.MetadataApplier; @@ -140,6 +144,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"); @@ -154,6 +165,22 @@ public static void applySchemaChangeEvent(SchemaChangeEvent event) { globalTables.put( tableId, new ValuesTable(tableId, ((CreateTableEvent) event).getSchema())); } + } else if (event instanceof RenameTableEvent) { + if (globalTables.containsKey(tableId)) { + TableId newTableId = + TableId.tableId( + tableId.getNamespace(), + tableId.getSchemaName(), + ((RenameTableEvent) event).newTableId().getTableName()); + globalTables.put(newTableId, globalTables.remove(tableId)); + } + } 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"); @@ -270,15 +297,19 @@ private void update(RecordData beforeRecordData, RecordData afterRecordData) { public void applySchemaChangeEvent(SchemaChangeEvent event) { synchronized (lock) { - 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); - } + SchemaChangeEvent.handleWithoutResult(event) + .onAddColumn(this::applyAddColumnEvent) + .onAlterColumnType(this::applyAlterColumnTypeEvent) + .onAlterColumnComment(this::applyAlterColumnCommentEvent) + .onDropColumn(this::applyDropColumnEvent) + .onRenameColumn(this::applyRenameColumnEvent) + .onTruncateTable(this::applyTruncateTableEvent) + .orElse( + e -> + LOG.warn( + "Received unsupported schema change event with type \"{}\"", + e.getClass().getCanonicalName())); + updatePrimaryKeyIndexes(); } } @@ -318,6 +349,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())) { @@ -400,5 +449,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 d6789452d8a..d4257eeb64d 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 @@ -24,9 +24,12 @@ 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.DropTableEvent; import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.event.RenameTableEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.common.sink.EventSinkProvider; @@ -136,12 +139,27 @@ public void write(Event event, Context context) { Schema schema = ((CreateTableEvent) event).getSchema(); schemaMaps.put(tableId, schema); fieldGetterMaps.put(tableId, SchemaUtils.createFieldGetters(schema)); + } else if (event instanceof RenameTableEvent) { + Schema schema = schemaMaps.remove(tableId); + List fieldGetters = fieldGetterMaps.remove(tableId); + TableId newTableId = + TableId.tableId( + tableId.getNamespace(), + tableId.getSchemaName(), + ((RenameTableEvent) event).newTableId().getTableName()); + schemaMaps.put(newTableId, schema); + fieldGetterMaps.put(newTableId, fieldGetters); + } else if (event instanceof DropTableEvent) { + schemaMaps.remove(tableId); + fieldGetterMaps.remove(tableId); + } else if (event instanceof TruncateTableEvent) { + // There's no need to change schema for truncate table events } else { if (!schemaMaps.containsKey(tableId)) { throw new RuntimeException("schema of " + tableId + " is not existed."); } Schema schema = - SchemaUtils.applySchemaChangeEvent( + SchemaUtils.applyColumnSchemaChangeEvent( schemaMaps.get(tableId), schemaChangeEvent); schemaMaps.put(tableId, schema); fieldGetterMaps.put(tableId, SchemaUtils.createFieldGetters(schema)); 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 f4876d5e04a..a6f15ac15cd 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 @@ -22,9 +22,12 @@ 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.DropTableEvent; import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.event.RenameTableEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.utils.SchemaUtils; import org.apache.flink.cdc.connectors.values.ValuesDatabase; @@ -57,19 +60,35 @@ public ValuesDataSinkFunction(boolean materializedInMemory, boolean print) { @Override public void invoke(Event event, Context context) throws Exception { + TableId tableId; if (event instanceof SchemaChangeEvent) { SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; - TableId tableId = schemaChangeEvent.tableId(); + tableId = schemaChangeEvent.tableId(); if (event instanceof CreateTableEvent) { Schema schema = ((CreateTableEvent) event).getSchema(); schemaMaps.put(tableId, schema); fieldGetterMaps.put(tableId, SchemaUtils.createFieldGetters(schema)); + } else if (event instanceof RenameTableEvent) { + Schema schema = schemaMaps.remove(tableId); + List fieldGetters = fieldGetterMaps.remove(tableId); + TableId newTableId = + TableId.tableId( + tableId.getNamespace(), + tableId.getSchemaName(), + ((RenameTableEvent) event).newTableId().getTableName()); + schemaMaps.put(newTableId, schema); + fieldGetterMaps.put(newTableId, fieldGetters); + } else if (event instanceof DropTableEvent) { + schemaMaps.remove(tableId); + fieldGetterMaps.remove(tableId); + } else if (event instanceof TruncateTableEvent) { + // There's no need to change schema for truncate table events } else { if (!schemaMaps.containsKey(tableId)) { throw new RuntimeException("schema of " + tableId + " is not existed."); } Schema schema = - SchemaUtils.applySchemaChangeEvent( + SchemaUtils.applyColumnSchemaChangeEvent( schemaMaps.get(tableId), schemaChangeEvent); schemaMaps.put(tableId, schema); fieldGetterMaps.put(tableId, SchemaUtils.createFieldGetters(schema)); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/test/java/org/apache/flink/cdc/connectors/values/ValuesDatabaseTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/test/java/org/apache/flink/cdc/connectors/values/ValuesDatabaseTest.java index f4ec43cf20f..716365067f5 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/test/java/org/apache/flink/cdc/connectors/values/ValuesDatabaseTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/test/java/org/apache/flink/cdc/connectors/values/ValuesDatabaseTest.java @@ -23,8 +23,11 @@ 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.RenameTableEvent; 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.sink.MetadataApplier; @@ -301,4 +304,23 @@ public void testSchemaChangeWithExistedData() { results.add("default.default.table1:col1=3;newCol3="); Assert.assertEquals(results, ValuesDatabase.getResults(table1)); } + + @Test + public void testApplyTableSchemaChangeEvent() { + TableId table1neo = TableId.parse("default.default.table1-neo"); + RenameTableEvent renameTableEvent = new RenameTableEvent(table1, table1neo); + metadataApplier.applySchemaChange(renameTableEvent); + Assert.assertEquals( + Collections.singletonList(table1neo), + metadataAccessor.listTables("default", "default")); + + TruncateTableEvent truncateTableEvent = new TruncateTableEvent(table1neo); + metadataApplier.applySchemaChange(truncateTableEvent); + Assert.assertEquals(Collections.emptyList(), ValuesDatabase.getResults(table1neo)); + + DropTableEvent dropTableEvent = new DropTableEvent(table1neo); + metadataApplier.applySchemaChange(dropTableEvent); + Assert.assertEquals( + Collections.emptyList(), metadataAccessor.listTables("default", "default")); + } } 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 33071758642..2b14c2902fd 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 @@ -114,16 +114,6 @@ public void testSyncWholeDatabase() throws Exception { submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); - waitUtilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.customers, before=[], after=[104, user_4, Shanghai, 123567891234], op=INSERT, meta=()}", - mysqlInventoryDatabase.getDatabaseName()), - 60000L); - waitUtilSpecificEvent( - String.format( - "DataChangeEvent{tableId=%s.products, before=[], after=[109, spare tire, 24 inch spare tire, 22.2, null, null, null], op=INSERT, meta=()}", - mysqlInventoryDatabase.getDatabaseName()), - 60000L); List expectedEvents = Arrays.asList( String.format( @@ -191,9 +181,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) { @@ -201,11 +197,8 @@ public void testSyncWholeDatabase() throws Exception { throw e; } - waitUtilSpecificEvent( - 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()), - 60000L); + Thread.sleep(10000); + System.out.println(taskManagerConsumer.toUtf8String()); expectedEvents = Arrays.asList( @@ -222,27 +215,41 @@ 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); } - private void validateResult(List expectedEvents) { - String stdout = taskManagerConsumer.toUtf8String(); + private void validateResult(List expectedEvents) throws Exception { for (String event : expectedEvents) { - if (!stdout.contains(event)) { - throw new RuntimeException( - "failed to get specific event: " + event + " from stdout: " + stdout); - } + waitUtilSpecificEvent(event, 6000L); } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/SchemaOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/SchemaOperator.java index d31fb026df8..f8da7e5fe78 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/SchemaOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/SchemaOperator.java @@ -25,6 +25,7 @@ import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.FlushEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventWithPreSchema; import org.apache.flink.cdc.common.event.TableId; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; @@ -70,6 +71,7 @@ import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; @@ -166,6 +168,26 @@ public void processElement(StreamRecord streamRecord) LOG.info( "Table {} received SchemaChangeEvent and start to be blocked.", tableId.toString()); + if (event instanceof SchemaChangeEventWithPreSchema) { + try { + SchemaChangeEventWithPreSchema preSchemaEvent = + (SchemaChangeEventWithPreSchema) event; + if (!preSchemaEvent.hasPreSchema()) { + Schema oldSchema = cachedSchemas.get(tableId); + preSchemaEvent.fillPreSchema(oldSchema); + String originalEventRepresentation = preSchemaEvent.toString(); + if (!preSchemaEvent.trimRedundantChanges()) { + // Discard events with no effective changing records + LOG.info( + "Redundant change stream request {} from upstream has been discarded.", + originalEventRepresentation); + return; + } + } + } catch (ExecutionException e) { + // failed to fetch cached schema, just keep running + } + } handleSchemaChangeEvent(tableId, (SchemaChangeEvent) event); // Update caches cachedSchemas.put(tableId, getLatestSchema(tableId)); 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 026dda626f3..6689d6c0bd3 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,9 +19,9 @@ import org.apache.flink.api.java.tuple.Tuple2; 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.DropColumnEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.event.TableId; @@ -46,6 +46,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -89,25 +90,33 @@ public List applySchemaChange(SchemaChangeEvent schemaChangeE // Many-to-1 mapping (merging tables) Schema derivedTableSchema = schemaManager.getLatestSchema(derivedTable).get(); - if (schemaChangeEvent instanceof CreateTableEvent) { - return handleCreateTableEvent( - (CreateTableEvent) schemaChangeEvent, derivedTableSchema, derivedTable); - } else if (schemaChangeEvent instanceof AddColumnEvent) { - return handleAddColumnEvent( - (AddColumnEvent) schemaChangeEvent, derivedTableSchema, derivedTable); - } else if (schemaChangeEvent instanceof AlterColumnTypeEvent) { - return handleAlterColumnTypeEvent( - (AlterColumnTypeEvent) schemaChangeEvent, derivedTableSchema, derivedTable); - } else if (schemaChangeEvent instanceof DropColumnEvent) { - return Collections.emptyList(); - } else if (schemaChangeEvent instanceof RenameColumnEvent) { - return handleRenameColumnEvent( - (RenameColumnEvent) schemaChangeEvent, derivedTableSchema, derivedTable); - } else { - throw new IllegalStateException( - String.format( - "Unrecognized SchemaChangeEvent type: %s", schemaChangeEvent)); - } + + return SchemaChangeEvent.>handle(schemaChangeEvent) + .onAddColumn( + event -> handleAddColumnEvent(event, derivedTableSchema, derivedTable)) + .onAlterColumnComment( + event -> + handleAlterColumnCommentEvent( + event, derivedTableSchema, derivedTable)) + .onAlterColumnType( + event -> + handleAlterColumnTypeEvent( + event, derivedTableSchema, derivedTable)) + .onCreateTable( + event -> + handleCreateTableEvent(event, derivedTableSchema, derivedTable)) + .onDropColumn(event -> Collections.emptyList()) + .onRenameColumn( + event -> + handleRenameColumnEvent( + event, derivedTableSchema, derivedTable)) + .get() + .orElseThrow( + () -> + new IllegalArgumentException( + String.format( + "Unrecognized SchemaChangeEvent type: %s", + schemaChangeEvent))); } // No routes are matched @@ -211,7 +220,8 @@ private List handleAlterColumnTypeEvent( List schemaChangeEvents = new ArrayList<>(); if (!typeDifference.isEmpty()) { AlterColumnTypeEvent derivedSchemaChangeEvent = - new AlterColumnTypeEvent(derivedTable, typeDifference); + new AlterColumnTypeEvent( + derivedTable, typeDifference, alterColumnTypeEvent.getOldTypeMapping()); schemaChangeEvents.add(derivedSchemaChangeEvent); } schemaChangeEvents.forEach(schemaManager::applySchemaChange); @@ -221,7 +231,11 @@ private List handleAlterColumnTypeEvent( private List handleAddColumnEvent( AddColumnEvent addColumnEvent, Schema derivedTableSchema, TableId derivedTable) { List newColumns = new ArrayList<>(); + Map oldTypeMapping = new HashMap<>(); Map newTypeMapping = new HashMap<>(); + + Map oldCommentMapping = new HashMap<>(); + Map newCommentMapping = new HashMap<>(); // Check if new column already existed in the derived table for (AddColumnEvent.ColumnWithPosition addedColumn : addColumnEvent.getAddedColumns()) { Optional optionalColumnInDerivedTable = @@ -241,8 +255,22 @@ private List handleAddColumnEvent( addedColumn.getAddColumn().getType()); if (!widerType.equals(existedColumnInDerivedTable.getType())) { newTypeMapping.put(existedColumnInDerivedTable.getName(), widerType); + oldTypeMapping.put( + existedColumnInDerivedTable.getName(), + existedColumnInDerivedTable.getType()); } } + + if (!Objects.equals( + existedColumnInDerivedTable.getComment(), + addedColumn.getAddColumn().getComment())) { + newCommentMapping.put( + existedColumnInDerivedTable.getName(), + addedColumn.getAddColumn().getComment()); + oldCommentMapping.put( + existedColumnInDerivedTable.getName(), + existedColumnInDerivedTable.getComment()); + } } } @@ -251,7 +279,13 @@ private List handleAddColumnEvent( schemaChangeEvents.add(new AddColumnEvent(derivedTable, newColumns)); } if (!newTypeMapping.isEmpty()) { - schemaChangeEvents.add(new AlterColumnTypeEvent(derivedTable, newTypeMapping)); + schemaChangeEvents.add( + new AlterColumnTypeEvent(derivedTable, newTypeMapping, oldTypeMapping)); + } + if (!newCommentMapping.isEmpty()) { + schemaChangeEvents.add( + new AlterColumnCommentEvent( + derivedTable, newCommentMapping, oldCommentMapping)); } schemaChangeEvents.forEach(schemaManager::applySchemaChange); return schemaChangeEvents; @@ -261,6 +295,10 @@ private List handleCreateTableEvent( CreateTableEvent createTableEvent, Schema derivedTableSchema, TableId derivedTable) { List newColumns = new ArrayList<>(); Map newTypeMapping = new HashMap<>(); + Map oldTypeMapping = new HashMap<>(); + + Map oldCommentMapping = new HashMap<>(); + Map newCommentMapping = new HashMap<>(); // Check if there is any columns that doesn't exist in the derived table // and perform add-column for non-existed columns. for (Column column : createTableEvent.getSchema().getColumns()) { @@ -277,8 +315,20 @@ private List handleCreateTableEvent( getWiderType(existedColumnInDerivedTable.getType(), column.getType()); if (!widerType.equals(existedColumnInDerivedTable.getType())) { newTypeMapping.put(existedColumnInDerivedTable.getName(), widerType); + oldTypeMapping.put( + existedColumnInDerivedTable.getName(), + existedColumnInDerivedTable.getType()); } } + + if (!Objects.equals( + existedColumnInDerivedTable.getComment(), column.getComment())) { + newCommentMapping.put( + existedColumnInDerivedTable.getName(), column.getComment()); + oldCommentMapping.put( + existedColumnInDerivedTable.getName(), + existedColumnInDerivedTable.getComment()); + } } } @@ -287,7 +337,43 @@ private List handleCreateTableEvent( schemaChangeEvents.add(new AddColumnEvent(derivedTable, newColumns)); } if (!newTypeMapping.isEmpty()) { - schemaChangeEvents.add(new AlterColumnTypeEvent(derivedTable, newTypeMapping)); + schemaChangeEvents.add( + new AlterColumnTypeEvent(derivedTable, newTypeMapping, oldTypeMapping)); + } + if (!newCommentMapping.isEmpty()) { + schemaChangeEvents.add( + new AlterColumnCommentEvent( + derivedTable, newCommentMapping, oldCommentMapping)); + } + schemaChangeEvents.forEach(schemaManager::applySchemaChange); + return schemaChangeEvents; + } + + private List handleAlterColumnCommentEvent( + AlterColumnCommentEvent alterColumnCommentEvent, + Schema derivedTableSchema, + TableId derivedTable) { + Map newCommentMap = new HashMap<>(); + Map oldCommentMap = new HashMap<>(); + alterColumnCommentEvent + .getCommentMapping() + .forEach( + (columnName, comment) -> { + Column existedColumnInDerivedTable = + derivedTableSchema.getColumn(columnName).get(); + if (!Objects.equals( + existedColumnInDerivedTable.getComment(), comment)) { + newCommentMap.put(existedColumnInDerivedTable.getName(), comment); + oldCommentMap.put( + existedColumnInDerivedTable.getName(), + existedColumnInDerivedTable.getComment()); + } + }); + List schemaChangeEvents = new ArrayList<>(); + if (!newCommentMap.isEmpty()) { + AlterColumnCommentEvent derivedSchemaChangeEvent = + new AlterColumnCommentEvent(derivedTable, newCommentMap, oldCommentMap); + schemaChangeEvents.add(derivedSchemaChangeEvent); } schemaChangeEvents.forEach(schemaManager::applySchemaChange); return schemaChangeEvents; 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 5b71e1c4e5c..c85319e9b76 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 @@ -48,8 +48,8 @@ import static org.apache.flink.cdc.common.utils.Preconditions.checkArgument; /** - * Schema manager handles handles schema changes for tables, and manages historical schema versions - * of tables. + * Schema manager handles schema changes for tables, and manages historical schema versions of + * tables. */ @Internal public class SchemaManager { @@ -111,7 +111,8 @@ public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) { LOG.info("Handling schema change event: {}", schemaChangeEvent); registerNewSchema( schemaChangeEvent.tableId(), - SchemaUtils.applySchemaChangeEvent(optionalSchema.get(), schemaChangeEvent)); + SchemaUtils.applyColumnSchemaChangeEvent( + optionalSchema.get(), schemaChangeEvent)); } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformDataOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformDataOperator.java index 1ccbf87fd72..6a11b294308 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformDataOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformDataOperator.java @@ -199,7 +199,7 @@ private SchemaChangeEvent cacheSchema(SchemaChangeEvent event) throws Exception newSchema = ((CreateTableEvent) event).getSchema(); } else { newSchema = - SchemaUtils.applySchemaChangeEvent( + SchemaUtils.applyColumnSchemaChangeEvent( getTableInfoFromSchemaEvolutionClient(tableId).getSchema(), event); } transformSchema(tableId, newSchema); diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformSchemaOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformSchemaOperator.java index 230fc1a6ffd..37f7a65b35d 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformSchemaOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TransformSchemaOperator.java @@ -188,9 +188,11 @@ private SchemaChangeEvent cacheChangeSchema(SchemaChangeEvent event) { TableId tableId = event.tableId(); TableChangeInfo tableChangeInfo = tableChangeInfoMap.get(tableId); Schema originalSchema = - SchemaUtils.applySchemaChangeEvent(tableChangeInfo.getOriginalSchema(), event); + SchemaUtils.applyColumnSchemaChangeEvent( + tableChangeInfo.getOriginalSchema(), event); Schema newSchema = - SchemaUtils.applySchemaChangeEvent(tableChangeInfo.getTransformedSchema(), event); + SchemaUtils.applyColumnSchemaChangeEvent( + tableChangeInfo.getTransformedSchema(), event); tableChangeInfoMap.put(tableId, TableChangeInfo.of(tableId, originalSchema, newSchema)); return 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 00000000000..4c669b85dee --- /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 ab228e4d6b2..978b9b87ecd 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 00000000000..c066896758b --- /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 00000000000..b5f8a0d3395 --- /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/RenameTableEventSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/RenameTableEventSerializer.java new file mode 100644 index 00000000000..a544c984a76 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/RenameTableEventSerializer.java @@ -0,0 +1,110 @@ +/* + * 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.RenameTableEvent; +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 RenameTableEvent}. */ +public class RenameTableEventSerializer extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + /** Sharable instance of the TableIdSerializer. */ + public static final RenameTableEventSerializer INSTANCE = new RenameTableEventSerializer(); + + private final TableIdSerializer tableIdSerializer = TableIdSerializer.INSTANCE; + private final MapSerializer typeMapSerializer = + new MapSerializer<>(StringSerializer.INSTANCE, new DataTypeSerializer()); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public RenameTableEvent createInstance() { + return new RenameTableEvent(TableId.tableId("unknown"), TableId.tableId("unknown")); + } + + @Override + public RenameTableEvent copy(RenameTableEvent from) { + return new RenameTableEvent(from.tableId(), from.newTableId()); + } + + @Override + public RenameTableEvent copy(RenameTableEvent from, RenameTableEvent reuse) { + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(RenameTableEvent record, DataOutputView target) throws IOException { + tableIdSerializer.serialize(record.tableId(), target); + tableIdSerializer.serialize(record.newTableId(), target); + } + + @Override + public RenameTableEvent deserialize(DataInputView source) throws IOException { + return new RenameTableEvent( + tableIdSerializer.deserialize(source), tableIdSerializer.deserialize(source)); + } + + @Override + public RenameTableEvent deserialize(RenameTableEvent 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 RenameTableEventSerializer.RenameTableEventSerializerSnapshot(); + } + + /** Serializer configuration snapshot for compatibility and format evolution. */ + @SuppressWarnings("WeakerAccess") + public static final class RenameTableEventSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + + public RenameTableEventSerializerSnapshot() { + 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 b2a9f0643c1..e9441919b41 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 @@ -20,11 +20,6 @@ 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.AddColumnEvent; -import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; -import org.apache.flink.cdc.common.event.CreateTableEvent; -import org.apache.flink.cdc.common.event.DropColumnEvent; -import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.event.TableId; import org.apache.flink.cdc.runtime.serializer.EnumSerializer; @@ -57,19 +52,20 @@ public SchemaChangeEvent createInstance() { @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 SchemaChangeEvent.handle(from) + .onAddColumn(AddColumnEventSerializer.INSTANCE::copy) + .onAlterColumnType(AlterColumnTypeEventSerializer.INSTANCE::copy) + .onAlterColumnComment(AlterColumnCommentEventSerializer.INSTANCE::copy) + .onAlterTableComment(AlterTableCommentEventSerializer.INSTANCE::copy) + .onCreateTable(CreateTableEventSerializer.INSTANCE::copy) + .onDropTable(DropTableEventSerializer.INSTANCE::copy) + .onDropColumn(DropColumnEventSerializer.INSTANCE::copy) + .onRenameColumn(RenameColumnEventSerializer.INSTANCE::copy) + .onRenameTable(RenameTableEventSerializer.INSTANCE::copy) + .onTruncateTable(TruncateTableEventSerializer.INSTANCE::copy) + .get() + .orElseThrow( + () -> new IllegalArgumentException("Unknown schema change event: " + from)); } @Override @@ -83,26 +79,66 @@ public int getLength() { } @Override - public void serialize(SchemaChangeEvent record, DataOutputView target) throws IOException { - if (record instanceof AlterColumnTypeEvent) { - enumSerializer.serialize(SchemaChangeEventClass.ALTER_COLUMN_TYPE, target); - AlterColumnTypeEventSerializer.INSTANCE.serialize( - (AlterColumnTypeEvent) record, target); - } else if (record instanceof CreateTableEvent) { - enumSerializer.serialize(SchemaChangeEventClass.CREATE_TABLE, target); - CreateTableEventSerializer.INSTANCE.serialize((CreateTableEvent) record, target); - } else if (record instanceof RenameColumnEvent) { - enumSerializer.serialize(SchemaChangeEventClass.RENAME_COLUMN, target); - RenameColumnEventSerializer.INSTANCE.serialize((RenameColumnEvent) record, target); - } else if (record instanceof AddColumnEvent) { - enumSerializer.serialize(SchemaChangeEventClass.ADD_COLUMN, target); - AddColumnEventSerializer.INSTANCE.serialize((AddColumnEvent) record, target); - } else if (record instanceof DropColumnEvent) { - enumSerializer.serialize(SchemaChangeEventClass.DROP_COLUMN, target); - DropColumnEventSerializer.INSTANCE.serialize((DropColumnEvent) record, target); - } else { - throw new IllegalArgumentException("Unknown schema change event: " + record); - } + public void serialize(SchemaChangeEvent event, DataOutputView target) throws IOException { + + SchemaChangeEvent.handleWithoutResult(event) + .onAddColumn( + record -> { + enumSerializer.serialize(SchemaChangeEventClass.ADD_COLUMN, target); + AddColumnEventSerializer.INSTANCE.serialize(record, target); + }) + .onAlterColumnComment( + record -> { + enumSerializer.serialize( + SchemaChangeEventClass.ALTER_COLUMN_COMMENT, target); + AlterColumnCommentEventSerializer.INSTANCE.serialize(record, target); + }) + .onAlterColumnType( + record -> { + enumSerializer.serialize( + SchemaChangeEventClass.ALTER_COLUMN_TYPE, target); + AlterColumnTypeEventSerializer.INSTANCE.serialize(record, target); + }) + .onAlterTableComment( + record -> { + enumSerializer.serialize( + SchemaChangeEventClass.ALTER_TABLE_COMMENT, target); + AlterTableCommentEventSerializer.INSTANCE.serialize(record, target); + }) + .onCreateTable( + record -> { + enumSerializer.serialize(SchemaChangeEventClass.CREATE_TABLE, target); + CreateTableEventSerializer.INSTANCE.serialize(record, target); + }) + .onDropColumn( + record -> { + enumSerializer.serialize(SchemaChangeEventClass.DROP_COLUMN, target); + DropColumnEventSerializer.INSTANCE.serialize(record, target); + }) + .onDropTable( + record -> { + enumSerializer.serialize(SchemaChangeEventClass.DROP_TABLE, target); + DropTableEventSerializer.INSTANCE.serialize(record, target); + }) + .onRenameColumn( + record -> { + enumSerializer.serialize(SchemaChangeEventClass.RENAME_COLUMN, target); + RenameColumnEventSerializer.INSTANCE.serialize(record, target); + }) + .onRenameTable( + record -> { + enumSerializer.serialize(SchemaChangeEventClass.RENAME_TABLE, target); + RenameTableEventSerializer.INSTANCE.serialize(record, target); + }) + .onTruncateTable( + record -> { + enumSerializer.serialize(SchemaChangeEventClass.TRUNCATE_TABLE, target); + TruncateTableEventSerializer.INSTANCE.serialize(record, target); + }) + .orElseThrow( + () -> + new IllegalArgumentException( + "Unknown schema change event: " + event)); } @Override @@ -119,6 +155,14 @@ 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 RENAME_TABLE: + return RenameTableEventSerializer.INSTANCE.deserialize(source); + case TRUNCATE_TABLE: + return TruncateTableEventSerializer.INSTANCE.deserialize(source); default: throw new IllegalArgumentException( "Unknown schema change event class: " + schemaChangeEventClass); @@ -152,10 +196,15 @@ public SchemaChangeEventSerializerSnapshot() { } enum SchemaChangeEventClass { - ALTER_COLUMN_TYPE, - RENAME_COLUMN, ADD_COLUMN, + ALTER_COLUMN_COMMENT, + ALTER_COLUMN_TYPE, + ALTER_TABLE_COMMENT, + CREATE_TABLE, DROP_COLUMN, - CREATE_TABLE; + DROP_TABLE, + RENAME_COLUMN, + RENAME_TABLE, + TRUNCATE_TABLE } } 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 00000000000..26eb6e744bd --- /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/coordinator/SchemaDerivationTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaDerivationTest.java index 19fd5a4a131..c946e675306 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 @@ -254,7 +254,9 @@ void testMergingTableWithDifferentSchemas() { new PhysicalColumn( "gender", DataTypes.STRING(), null)))), new AlterColumnTypeEvent( - MERGED_TABLE, ImmutableMap.of("age", DataTypes.BIGINT()))); + MERGED_TABLE, + ImmutableMap.of("age", DataTypes.BIGINT()), + ImmutableMap.of("age", DataTypes.INT()))); // Add column for table 1 AddColumnEvent.ColumnWithPosition newCol1 = @@ -365,8 +367,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 00000000000..346a6b580c2 --- /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 baddd0bf209..1c45d717712 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 00000000000..897b17a72ad --- /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 00000000000..bbeb92c4133 --- /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/RenameTableEventSerializerTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/RenameTableEventSerializerTest.java new file mode 100644 index 00000000000..2b16f3b443d --- /dev/null +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/serializer/event/RenameTableEventSerializerTest.java @@ -0,0 +1,53 @@ +/* + * 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.RenameTableEvent; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.runtime.serializer.SerializerTestBase; + +/** A test for the {@link RenameTableEventSerializer}. */ +public class RenameTableEventSerializerTest extends SerializerTestBase { + @Override + protected TypeSerializer createSerializer() { + return RenameTableEventSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return RenameTableEvent.class; + } + + @Override + protected RenameTableEvent[] getTestData() { + return new RenameTableEvent[] { + new RenameTableEvent(TableId.tableId("table"), TableId.tableId("newTable")), + new RenameTableEvent( + TableId.tableId("schema", "table"), TableId.tableId("schema", "newTable")), + new RenameTableEvent( + TableId.tableId("namespace", "schema", "table"), + TableId.tableId("namespace", "schema", "newTable")) + }; + } +} 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 00000000000..ba2f439b19f --- /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")) + }; + } +}