diff --git a/flink-cdc-cli/src/main/java/org/apache/flink/cdc/cli/parser/YamlPipelineDefinitionParser.java b/flink-cdc-cli/src/main/java/org/apache/flink/cdc/cli/parser/YamlPipelineDefinitionParser.java index 23b2c63ff7b..f1b00d32ab4 100644 --- a/flink-cdc-cli/src/main/java/org/apache/flink/cdc/cli/parser/YamlPipelineDefinitionParser.java +++ b/flink-cdc-cli/src/main/java/org/apache/flink/cdc/cli/parser/YamlPipelineDefinitionParser.java @@ -18,6 +18,7 @@ package org.apache.flink.cdc.cli.parser; import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; import org.apache.flink.cdc.common.utils.StringUtils; import org.apache.flink.cdc.composer.definition.PipelineDef; import org.apache.flink.cdc.composer.definition.RouteDef; @@ -28,6 +29,7 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.type.TypeReference; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import java.nio.file.Path; @@ -35,7 +37,9 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; +import static org.apache.flink.cdc.common.utils.ChangeEventUtils.resolveSchemaEvolutionOptions; import static org.apache.flink.cdc.common.utils.Preconditions.checkNotNull; /** Parser for converting YAML formatted pipeline definition to {@link PipelineDef}. */ @@ -51,6 +55,8 @@ public class YamlPipelineDefinitionParser implements PipelineDefinitionParser { // Source / sink keys private static final String TYPE_KEY = "type"; private static final String NAME_KEY = "name"; + private static final String INCLUDE_SCHEMA_EVOLUTION_TYPES = "include.schema.changes"; + private static final String EXCLUDE_SCHEMA_EVOLUTION_TYPES = "exclude.schema.changes"; // Route keys private static final String ROUTE_SOURCE_TABLE_KEY = "source-table"; @@ -135,6 +141,23 @@ private SourceDef toSourceDef(JsonNode sourceNode) { } private SinkDef toSinkDef(JsonNode sinkNode) { + List includedSETypes = new ArrayList<>(); + List excludedSETypes = new ArrayList<>(); + + Optional.ofNullable(sinkNode.get(INCLUDE_SCHEMA_EVOLUTION_TYPES)) + .ifPresent(e -> e.forEach(tag -> includedSETypes.add(tag.asText()))); + + Optional.ofNullable(sinkNode.get(EXCLUDE_SCHEMA_EVOLUTION_TYPES)) + .ifPresent(e -> e.forEach(tag -> excludedSETypes.add(tag.asText()))); + + Set declaredSETypes = + resolveSchemaEvolutionOptions(includedSETypes, excludedSETypes); + + if (sinkNode instanceof ObjectNode) { + ((ObjectNode) sinkNode).remove(INCLUDE_SCHEMA_EVOLUTION_TYPES); + ((ObjectNode) sinkNode).remove(EXCLUDE_SCHEMA_EVOLUTION_TYPES); + } + Map sinkMap = mapper.convertValue(sinkNode, new TypeReference>() {}); @@ -148,7 +171,7 @@ private SinkDef toSinkDef(JsonNode sinkNode) { // "name" field is optional String name = sinkMap.remove(NAME_KEY); - return new SinkDef(type, name, Configuration.fromMap(sinkMap)); + return new SinkDef(type, name, Configuration.fromMap(sinkMap), declaredSETypes); } private RouteDef toRouteDef(JsonNode routeNode) { diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AddColumnEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AddColumnEvent.java index d41dfcb3a84..a54cfe14f8d 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AddColumnEvent.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AddColumnEvent.java @@ -31,7 +31,7 @@ * lenient column type changes. */ @PublicEvolving -public final class AddColumnEvent implements SchemaChangeEvent { +public final class AddColumnEvent implements ColumnSchemaChangeEvent { private static final long serialVersionUID = 1L; @@ -173,4 +173,9 @@ public String toString() { public TableId tableId() { return tableId; } + + @Override + public SchemaChangeEventType getType() { + return SchemaChangeEventType.ADD_COLUMN; + } } 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..2ee3eaef9ff --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterColumnCommentEvent.java @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.common.event; + +import org.apache.flink.cdc.common.schema.Schema; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * A {@link SchemaChangeEvent} that represents an {@code ALTER COLUMN} DDL, which may contain the + * comment changes. + */ +public class AlterColumnCommentEvent + implements ColumnSchemaChangeEvent, SchemaChangeEventWithPreSchema { + + private static final long serialVersionUID = 1L; + + private final TableId tableId; + + /** key => column name, value => column type after changing. */ + private final Map commentMapping; + + private final Map oldCommentMapping; + + public AlterColumnCommentEvent(TableId tableId, Map commentMapping) { + this.tableId = tableId; + this.commentMapping = commentMapping; + this.oldCommentMapping = new HashMap<>(); + } + + public AlterColumnCommentEvent( + TableId tableId, + Map commentMapping, + Map oldCommentMapping) { + this.tableId = tableId; + this.commentMapping = commentMapping; + this.oldCommentMapping = oldCommentMapping; + } + + /** Returns the type mapping. */ + public Map getCommentMapping() { + return commentMapping; + } + + public Map getOldCommentMapping() { + return oldCommentMapping; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof AlterColumnCommentEvent)) { + return false; + } + AlterColumnCommentEvent that = (AlterColumnCommentEvent) o; + return Objects.equals(tableId, that.tableId) + && Objects.equals(commentMapping, that.commentMapping) + && Objects.equals(oldCommentMapping, that.oldCommentMapping); + } + + @Override + public int hashCode() { + return Objects.hash(tableId, commentMapping, oldCommentMapping); + } + + @Override + public String toString() { + if (hasPreSchema()) { + return "AlterColumnCommentEvent{" + + "tableId=" + + tableId + + ", commentMapping=" + + commentMapping + + ", oldCommentMapping=" + + oldCommentMapping + + '}'; + } else { + return "AlterColumnCommentEvent{" + + "tableId=" + + tableId + + ", commentMapping=" + + commentMapping + + '}'; + } + } + + @Override + public TableId tableId() { + return tableId; + } + + @Override + public boolean hasPreSchema() { + return !oldCommentMapping.isEmpty(); + } + + @Override + public void fillPreSchema(Schema oldTypeSchema) { + oldCommentMapping.clear(); + oldTypeSchema.getColumns().stream() + .filter(e -> commentMapping.containsKey(e.getName())) + .forEach(e -> oldCommentMapping.put(e.getName(), e.getComment())); + } + + @Override + public boolean trimRedundantChanges() { + if (hasPreSchema()) { + Set redundantlyChangedColumns = + commentMapping.keySet().stream() + .filter( + e -> + Objects.equals( + commentMapping.get(e), + oldCommentMapping.get(e))) + .collect(Collectors.toSet()); + + // Remove redundant alter column type records that doesn't really change the type + commentMapping.keySet().removeAll(redundantlyChangedColumns); + oldCommentMapping.keySet().removeAll(redundantlyChangedColumns); + } + return !commentMapping.isEmpty(); + } + + @Override + public SchemaChangeEventType getType() { + return SchemaChangeEventType.ALTER_COLUMN_COMMENT; + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterColumnTypeEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterColumnTypeEvent.java index 5eb5da0d121..1ace6394222 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterColumnTypeEvent.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterColumnTypeEvent.java @@ -17,16 +17,22 @@ package org.apache.flink.cdc.common.event; +import org.apache.flink.cdc.common.schema.Column; +import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.types.DataType; +import java.util.HashMap; import java.util.Map; import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; /** * A {@link SchemaChangeEvent} that represents an {@code ALTER COLUMN} DDL, which may contain the * lenient column type changes. */ -public class AlterColumnTypeEvent implements SchemaChangeEvent { +public class AlterColumnTypeEvent + implements ColumnSchemaChangeEvent, SchemaChangeEventWithPreSchema { private static final long serialVersionUID = 1L; @@ -35,9 +41,21 @@ public class AlterColumnTypeEvent implements SchemaChangeEvent { /** key => column name, value => column type after changing. */ private final Map typeMapping; + private final Map oldTypeMapping; + public AlterColumnTypeEvent(TableId tableId, Map typeMapping) { this.tableId = tableId; this.typeMapping = typeMapping; + this.oldTypeMapping = new HashMap<>(); + } + + public AlterColumnTypeEvent( + TableId tableId, + Map typeMapping, + Map oldTypeMapping) { + this.tableId = tableId; + this.typeMapping = typeMapping; + this.oldTypeMapping = oldTypeMapping; } /** Returns the type mapping. */ @@ -55,26 +73,76 @@ 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(); + } + + @Override + public SchemaChangeEventType getType() { + return SchemaChangeEventType.ALTER_COLUMN_TYPE; + } } diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterTableCommentEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterTableCommentEvent.java new file mode 100644 index 00000000000..be21f08fde7 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterTableCommentEvent.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.common.event; + +import java.util.Objects; + +/** A {@link SchemaChangeEvent} that represents an {@code ALTER TABLE COMMENT = ...} DDL. */ +public class AlterTableCommentEvent implements TableSchemaChangeEvent { + private static final long serialVersionUID = 1L; + + private final TableId tableId; + + /** key => column name, value => column type after changing. */ + private final String tableComment; + + public AlterTableCommentEvent(TableId tableId, String tableComment) { + this.tableId = tableId; + this.tableComment = tableComment; + } + + public String getTableComment() { + return tableComment; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof AlterTableCommentEvent)) { + return false; + } + AlterTableCommentEvent that = (AlterTableCommentEvent) o; + return Objects.equals(tableId, that.tableId) + && Objects.equals(tableComment, that.tableComment); + } + + @Override + public int hashCode() { + return Objects.hash(tableId, tableComment); + } + + @Override + public String toString() { + return "AlterTableCommentEvent{" + + "tableId=" + + tableId + + ", tableComment=" + + tableComment + + '}'; + } + + @Override + public TableId tableId() { + return tableId; + } + + @Override + public SchemaChangeEventType getType() { + return SchemaChangeEventType.ALTER_TABLE_COMMENT; + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEvent.java new file mode 100644 index 00000000000..996d4327bbc --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEvent.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.common.event; + +/** Schema change events on column-level. */ +public interface ColumnSchemaChangeEvent extends SchemaChangeEvent { + default void visit(ColumnSchemaChangeEventVisitorVoid visitor) { + try { + if (this instanceof AddColumnEvent) { + visitor.visit((AddColumnEvent) this); + } else if (this instanceof AlterColumnCommentEvent) { + visitor.visit((AlterColumnCommentEvent) this); + } else if (this instanceof AlterColumnTypeEvent) { + visitor.visit((AlterColumnTypeEvent) this); + } else if (this instanceof DropColumnEvent) { + visitor.visit((DropColumnEvent) this); + } else if (this instanceof RenameColumnEvent) { + visitor.visit((RenameColumnEvent) this); + } else { + throw new IllegalArgumentException("Unknown schema change event type " + getType()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + default T visit(ColumnSchemaChangeEventVisitor visitor) { + try { + if (this instanceof AddColumnEvent) { + return visitor.visit((AddColumnEvent) this); + } else if (this instanceof AlterColumnCommentEvent) { + return visitor.visit((AlterColumnCommentEvent) this); + } else if (this instanceof AlterColumnTypeEvent) { + return visitor.visit((AlterColumnTypeEvent) this); + } else if (this instanceof DropColumnEvent) { + return visitor.visit((DropColumnEvent) this); + } else if (this instanceof RenameColumnEvent) { + return visitor.visit((RenameColumnEvent) this); + } else { + throw new IllegalArgumentException("Unknown schema change event type " + getType()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEventVisitor.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEventVisitor.java new file mode 100644 index 00000000000..fb9393e9aa8 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEventVisitor.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.common.event; + +/** Visitor class for all {@link ColumnSchemaChangeEvent}s. */ +public interface ColumnSchemaChangeEventVisitor { + T visit(AddColumnEvent event) throws Exception; + + T visit(AlterColumnCommentEvent event) throws Exception; + + T visit(AlterColumnTypeEvent event) throws Exception; + + T visit(DropColumnEvent event) throws Exception; + + T visit(RenameColumnEvent event) throws Exception; +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEventVisitorVoid.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEventVisitorVoid.java new file mode 100644 index 00000000000..0946cee9af2 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/ColumnSchemaChangeEventVisitorVoid.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.common.event; + +/** Visitor class for all {@link ColumnSchemaChangeEvent}s and returns nothing. */ +public interface ColumnSchemaChangeEventVisitorVoid { + void visit(AddColumnEvent event) throws Exception; + + void visit(AlterColumnCommentEvent event) throws Exception; + + void visit(AlterColumnTypeEvent event) throws Exception; + + void visit(DropColumnEvent event) throws Exception; + + void visit(RenameColumnEvent event) throws Exception; +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/CreateTableEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/CreateTableEvent.java index 712ad8c58a7..ebe704a5ef1 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/CreateTableEvent.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/CreateTableEvent.java @@ -28,7 +28,7 @@ * {@link DataSource} before all {@link DataChangeEvent} with the same tableId */ @PublicEvolving -public class CreateTableEvent implements SchemaChangeEvent { +public class CreateTableEvent implements TableSchemaChangeEvent { private static final long serialVersionUID = 1L; @@ -72,4 +72,9 @@ public String toString() { public TableId tableId() { return tableId; } + + @Override + public SchemaChangeEventType getType() { + return SchemaChangeEventType.CREATE_TABLE; + } } diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/DropColumnEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/DropColumnEvent.java index 301bf3d7569..264d731d61b 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/DropColumnEvent.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/DropColumnEvent.java @@ -27,7 +27,7 @@ * lenient column type changes. */ @PublicEvolving -public class DropColumnEvent implements SchemaChangeEvent { +public class DropColumnEvent implements ColumnSchemaChangeEvent { private static final long serialVersionUID = 1L; @@ -76,4 +76,9 @@ public String toString() { public TableId tableId() { return tableId; } + + @Override + public SchemaChangeEventType getType() { + return SchemaChangeEventType.DROP_COLUMN; + } } 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..198d49d67e8 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/DropTableEvent.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.common.event; + +import org.apache.flink.cdc.common.source.DataSource; + +import java.util.Objects; + +/** + * A {@link SchemaChangeEvent} that represents an {@code DROP TABLE} DDL. this will be sent by + * {@link DataSource} before all {@link DataChangeEvent} with the same tableId. + */ +public class DropTableEvent implements TableSchemaChangeEvent { + private static final long serialVersionUID = 1L; + private final TableId tableId; + + public DropTableEvent(TableId tableId) { + this.tableId = tableId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof DropTableEvent)) { + return false; + } + DropTableEvent that = (DropTableEvent) o; + return Objects.equals(tableId, that.tableId); + } + + @Override + public int hashCode() { + return Objects.hash(tableId); + } + + @Override + public String toString() { + return "DropTableEvent{" + "tableId=" + tableId + '}'; + } + + @Override + public TableId tableId() { + return tableId; + } + + @Override + public SchemaChangeEventType getType() { + return SchemaChangeEventType.DROP_TABLE; + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/RenameColumnEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/RenameColumnEvent.java index 63dd723812b..5980e55154e 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/RenameColumnEvent.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/RenameColumnEvent.java @@ -24,7 +24,7 @@ * A {@link SchemaChangeEvent} that represents an {@code RENAME COLUMN} DDL, which may contain the * lenient column type changes. */ -public class RenameColumnEvent implements SchemaChangeEvent { +public class RenameColumnEvent implements ColumnSchemaChangeEvent { private static final long serialVersionUID = 1L; @@ -70,4 +70,9 @@ public String toString() { public TableId tableId() { return tableId; } + + @Override + public SchemaChangeEventType getType() { + return SchemaChangeEventType.RENAME_COLUMN; + } } 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..c5854ed87b5 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/RenameTableEvent.java @@ -0,0 +1,75 @@ +/* + * 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 TableSchemaChangeEvent { + 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; + } + + @Override + public SchemaChangeEventType getType() { + return SchemaChangeEventType.RENAME_TABLE; + } +} 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..b76e68d7564 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,67 @@ * system, such as CREATE, DROP, RENAME and so on. */ @PublicEvolving -public interface SchemaChangeEvent extends ChangeEvent, Serializable {} +public interface SchemaChangeEvent extends ChangeEvent, Serializable { + /** Returns its {@link SchemaChangeEventType}. */ + SchemaChangeEventType getType(); + + default void visit(SchemaChangeEventVisitorVoid visitor) { + try { + if (this instanceof AddColumnEvent) { + visitor.visit((AddColumnEvent) this); + } else if (this instanceof AlterColumnCommentEvent) { + visitor.visit((AlterColumnCommentEvent) this); + } else if (this instanceof AlterColumnTypeEvent) { + visitor.visit((AlterColumnTypeEvent) this); + } else if (this instanceof AlterTableCommentEvent) { + visitor.visit((AlterTableCommentEvent) this); + } else if (this instanceof CreateTableEvent) { + visitor.visit((CreateTableEvent) this); + } else if (this instanceof DropColumnEvent) { + visitor.visit((DropColumnEvent) this); + } else if (this instanceof DropTableEvent) { + visitor.visit((DropTableEvent) this); + } else if (this instanceof RenameColumnEvent) { + visitor.visit((RenameColumnEvent) this); + } else if (this instanceof RenameTableEvent) { + visitor.visit((RenameTableEvent) this); + } else if (this instanceof TruncateTableEvent) { + visitor.visit((TruncateTableEvent) this); + } else { + throw new IllegalArgumentException("Unknown schema change event type " + getType()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + default T visit(SchemaChangeEventVisitor visitor) { + try { + if (this instanceof AddColumnEvent) { + return visitor.visit((AddColumnEvent) this); + } else if (this instanceof AlterColumnCommentEvent) { + return visitor.visit((AlterColumnCommentEvent) this); + } else if (this instanceof AlterColumnTypeEvent) { + return visitor.visit((AlterColumnTypeEvent) this); + } else if (this instanceof AlterTableCommentEvent) { + return visitor.visit((AlterTableCommentEvent) this); + } else if (this instanceof CreateTableEvent) { + return visitor.visit((CreateTableEvent) this); + } else if (this instanceof DropColumnEvent) { + return visitor.visit((DropColumnEvent) this); + } else if (this instanceof DropTableEvent) { + return visitor.visit((DropTableEvent) this); + } else if (this instanceof RenameColumnEvent) { + return visitor.visit((RenameColumnEvent) this); + } else if (this instanceof RenameTableEvent) { + return visitor.visit((RenameTableEvent) this); + } else if (this instanceof TruncateTableEvent) { + return visitor.visit((TruncateTableEvent) this); + } else { + throw new IllegalArgumentException("Unknown schema change event type " + getType()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventType.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventType.java new file mode 100644 index 00000000000..297d387d271 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventType.java @@ -0,0 +1,59 @@ +/* + * 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; + +/** An enumeration of schema change event types for {@link SchemaChangeEvent}. */ +public enum SchemaChangeEventType { + ADD_COLUMN, + ALTER_COLUMN_COMMENT, + ALTER_COLUMN_TYPE, + ALTER_TABLE_COMMENT, + CREATE_TABLE, + DROP_COLUMN, + DROP_TABLE, + RENAME_COLUMN, + RENAME_TABLE, + TRUNCATE_TABLE; + + public static SchemaChangeEventType ofTag(String tag) { + switch (tag) { + case "add.column": + return ADD_COLUMN; + case "alter.column.comment": + return ALTER_COLUMN_COMMENT; + case "alter.column.type": + return ALTER_COLUMN_TYPE; + case "alter.table.comment": + return ALTER_TABLE_COMMENT; + case "create.table": + return CREATE_TABLE; + case "drop.column": + return DROP_COLUMN; + case "drop.table": + return DROP_TABLE; + case "rename.column": + return RENAME_COLUMN; + case "rename.table": + return RENAME_TABLE; + case "truncate.table": + return TRUNCATE_TABLE; + default: + return null; + } + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventTypeFamily.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventTypeFamily.java new file mode 100644 index 00000000000..ca58f672fb1 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventTypeFamily.java @@ -0,0 +1,94 @@ +/* + * 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 static org.apache.flink.cdc.common.event.SchemaChangeEventType.ADD_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ALTER_COLUMN_COMMENT; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ALTER_COLUMN_TYPE; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ALTER_TABLE_COMMENT; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.CREATE_TABLE; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.DROP_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.DROP_TABLE; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.RENAME_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.RENAME_TABLE; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.TRUNCATE_TABLE; + +/** + * An enumeration of schema change event families for clustering {@link SchemaChangeEvent}s into + * categories. + */ +public class SchemaChangeEventTypeFamily { + + public static final SchemaChangeEventType[] ADD = {ADD_COLUMN}; + + public static final SchemaChangeEventType[] ALTER = { + ALTER_COLUMN_COMMENT, ALTER_COLUMN_TYPE, ALTER_TABLE_COMMENT + }; + + public static final SchemaChangeEventType[] CREATE = {CREATE_TABLE}; + + public static final SchemaChangeEventType[] DROP = {DROP_COLUMN, DROP_TABLE}; + + public static final SchemaChangeEventType[] RENAME = {RENAME_COLUMN, RENAME_TABLE}; + + public static final SchemaChangeEventType[] TABLE = { + ALTER_TABLE_COMMENT, CREATE_TABLE, DROP_TABLE, RENAME_TABLE, TRUNCATE_TABLE + }; + + public static final SchemaChangeEventType[] COLUMN = { + ADD_COLUMN, ALTER_COLUMN_COMMENT, ALTER_COLUMN_TYPE, DROP_COLUMN, RENAME_COLUMN + }; + + public static final SchemaChangeEventType[] ALL = { + ADD_COLUMN, + ALTER_COLUMN_COMMENT, + ALTER_COLUMN_TYPE, + ALTER_TABLE_COMMENT, + CREATE_TABLE, + DROP_COLUMN, + DROP_TABLE, + RENAME_COLUMN, + RENAME_TABLE, + TRUNCATE_TABLE + }; + + public static final SchemaChangeEventType[] NONE = {}; + + public static SchemaChangeEventType[] ofTag(String tag) { + switch (tag) { + case "add": + return ADD; + case "alter": + return ALTER; + case "create": + return CREATE; + case "drop": + return DROP; + case "rename": + return RENAME; + case "table": + return TABLE; + case "column": + return COLUMN; + case "all": + return ALL; + default: + return NONE; + } + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventVisitor.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventVisitor.java new file mode 100644 index 00000000000..8d79b72f418 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventVisitor.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.common.event; + +/** Visitor clas for all {@link SchemaChangeEvent}s and returns a {@link T}-typed object. */ +public interface SchemaChangeEventVisitor + extends ColumnSchemaChangeEventVisitor, TableSchemaChangeEventVisitor {} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventVisitorVoid.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventVisitorVoid.java new file mode 100644 index 00000000000..9608a14d6ff --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventVisitorVoid.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.common.event; + +/** Visitor clas for all {@link SchemaChangeEvent}s and returns nothing. */ +public interface SchemaChangeEventVisitorVoid + extends ColumnSchemaChangeEventVisitorVoid, TableSchemaChangeEventVisitorVoid {} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventWithPreSchema.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventWithPreSchema.java new file mode 100644 index 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/TableSchemaChangeEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEvent.java new file mode 100644 index 00000000000..18acada13a0 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEvent.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; + +/** Schema change events on table-level. */ +public interface TableSchemaChangeEvent extends SchemaChangeEvent { + + default void visit(TableSchemaChangeEventVisitorVoid visitor) { + try { + if (this instanceof AlterTableCommentEvent) { + visitor.visit((AlterTableCommentEvent) this); + } else if (this instanceof CreateTableEvent) { + visitor.visit((CreateTableEvent) this); + } else if (this instanceof DropTableEvent) { + visitor.visit((DropTableEvent) this); + } else if (this instanceof RenameTableEvent) { + visitor.visit((RenameTableEvent) this); + } else if (this instanceof TruncateTableEvent) { + visitor.visit((TruncateTableEvent) this); + } else { + throw new IllegalArgumentException("Unknown schema change event type " + getType()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + default T visit(TableSchemaChangeEventVisitor visitor) { + try { + if (this instanceof AlterTableCommentEvent) { + return visitor.visit((AlterTableCommentEvent) this); + } else if (this instanceof CreateTableEvent) { + return visitor.visit((CreateTableEvent) this); + } else if (this instanceof DropTableEvent) { + return visitor.visit((DropTableEvent) this); + } else if (this instanceof RenameTableEvent) { + return visitor.visit((RenameTableEvent) this); + } else if (this instanceof TruncateTableEvent) { + return visitor.visit((TruncateTableEvent) this); + } else { + throw new IllegalArgumentException("Unknown schema change event type " + getType()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEventVisitor.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEventVisitor.java new file mode 100644 index 00000000000..559a8a36314 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEventVisitor.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.common.event; + +/** Visitor class for all {@link TableSchemaChangeEvent}s. */ +public interface TableSchemaChangeEventVisitor { + T visit(AlterTableCommentEvent event) throws Exception; + + T visit(CreateTableEvent event) throws Exception; + + T visit(DropTableEvent event) throws Exception; + + T visit(RenameTableEvent event) throws Exception; + + T visit(TruncateTableEvent event) throws Exception; +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEventVisitorVoid.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEventVisitorVoid.java new file mode 100644 index 00000000000..da0bb23e974 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TableSchemaChangeEventVisitorVoid.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.common.event; + +/** Visitor class for all {@link TableSchemaChangeEvent}s and returns nothing. */ +public interface TableSchemaChangeEventVisitorVoid { + void visit(AlterTableCommentEvent event) throws Exception; + + void visit(CreateTableEvent event) throws Exception; + + void visit(DropTableEvent event) throws Exception; + + void visit(RenameTableEvent event) throws Exception; + + void visit(TruncateTableEvent event) throws Exception; +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TruncateTableEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TruncateTableEvent.java new file mode 100644 index 00000000000..ef13ea6b987 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/TruncateTableEvent.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.common.event; + +import org.apache.flink.cdc.common.source.DataSource; + +import java.util.Objects; + +/** + * A {@link SchemaChangeEvent} that represents an {@code TRUNCATE TABLE} DDL. this will be sent by + * {@link DataSource} before all {@link DataChangeEvent} with the same tableId. + */ +public class TruncateTableEvent implements TableSchemaChangeEvent { + private static final long serialVersionUID = 1L; + private final TableId tableId; + + public TruncateTableEvent(TableId tableId) { + this.tableId = tableId; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof TruncateTableEvent)) { + return false; + } + TruncateTableEvent that = (TruncateTableEvent) o; + return Objects.equals(tableId, that.tableId); + } + + @Override + public int hashCode() { + return Objects.hash(tableId); + } + + @Override + public String toString() { + return "TruncateTableEvent{" + "tableId=" + tableId + '}'; + } + + @Override + public TableId tableId() { + return tableId; + } + + @Override + public SchemaChangeEventType getType() { + return SchemaChangeEventType.TRUNCATE_TABLE; + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/pipeline/SchemaChangeBehavior.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/pipeline/SchemaChangeBehavior.java index 1ac5ddbb7e2..3634caba44e 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/pipeline/SchemaChangeBehavior.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/pipeline/SchemaChangeBehavior.java @@ -22,7 +22,8 @@ /** Behavior for handling schema changes. */ @PublicEvolving public enum SchemaChangeBehavior { - EVOLVE, IGNORE, + TRY_EVOLVE, + EVOLVE, EXCEPTION } diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/Column.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/Column.java index 5c1c59d5e69..8bab9718f02 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/Column.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/Column.java @@ -92,6 +92,9 @@ public String asSummaryString() { /** Returns a copy of the column with a replaced name. */ public abstract Column copy(String newName); + /** Returns a copy of the column with a replaced comment. */ + public abstract Column copyComment(String newComment); + @Override public boolean equals(Object o) { if (this == o) { diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/MetadataColumn.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/MetadataColumn.java index 89eecf46f87..b1d0d340169 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/MetadataColumn.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/MetadataColumn.java @@ -58,6 +58,11 @@ public Column copy(String newName) { return new MetadataColumn(newName, type, metadataKey, comment); } + @Override + public Column copyComment(String newComment) { + return new MetadataColumn(name, type, metadataKey, newComment); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/PhysicalColumn.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/PhysicalColumn.java index 1f3e2642796..1af200edfc8 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/PhysicalColumn.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/PhysicalColumn.java @@ -46,4 +46,9 @@ public Column copy(DataType newType) { public Column copy(String newName) { return new PhysicalColumn(newName, type, comment); } + + @Override + public Column copyComment(String newComment) { + return new PhysicalColumn(name, type, newComment); + } } diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/Schema.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/Schema.java index 62657be8855..4fc02b45241 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/Schema.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/schema/Schema.java @@ -168,6 +168,16 @@ public Schema copy(List columns) { comment); } + /** Returns a copy of the schema with replaced comments. */ + public Schema copyComment(String newComment) { + return new Schema( + columns, + new ArrayList<>(primaryKeys), + new ArrayList<>(partitionKeys), + new HashMap<>(options), + newComment); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/sink/DataSink.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/sink/DataSink.java index a6f92e44af2..f0f32d33c1d 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/sink/DataSink.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/sink/DataSink.java @@ -18,6 +18,9 @@ package org.apache.flink.cdc.common.sink; import org.apache.flink.cdc.common.annotation.PublicEvolving; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; + +import java.util.Set; /** * {@code DataSink} is used to write change data to external system and apply metadata changes to @@ -30,5 +33,5 @@ public interface DataSink { EventSinkProvider getEventSinkProvider(); /** Get the {@link MetadataApplier} for applying metadata changes to external systems. */ - MetadataApplier getMetadataApplier(); + MetadataApplier getMetadataApplier(Set enabledEventTypes); } diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/sink/MetadataApplier.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/sink/MetadataApplier.java index 219f833cfe0..28aa63c726b 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/sink/MetadataApplier.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/sink/MetadataApplier.java @@ -19,13 +19,21 @@ import org.apache.flink.cdc.common.annotation.PublicEvolving; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; import java.io.Serializable; +import java.util.Set; /** {@code MetadataApplier} is used to apply metadata changes to external systems. */ @PublicEvolving public interface MetadataApplier extends Serializable { + /** Checks if this metadata applier should handle this event type. */ + boolean acceptsSchemaEvolutionType(SchemaChangeEventType schemaChangeEventType); + + /** Checks what kind of schema change events downstream can handle. */ + Set getSupportedSchemaEvolutionTypes(); + /** Apply the given {@link SchemaChangeEvent} to external systems. */ void applySchemaChange(SchemaChangeEvent schemaChangeEvent); } 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..61788a7e546 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 @@ -17,14 +17,29 @@ package org.apache.flink.cdc.common.utils; +import org.apache.flink.cdc.common.annotation.VisibleForTesting; import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnCommentEvent; import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; +import org.apache.flink.cdc.common.event.AlterTableCommentEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DataChangeEvent; import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.DropTableEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; +import org.apache.flink.cdc.common.event.RenameTableEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; +import org.apache.flink.cdc.common.event.SchemaChangeEventTypeFamily; +import org.apache.flink.cdc.common.event.SchemaChangeEventVisitor; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; /** Utilities for handling {@link org.apache.flink.cdc.common.event.ChangeEvent}s. */ public class ChangeEventUtils { @@ -56,29 +71,93 @@ 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()); + + return schemaChangeEvent.visit( + new SchemaChangeEventVisitor() { + @Override + public SchemaChangeEvent visit(AddColumnEvent event) { + return new AddColumnEvent(tableId, event.getAddedColumns()); + } + + @Override + public SchemaChangeEvent visit(AlterColumnCommentEvent event) { + return new AlterColumnCommentEvent( + tableId, event.getCommentMapping(), event.getOldCommentMapping()); + } + + @Override + public SchemaChangeEvent visit(AlterColumnTypeEvent event) { + return new AlterColumnTypeEvent( + tableId, event.getTypeMapping(), event.getOldTypeMapping()); + } + + @Override + public SchemaChangeEvent visit(AlterTableCommentEvent event) { + return new AlterTableCommentEvent(tableId, event.getTableComment()); + } + + @Override + public SchemaChangeEvent visit(CreateTableEvent event) { + return new CreateTableEvent(tableId, event.getSchema()); + } + + @Override + public SchemaChangeEvent visit(DropColumnEvent event) { + return new DropColumnEvent(tableId, event.getDroppedColumnNames()); + } + + @Override + public SchemaChangeEvent visit(DropTableEvent event) { + return new DropTableEvent(tableId); + } + + @Override + public SchemaChangeEvent visit(RenameColumnEvent event) { + return new RenameColumnEvent(tableId, event.getNameMapping()); + } + + @Override + public SchemaChangeEvent visit(RenameTableEvent event) { + return new RenameTableEvent(tableId, event.newTableId()); + } + + @Override + public SchemaChangeEvent visit(TruncateTableEvent event) { + return new TruncateTableEvent(tableId); + } + }); + } + + public static Set resolveSchemaEvolutionOptions( + List includedSchemaEvolutionTypes, List excludedSchemaEvolutionTypes) { + List resultTypes = new ArrayList<>(); + + if (includedSchemaEvolutionTypes.isEmpty()) { + resultTypes.addAll(Arrays.asList(SchemaChangeEventTypeFamily.ALL)); + } else { + for (String includeTag : includedSchemaEvolutionTypes) { + resultTypes.addAll(resolveSchemaEvolutionTag(includeTag)); + } } - if (schemaChangeEvent instanceof DropColumnEvent) { - DropColumnEvent dropColumnEvent = (DropColumnEvent) schemaChangeEvent; - return new DropColumnEvent(tableId, dropColumnEvent.getDroppedColumnNames()); + + for (String excludeTag : excludedSchemaEvolutionTypes) { + resultTypes.removeAll(resolveSchemaEvolutionTag(excludeTag)); } - if (schemaChangeEvent instanceof AddColumnEvent) { - AddColumnEvent addColumnEvent = (AddColumnEvent) schemaChangeEvent; - return new AddColumnEvent(tableId, addColumnEvent.getAddedColumns()); + + return new HashSet<>(resultTypes); + } + + @VisibleForTesting + public static List resolveSchemaEvolutionTag(String tag) { + List types = + new ArrayList<>(Arrays.asList(SchemaChangeEventTypeFamily.ofTag(tag))); + if (types.isEmpty()) { + // It's a specified tag + SchemaChangeEventType type = SchemaChangeEventType.ofTag(tag); + if (type != null) { + types.add(type); + } } - throw new UnsupportedOperationException( - String.format( - "Unsupported schema change event with type \"%s\"", - schemaChangeEvent.getClass().getCanonicalName())); + return types; } } 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..6216d7dcbce 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 @@ -17,13 +17,22 @@ package org.apache.flink.cdc.common.utils; +import org.apache.flink.api.java.tuple.Tuple2; 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.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.SchemaChangeEventVisitor; +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; @@ -57,21 +66,63 @@ 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 Tuple2 applySchemaChangeEvent( + SchemaChangeEvent event, Schema oldSchema) { + return event.visit( + new SchemaChangeEventVisitor>() { + @Override + public Tuple2 visit(AddColumnEvent event) { + return Tuple2.of(event.tableId(), applyAddColumnEvent(event, oldSchema)); + } + + @Override + public Tuple2 visit(AlterColumnCommentEvent event) { + return Tuple2.of( + event.tableId(), applyAlterColumnCommentEvent(event, oldSchema)); + } + + @Override + public Tuple2 visit(AlterColumnTypeEvent event) { + return Tuple2.of( + event.tableId(), applyAlterColumnTypeEvent(event, oldSchema)); + } + + @Override + public Tuple2 visit(AlterTableCommentEvent event) { + return Tuple2.of( + event.tableId(), oldSchema.copyComment(event.getTableComment())); + } + + @Override + public Tuple2 visit(CreateTableEvent event) { + return Tuple2.of(event.tableId(), event.getSchema()); + } + + @Override + public Tuple2 visit(DropColumnEvent event) { + return Tuple2.of(event.tableId(), applyDropColumnEvent(event, oldSchema)); + } + + @Override + public Tuple2 visit(DropTableEvent event) { + return Tuple2.of(event.tableId(), null); + } + + @Override + public Tuple2 visit(RenameColumnEvent event) { + return Tuple2.of(event.tableId(), applyRenameColumnEvent(event, oldSchema)); + } + + @Override + public Tuple2 visit(RenameTableEvent event) { + return Tuple2.of(event.newTableId(), oldSchema); + } + + @Override + public Tuple2 visit(TruncateTableEvent event) { + return Tuple2.of(event.tableId(), oldSchema); + } + }); } private static Schema applyAddColumnEvent(AddColumnEvent event, Schema oldSchema) { @@ -166,4 +217,22 @@ private static Schema applyAlterColumnTypeEvent(AlterColumnTypeEvent event, Sche }); return oldSchema.copy(columns); } + + private static Schema applyAlterColumnCommentEvent( + AlterColumnCommentEvent event, Schema oldSchema) { + List columns = new ArrayList<>(); + oldSchema + .getColumns() + .forEach( + column -> { + if (event.getCommentMapping().containsKey(column.getName())) { + columns.add( + column.copyComment( + event.getCommentMapping().get(column.getName()))); + } else { + columns.add(column); + } + }); + return oldSchema.copy(columns); + } } diff --git a/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/ChangeEventUtilsTest.java b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/ChangeEventUtilsTest.java new file mode 100644 index 00000000000..fd949690e81 --- /dev/null +++ b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/ChangeEventUtilsTest.java @@ -0,0 +1,170 @@ +/* + * 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.utils; + +import org.assertj.core.api.Assertions; +import org.assertj.core.util.Sets; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; + +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ADD_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ALTER_COLUMN_COMMENT; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ALTER_COLUMN_TYPE; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ALTER_TABLE_COMMENT; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.CREATE_TABLE; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.DROP_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.DROP_TABLE; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.RENAME_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.RENAME_TABLE; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.TRUNCATE_TABLE; + +/** A test for the {@link org.apache.flink.cdc.common.utils.ChangeEventUtils}. */ +public class ChangeEventUtilsTest { + @Test + public void testResolveSchemaEvolutionOptions() { + Assertions.assertThat( + ChangeEventUtils.resolveSchemaEvolutionOptions( + Collections.emptyList(), Collections.emptyList())) + .isEqualTo( + Sets.set( + ALTER_COLUMN_COMMENT, + TRUNCATE_TABLE, + RENAME_COLUMN, + CREATE_TABLE, + DROP_TABLE, + ALTER_COLUMN_TYPE, + ALTER_TABLE_COMMENT, + RENAME_TABLE, + ADD_COLUMN, + DROP_COLUMN)); + + Assertions.assertThat( + ChangeEventUtils.resolveSchemaEvolutionOptions( + Collections.emptyList(), Collections.singletonList("drop"))) + .isEqualTo( + Sets.set( + ADD_COLUMN, + RENAME_TABLE, + ALTER_TABLE_COMMENT, + ALTER_COLUMN_COMMENT, + ALTER_COLUMN_TYPE, + RENAME_COLUMN, + CREATE_TABLE, + TRUNCATE_TABLE)); + + Assertions.assertThat( + ChangeEventUtils.resolveSchemaEvolutionOptions( + Arrays.asList("create", "add"), Collections.emptyList())) + .isEqualTo(Sets.set(ADD_COLUMN, CREATE_TABLE)); + + Assertions.assertThat( + ChangeEventUtils.resolveSchemaEvolutionOptions( + Collections.singletonList("column"), + Collections.singletonList("drop.column"))) + .isEqualTo( + Sets.set( + ADD_COLUMN, + ALTER_COLUMN_COMMENT, + ALTER_COLUMN_TYPE, + RENAME_COLUMN)); + + Assertions.assertThat( + ChangeEventUtils.resolveSchemaEvolutionOptions( + Collections.emptyList(), Collections.singletonList("drop.column"))) + .isEqualTo( + Sets.set( + RENAME_TABLE, + ADD_COLUMN, + ALTER_COLUMN_COMMENT, + DROP_TABLE, + TRUNCATE_TABLE, + RENAME_COLUMN, + ALTER_COLUMN_TYPE, + ALTER_TABLE_COMMENT, + CREATE_TABLE)); + } + + @Test + public void testResolveSchemaEvolutionTag() { + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("all")) + .isEqualTo( + Arrays.asList( + ADD_COLUMN, + ALTER_COLUMN_COMMENT, + ALTER_COLUMN_TYPE, + ALTER_TABLE_COMMENT, + CREATE_TABLE, + DROP_COLUMN, + DROP_TABLE, + RENAME_COLUMN, + RENAME_TABLE, + TRUNCATE_TABLE)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("column")) + .isEqualTo( + Arrays.asList( + ADD_COLUMN, + ALTER_COLUMN_COMMENT, + ALTER_COLUMN_TYPE, + DROP_COLUMN, + RENAME_COLUMN)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("table")) + .isEqualTo( + Arrays.asList( + ALTER_TABLE_COMMENT, + CREATE_TABLE, + DROP_TABLE, + RENAME_TABLE, + TRUNCATE_TABLE)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("rename")) + .isEqualTo(Arrays.asList(RENAME_COLUMN, RENAME_TABLE)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("rename.column")) + .isEqualTo(Collections.singletonList(RENAME_COLUMN)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("drop")) + .isEqualTo(Arrays.asList(DROP_COLUMN, DROP_TABLE)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("drop.column")) + .isEqualTo(Collections.singletonList(DROP_COLUMN)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("create")) + .isEqualTo(Collections.singletonList(CREATE_TABLE)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("create.table")) + .isEqualTo(Collections.singletonList(CREATE_TABLE)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("alter")) + .isEqualTo( + Arrays.asList( + ALTER_COLUMN_COMMENT, ALTER_COLUMN_TYPE, ALTER_TABLE_COMMENT)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("alter.column.type")) + .isEqualTo(Collections.singletonList(ALTER_COLUMN_TYPE)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("add")) + .isEqualTo(Collections.singletonList(ADD_COLUMN)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("add.column")) + .isEqualTo(Collections.singletonList(ADD_COLUMN)); + } +} 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..7bdc88215f8 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 @@ -27,8 +27,8 @@ import org.apache.flink.cdc.common.types.DataType; import org.apache.flink.cdc.common.types.DataTypes; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Arrays; @@ -40,7 +40,7 @@ public class SchemaUtilsTest { @Test - public void testApplySchemaChangeEvent() { + public void testApplyColumnSchemaChangeEvent() { TableId tableId = TableId.parse("default.default.table1"); Schema schema = Schema.newBuilder() @@ -54,14 +54,14 @@ public void testApplySchemaChangeEvent() { new AddColumnEvent.ColumnWithPosition( Column.physicalColumn("col3", DataTypes.STRING()))); AddColumnEvent addColumnEvent = new AddColumnEvent(tableId, addedColumns); - schema = SchemaUtils.applySchemaChangeEvent(schema, addColumnEvent); - Assert.assertEquals( - schema, - Schema.newBuilder() - .physicalColumn("col1", DataTypes.STRING()) - .physicalColumn("col2", DataTypes.STRING()) - .physicalColumn("col3", DataTypes.STRING()) - .build()); + schema = SchemaUtils.applySchemaChangeEvent(addColumnEvent, schema).f1; + Assertions.assertThat(schema) + .isEqualTo( + Schema.newBuilder() + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("col2", DataTypes.STRING()) + .physicalColumn("col3", DataTypes.STRING()) + .build()); // add new column before existed column addedColumns = new ArrayList<>(); @@ -71,15 +71,15 @@ public void testApplySchemaChangeEvent() { AddColumnEvent.ColumnPosition.BEFORE, "col3")); addColumnEvent = new AddColumnEvent(tableId, addedColumns); - schema = SchemaUtils.applySchemaChangeEvent(schema, addColumnEvent); - Assert.assertEquals( - schema, - Schema.newBuilder() - .physicalColumn("col1", DataTypes.STRING()) - .physicalColumn("col2", DataTypes.STRING()) - .physicalColumn("col4", DataTypes.STRING()) - .physicalColumn("col3", DataTypes.STRING()) - .build()); + schema = SchemaUtils.applySchemaChangeEvent(addColumnEvent, schema).f1; + Assertions.assertThat(schema) + .isEqualTo( + Schema.newBuilder() + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("col2", DataTypes.STRING()) + .physicalColumn("col4", DataTypes.STRING()) + .physicalColumn("col3", DataTypes.STRING()) + .build()); // add new column after existed column addedColumns = new ArrayList<>(); @@ -89,16 +89,16 @@ public void testApplySchemaChangeEvent() { AddColumnEvent.ColumnPosition.AFTER, "col4")); addColumnEvent = new AddColumnEvent(tableId, addedColumns); - schema = SchemaUtils.applySchemaChangeEvent(schema, addColumnEvent); - Assert.assertEquals( - schema, - Schema.newBuilder() - .physicalColumn("col1", DataTypes.STRING()) - .physicalColumn("col2", DataTypes.STRING()) - .physicalColumn("col4", DataTypes.STRING()) - .physicalColumn("col5", DataTypes.STRING()) - .physicalColumn("col3", DataTypes.STRING()) - .build()); + schema = SchemaUtils.applySchemaChangeEvent(addColumnEvent, schema).f1; + Assertions.assertThat(schema) + .isEqualTo( + Schema.newBuilder() + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("col2", DataTypes.STRING()) + .physicalColumn("col4", DataTypes.STRING()) + .physicalColumn("col5", DataTypes.STRING()) + .physicalColumn("col3", DataTypes.STRING()) + .build()); // add column in first position addedColumns = new ArrayList<>(); @@ -108,59 +108,59 @@ public void testApplySchemaChangeEvent() { AddColumnEvent.ColumnPosition.FIRST, null)); addColumnEvent = new AddColumnEvent(tableId, addedColumns); - schema = SchemaUtils.applySchemaChangeEvent(schema, addColumnEvent); - Assert.assertEquals( - schema, - Schema.newBuilder() - .physicalColumn("col0", DataTypes.STRING()) - .physicalColumn("col1", DataTypes.STRING()) - .physicalColumn("col2", DataTypes.STRING()) - .physicalColumn("col4", DataTypes.STRING()) - .physicalColumn("col5", DataTypes.STRING()) - .physicalColumn("col3", DataTypes.STRING()) - .build()); + schema = SchemaUtils.applySchemaChangeEvent(addColumnEvent, schema).f1; + Assertions.assertThat(schema) + .isEqualTo( + Schema.newBuilder() + .physicalColumn("col0", DataTypes.STRING()) + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("col2", DataTypes.STRING()) + .physicalColumn("col4", DataTypes.STRING()) + .physicalColumn("col5", DataTypes.STRING()) + .physicalColumn("col3", DataTypes.STRING()) + .build()); // drop columns DropColumnEvent dropColumnEvent = new DropColumnEvent(tableId, Arrays.asList("col3", "col5")); - schema = SchemaUtils.applySchemaChangeEvent(schema, dropColumnEvent); - Assert.assertEquals( - schema, - Schema.newBuilder() - .physicalColumn("col0", DataTypes.STRING()) - .physicalColumn("col1", DataTypes.STRING()) - .physicalColumn("col2", DataTypes.STRING()) - .physicalColumn("col4", DataTypes.STRING()) - .build()); + schema = SchemaUtils.applySchemaChangeEvent(dropColumnEvent, schema).f1; + Assertions.assertThat(schema) + .isEqualTo( + Schema.newBuilder() + .physicalColumn("col0", DataTypes.STRING()) + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("col2", DataTypes.STRING()) + .physicalColumn("col4", DataTypes.STRING()) + .build()); // rename columns Map nameMapping = new HashMap<>(); nameMapping.put("col2", "newCol2"); nameMapping.put("col4", "newCol4"); RenameColumnEvent renameColumnEvent = new RenameColumnEvent(tableId, nameMapping); - schema = SchemaUtils.applySchemaChangeEvent(schema, renameColumnEvent); - Assert.assertEquals( - schema, - Schema.newBuilder() - .physicalColumn("col0", DataTypes.STRING()) - .physicalColumn("col1", DataTypes.STRING()) - .physicalColumn("newCol2", DataTypes.STRING()) - .physicalColumn("newCol4", DataTypes.STRING()) - .build()); + schema = SchemaUtils.applySchemaChangeEvent(renameColumnEvent, schema).f1; + Assertions.assertThat(schema) + .isEqualTo( + Schema.newBuilder() + .physicalColumn("col0", DataTypes.STRING()) + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("newCol2", DataTypes.STRING()) + .physicalColumn("newCol4", DataTypes.STRING()) + .build()); // alter column types Map typeMapping = new HashMap<>(); typeMapping.put("newCol2", DataTypes.VARCHAR(10)); typeMapping.put("newCol4", DataTypes.VARCHAR(10)); AlterColumnTypeEvent alterColumnTypeEvent = new AlterColumnTypeEvent(tableId, typeMapping); - schema = SchemaUtils.applySchemaChangeEvent(schema, alterColumnTypeEvent); - Assert.assertEquals( - schema, - Schema.newBuilder() - .physicalColumn("col0", DataTypes.STRING()) - .physicalColumn("col1", DataTypes.STRING()) - .physicalColumn("newCol2", DataTypes.VARCHAR(10)) - .physicalColumn("newCol4", DataTypes.VARCHAR(10)) - .build()); + schema = SchemaUtils.applySchemaChangeEvent(alterColumnTypeEvent, schema).f1; + Assertions.assertThat(schema) + .isEqualTo( + Schema.newBuilder() + .physicalColumn("col0", DataTypes.STRING()) + .physicalColumn("col1", DataTypes.STRING()) + .physicalColumn("newCol2", DataTypes.VARCHAR(10)) + .physicalColumn("newCol4", DataTypes.VARCHAR(10)) + .build()); } } diff --git a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/definition/SinkDef.java b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/definition/SinkDef.java index e2579f75bf5..10772dabf15 100644 --- a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/definition/SinkDef.java +++ b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/definition/SinkDef.java @@ -18,11 +18,16 @@ package org.apache.flink.cdc.composer.definition; import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; +import org.apache.flink.cdc.common.event.SchemaChangeEventTypeFamily; import javax.annotation.Nullable; +import java.util.Arrays; import java.util.Objects; import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; /** * Definition of a data sink. @@ -40,11 +45,25 @@ public class SinkDef { private final String type; @Nullable private final String name; private final Configuration config; + private final Set includedSchemaEvolutionTypes; public SinkDef(String type, @Nullable String name, Configuration config) { this.type = type; this.name = name; this.config = config; + this.includedSchemaEvolutionTypes = + Arrays.stream(SchemaChangeEventTypeFamily.ALL).collect(Collectors.toSet()); + } + + public SinkDef( + String type, + @Nullable String name, + Configuration config, + Set includedSchemaEvolutionTypes) { + this.type = type; + this.name = name; + this.config = config; + this.includedSchemaEvolutionTypes = includedSchemaEvolutionTypes; } public String getType() { @@ -59,6 +78,10 @@ public Configuration getConfig() { return config; } + public Set getIncludedSchemaEvolutionTypes() { + return includedSchemaEvolutionTypes; + } + @Override public String toString() { return "SinkDef{" @@ -70,6 +93,8 @@ public String toString() { + '\'' + ", config=" + config + + ", includedSchemaEvolutionTypes=" + + includedSchemaEvolutionTypes + '}'; } @@ -84,11 +109,13 @@ public boolean equals(Object o) { SinkDef sinkDef = (SinkDef) o; return Objects.equals(type, sinkDef.type) && Objects.equals(name, sinkDef.name) - && Objects.equals(config, sinkDef.config); + && Objects.equals(config, sinkDef.config) + && Objects.equals( + includedSchemaEvolutionTypes, sinkDef.includedSchemaEvolutionTypes); } @Override public int hashCode() { - return Objects.hash(type, name, config); + return Objects.hash(type, name, config, includedSchemaEvolutionTypes); } } diff --git a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/FlinkPipelineComposer.java b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/FlinkPipelineComposer.java index 73910d59be7..98435fdfba9 100644 --- a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/FlinkPipelineComposer.java +++ b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/FlinkPipelineComposer.java @@ -23,6 +23,7 @@ import org.apache.flink.cdc.common.factories.DataSinkFactory; import org.apache.flink.cdc.common.factories.FactoryHelper; import org.apache.flink.cdc.common.pipeline.PipelineOptions; +import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.composer.PipelineComposer; import org.apache.flink.cdc.composer.PipelineExecution; @@ -95,6 +96,9 @@ public PipelineExecution compose(PipelineDef pipelineDef) { int parallelism = pipelineDef.getConfig().get(PipelineOptions.PIPELINE_PARALLELISM); env.getConfig().setParallelism(parallelism); + SchemaChangeBehavior schemaChangeBehavior = + pipelineDef.getConfig().get(PipelineOptions.PIPELINE_SCHEMA_CHANGE_BEHAVIOR); + // Build Source Operator DataSourceTranslator sourceTranslator = new DataSourceTranslator(); DataStream stream = @@ -107,9 +111,7 @@ public PipelineExecution compose(PipelineDef pipelineDef) { // Schema operator SchemaOperatorTranslator schemaOperatorTranslator = new SchemaOperatorTranslator( - pipelineDef - .getConfig() - .get(PipelineOptions.PIPELINE_SCHEMA_CHANGE_BEHAVIOR), + schemaChangeBehavior, pipelineDef.getConfig().get(PipelineOptions.PIPELINE_SCHEMA_OPERATOR_UID), pipelineDef .getConfig() @@ -130,7 +132,11 @@ public PipelineExecution compose(PipelineDef pipelineDef) { stream = schemaOperatorTranslator.translate( - stream, parallelism, dataSink.getMetadataApplier(), pipelineDef.getRoute()); + stream, + parallelism, + dataSink.getMetadataApplier( + pipelineDef.getSink().getIncludedSchemaEvolutionTypes()), + pipelineDef.getRoute()); // Build Partitioner used to shuffle Event PartitioningTranslator partitioningTranslator = new PartitioningTranslator(); diff --git a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslator.java b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslator.java index ec0bcb593cf..1ac924e2458 100644 --- a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslator.java +++ b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslator.java @@ -82,7 +82,7 @@ void sinkTo( String sinkName, OperatorID schemaOperatorID) { DataStream stream = input; - // Pre write topology + // Pre-write topology if (sink instanceof WithPreWriteTopology) { stream = ((WithPreWriteTopology) sink).addPreWriteTopology(stream); } diff --git a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/SchemaOperatorTranslator.java b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/SchemaOperatorTranslator.java index a69741c7b4a..d93fddda341 100644 --- a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/SchemaOperatorTranslator.java +++ b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/SchemaOperatorTranslator.java @@ -57,18 +57,7 @@ public DataStream translate( int parallelism, MetadataApplier metadataApplier, List routes) { - switch (schemaChangeBehavior) { - case EVOLVE: - return addSchemaOperator(input, parallelism, metadataApplier, routes); - case IGNORE: - return dropSchemaChangeEvent(input, parallelism); - case EXCEPTION: - return exceptionOnSchemaChange(input, parallelism); - default: - throw new IllegalArgumentException( - String.format( - "Unrecognized schema change behavior: %s", schemaChangeBehavior)); - } + return addSchemaOperator(input, parallelism, metadataApplier, routes, schemaChangeBehavior); } public String getSchemaOperatorUid() { @@ -79,7 +68,8 @@ private DataStream addSchemaOperator( DataStream input, int parallelism, MetadataApplier metadataApplier, - List routes) { + List routes, + SchemaChangeBehavior schemaChangeBehavior) { List> routingRules = new ArrayList<>(); for (RouteDef route : routes) { routingRules.add( @@ -89,7 +79,8 @@ private DataStream addSchemaOperator( input.transform( "SchemaOperator", new EventTypeInfo(), - new SchemaOperatorFactory(metadataApplier, routingRules, rpcTimeOut)); + new SchemaOperatorFactory( + metadataApplier, routingRules, rpcTimeOut, schemaChangeBehavior)); stream.uid(schemaOperatorUid).setParallelism(parallelism); return stream; } 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-composer/src/test/java/org/apache/flink/cdc/composer/utils/factory/DataSinkFactory1.java b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/utils/factory/DataSinkFactory1.java index 19a9e00e6ee..a1eda19152e 100644 --- a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/utils/factory/DataSinkFactory1.java +++ b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/utils/factory/DataSinkFactory1.java @@ -18,6 +18,7 @@ package org.apache.flink.cdc.composer.utils.factory; import org.apache.flink.cdc.common.configuration.ConfigOption; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; import org.apache.flink.cdc.common.factories.DataSinkFactory; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.common.sink.EventSinkProvider; @@ -69,7 +70,7 @@ public EventSinkProvider getEventSinkProvider() { } @Override - public MetadataApplier getMetadataApplier() { + public MetadataApplier getMetadataApplier(Set enabledEventTypes) { return null; } } diff --git a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/utils/factory/DataSinkFactory2.java b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/utils/factory/DataSinkFactory2.java index c010faae44e..0caaaf4d2d9 100644 --- a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/utils/factory/DataSinkFactory2.java +++ b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/utils/factory/DataSinkFactory2.java @@ -18,6 +18,7 @@ package org.apache.flink.cdc.composer.utils.factory; import org.apache.flink.cdc.common.configuration.ConfigOption; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; import org.apache.flink.cdc.common.factories.DataSinkFactory; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.common.sink.EventSinkProvider; @@ -37,7 +38,8 @@ public EventSinkProvider getEventSinkProvider() { } @Override - public MetadataApplier getMetadataApplier() { + public MetadataApplier getMetadataApplier( + Set enabledEventTypes) { return null; } }; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisDataSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisDataSink.java index 130d2170c64..1fe9bb5246b 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisDataSink.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisDataSink.java @@ -18,6 +18,7 @@ package org.apache.flink.cdc.connectors.doris.sink; import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.common.sink.EventSinkProvider; import org.apache.flink.cdc.common.sink.FlinkSinkProvider; @@ -31,6 +32,7 @@ import java.io.Serializable; import java.time.ZoneId; +import java.util.Set; /** A {@link DataSink} for "Doris" connector. */ public class DorisDataSink implements DataSink, Serializable { @@ -74,7 +76,7 @@ public EventSinkProvider getEventSinkProvider() { } @Override - public MetadataApplier getMetadataApplier() { - return new DorisMetadataApplier(dorisOptions, configuration); + public MetadataApplier getMetadataApplier(Set enabledEventTypes) { + return new DorisMetadataApplier(dorisOptions, configuration, enabledEventTypes); } } 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 a1cb9182d1a..47a4f4491e0 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 @@ -17,8 +17,8 @@ package org.apache.flink.cdc.connectors.doris.sink; +import org.apache.flink.api.java.tuple.Tuple2; 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.Event; import org.apache.flink.cdc.common.event.OperationType; @@ -71,17 +71,14 @@ public DorisRecord serialize(Event event) throws IOException { return applyDataChangeEvent((DataChangeEvent) event); } else if (event instanceof SchemaChangeEvent) { SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; - TableId tableId = schemaChangeEvent.tableId(); - if (event instanceof CreateTableEvent) { - schemaMaps.put(tableId, ((CreateTableEvent) event).getSchema()); + Tuple2 appliedSchema = + SchemaUtils.applySchemaChangeEvent( + schemaChangeEvent, schemaMaps.get(schemaChangeEvent.tableId())); + + if (appliedSchema.f1 != null) { + schemaMaps.put(appliedSchema.f0, appliedSchema.f1); } else { - if (!schemaMaps.containsKey(tableId)) { - throw new RuntimeException("schema of " + tableId + " is not existed."); - } - schemaMaps.put( - tableId, - SchemaUtils.applySchemaChangeEvent( - schemaMaps.get(tableId), schemaChangeEvent)); + schemaMaps.remove(appliedSchema.f0); } } return null; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisMetadataApplier.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/main/java/org/apache/flink/cdc/connectors/doris/sink/DorisMetadataApplier.java index 7e107eac17f..caa8411d46f 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,12 +19,19 @@ import org.apache.flink.cdc.common.configuration.Configuration; import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnCommentEvent; import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; +import org.apache.flink.cdc.common.event.AlterTableCommentEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.DropTableEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; +import org.apache.flink.cdc.common.event.RenameTableEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; +import org.apache.flink.cdc.common.event.SchemaChangeEventVisitorVoid; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.sink.MetadataApplier; @@ -35,6 +42,8 @@ import org.apache.flink.cdc.common.types.utils.DataTypeUtils; import org.apache.flink.util.CollectionUtil; +import org.apache.flink.shaded.guava31.com.google.common.collect.Sets; + import org.apache.doris.flink.catalog.DorisTypeMapper; import org.apache.doris.flink.catalog.doris.DataModel; import org.apache.doris.flink.catalog.doris.FieldSchema; @@ -51,7 +60,11 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Set; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ADD_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.DROP_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.RENAME_COLUMN; import static org.apache.flink.cdc.connectors.doris.sink.DorisDataSinkOptions.TABLE_CREATE_PROPERTIES_PREFIX; /** Supports {@link DorisDataSink} to schema evolution. */ @@ -60,32 +73,96 @@ public class DorisMetadataApplier implements MetadataApplier { private DorisOptions dorisOptions; private SchemaChangeManager schemaChangeManager; private Configuration config; + private final Set enabledSchemaEvolutionTypes; public DorisMetadataApplier(DorisOptions dorisOptions, Configuration config) { this.dorisOptions = dorisOptions; this.schemaChangeManager = new SchemaChangeManager(dorisOptions); this.config = config; + this.enabledSchemaEvolutionTypes = getSupportedSchemaEvolutionTypes(); + } + + public DorisMetadataApplier( + DorisOptions dorisOptions, + Configuration config, + Set enabledSchemaEvolutionTypes) { + this.dorisOptions = dorisOptions; + this.schemaChangeManager = new SchemaChangeManager(dorisOptions); + this.config = config; + this.enabledSchemaEvolutionTypes = enabledSchemaEvolutionTypes; + } + + @Override + public boolean acceptsSchemaEvolutionType(SchemaChangeEventType schemaChangeEventType) { + return enabledSchemaEvolutionTypes.contains(schemaChangeEventType); + } + + @Override + public Set getSupportedSchemaEvolutionTypes() { + return Sets.newHashSet(ADD_COLUMN, DROP_COLUMN, RENAME_COLUMN); } @Override 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); - } - } catch (Exception ex) { - throw new RuntimeException( - "Failed to schema change, " + event + ", reason: " + ex.getMessage()); - } + event.visit( + new SchemaChangeEventVisitorVoid() { + + @Override + public void visit(AddColumnEvent event) throws Exception { + applyAddColumnEvent(event); + } + + @Override + public void visit(AlterColumnCommentEvent event) { + throw new UnsupportedOperationException( + "Unsupported schema change event: " + event); + } + + @Override + public void visit(AlterColumnTypeEvent event) { + throw new UnsupportedOperationException( + "Unsupported schema change event: " + event); + } + + @Override + public void visit(AlterTableCommentEvent event) { + throw new UnsupportedOperationException( + "Unsupported schema change event: " + event); + } + + @Override + public void visit(CreateTableEvent event) throws Exception { + applyCreateTableEvent(event); + } + + @Override + public void visit(DropColumnEvent event) throws Exception { + applyDropColumnEvent(event); + } + + @Override + public void visit(DropTableEvent event) { + throw new UnsupportedOperationException( + "Unsupported schema change event: " + event); + } + + @Override + public void visit(RenameColumnEvent event) throws Exception { + applyRenameColumnEvent(event); + } + + @Override + public void visit(RenameTableEvent event) { + throw new UnsupportedOperationException( + "Unsupported schema change event: " + event); + } + + @Override + public void visit(TruncateTableEvent event) { + throw new UnsupportedOperationException( + "Unsupported schema change event: " + event); + } + }); } private void applyCreateTableEvent(CreateTableEvent event) diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/test/java/org/apache/flink/cdc/connectors/doris/sink/DorisMetadataApplierITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/test/java/org/apache/flink/cdc/connectors/doris/sink/DorisMetadataApplierITCase.java index 0d3c0d99ceb..687d2e7f8bc 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/test/java/org/apache/flink/cdc/connectors/doris/sink/DorisMetadataApplierITCase.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris/src/test/java/org/apache/flink/cdc/connectors/doris/sink/DorisMetadataApplierITCase.java @@ -26,6 +26,7 @@ import org.apache.flink.cdc.common.event.DropColumnEvent; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.RenameColumnEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventTypeFamily; import org.apache.flink.cdc.common.event.TableId; import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; import org.apache.flink.cdc.common.schema.PhysicalColumn; @@ -55,6 +56,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.stream.Collectors; import static org.apache.flink.cdc.common.pipeline.PipelineOptions.DEFAULT_SCHEMA_OPERATOR_RPC_TIMEOUT; import static org.apache.flink.cdc.connectors.doris.sink.DorisDataSinkOptions.BENODES; @@ -426,7 +428,9 @@ private void runJobWithEvents(List events) throws Exception { schemaOperatorTranslator.translate( stream, DEFAULT_PARALLELISM, - dorisSink.getMetadataApplier(), + dorisSink.getMetadataApplier( + Arrays.stream(SchemaChangeEventTypeFamily.ALL) + .collect(Collectors.toSet())), new ArrayList<>()); DataSinkTranslator sinkTranslator = new DataSinkTranslator(); 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 78548e31b77..cd4e0e455fe 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/canal/CanalJsonSerializationSchema.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/canal/CanalJsonSerializationSchema.java @@ -18,7 +18,7 @@ package org.apache.flink.cdc.connectors.kafka.json.canal; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.event.DataChangeEvent; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.SchemaChangeEvent; @@ -42,6 +42,7 @@ import java.time.ZoneId; import java.util.HashMap; import java.util.Map; +import java.util.Optional; import static java.lang.String.format; import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; @@ -100,34 +101,37 @@ public void open(InitializationContext context) { @Override public byte[] serialize(Event event) { if (event instanceof SchemaChangeEvent) { - Schema schema; SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; - if (event instanceof CreateTableEvent) { - CreateTableEvent createTableEvent = (CreateTableEvent) event; - schema = createTableEvent.getSchema(); + Tuple2 appliedSchema = + SchemaUtils.applySchemaChangeEvent( + schemaChangeEvent, + Optional.ofNullable( + jsonSerializers.getOrDefault( + schemaChangeEvent.tableId(), null)) + .map(TableSchemaInfo::getSchema) + .orElse(null)); + if (appliedSchema.f1 != null) { + LogicalType rowType = + DataTypeUtils.toFlinkDataType(appliedSchema.f1.toRowDataType()) + .getLogicalType(); + JsonRowDataSerializationSchema jsonSerializer = + new JsonRowDataSerializationSchema( + createJsonRowType(fromLogicalToDataType(rowType)), + timestampFormat, + mapNullKeyMode, + mapNullKeyLiteral, + encodeDecimalAsPlainNumber); + try { + jsonSerializer.open(context); + } catch (Exception e) { + throw new RuntimeException(e); + } + jsonSerializers.put( + appliedSchema.f0, + new TableSchemaInfo(appliedSchema.f1, jsonSerializer, zoneId)); } else { - schema = - SchemaUtils.applySchemaChangeEvent( - jsonSerializers.get(schemaChangeEvent.tableId()).getSchema(), - schemaChangeEvent); + jsonSerializers.remove(appliedSchema.f0); } - LogicalType rowType = - DataTypeUtils.toFlinkDataType(schema.toRowDataType()).getLogicalType(); - JsonRowDataSerializationSchema jsonSerializer = - new JsonRowDataSerializationSchema( - createJsonRowType(fromLogicalToDataType(rowType)), - timestampFormat, - mapNullKeyMode, - mapNullKeyLiteral, - encodeDecimalAsPlainNumber); - try { - jsonSerializer.open(context); - } catch (Exception e) { - throw new RuntimeException(e); - } - jsonSerializers.put( - schemaChangeEvent.tableId(), - new TableSchemaInfo(schema, jsonSerializer, zoneId)); return null; } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/debezium/DebeziumJsonSerializationSchema.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/debezium/DebeziumJsonSerializationSchema.java index 2f305ce4258..181eab70bdb 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/debezium/DebeziumJsonSerializationSchema.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/json/debezium/DebeziumJsonSerializationSchema.java @@ -18,7 +18,7 @@ package org.apache.flink.cdc.connectors.kafka.json.debezium; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.event.DataChangeEvent; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.SchemaChangeEvent; @@ -40,6 +40,7 @@ import java.time.ZoneId; import java.util.HashMap; import java.util.Map; +import java.util.Optional; import static java.lang.String.format; import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; @@ -99,34 +100,38 @@ public void open(InitializationContext context) { @Override public byte[] serialize(Event event) { if (event instanceof SchemaChangeEvent) { - Schema schema; SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; - if (event instanceof CreateTableEvent) { - CreateTableEvent createTableEvent = (CreateTableEvent) event; - schema = createTableEvent.getSchema(); + Tuple2 appliedSchema = + SchemaUtils.applySchemaChangeEvent( + schemaChangeEvent, + Optional.ofNullable( + jsonSerializers.getOrDefault( + schemaChangeEvent.tableId(), null)) + .map(TableSchemaInfo::getSchema) + .orElse(null)); + + if (appliedSchema.f1 != null) { + LogicalType rowType = + DataTypeUtils.toFlinkDataType(appliedSchema.f1.toRowDataType()) + .getLogicalType(); + JsonRowDataSerializationSchema jsonSerializer = + new JsonRowDataSerializationSchema( + createJsonRowType(fromLogicalToDataType(rowType)), + timestampFormat, + mapNullKeyMode, + mapNullKeyLiteral, + encodeDecimalAsPlainNumber); + try { + jsonSerializer.open(context); + } catch (Exception e) { + throw new RuntimeException(e); + } + jsonSerializers.put( + appliedSchema.f0, + new TableSchemaInfo(appliedSchema.f1, jsonSerializer, zoneId)); } else { - schema = - SchemaUtils.applySchemaChangeEvent( - jsonSerializers.get(schemaChangeEvent.tableId()).getSchema(), - schemaChangeEvent); + jsonSerializers.remove(appliedSchema.f0, null); } - LogicalType rowType = - DataTypeUtils.toFlinkDataType(schema.toRowDataType()).getLogicalType(); - JsonRowDataSerializationSchema jsonSerializer = - new JsonRowDataSerializationSchema( - createJsonRowType(fromLogicalToDataType(rowType)), - timestampFormat, - mapNullKeyMode, - mapNullKeyLiteral, - encodeDecimalAsPlainNumber); - try { - jsonSerializer.open(context); - } catch (Exception e) { - throw new RuntimeException(e); - } - jsonSerializers.put( - schemaChangeEvent.tableId(), - new TableSchemaInfo(schema, jsonSerializer, zoneId)); return null; } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSink.java index 947bde1a8a1..0248f4edf99 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSink.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/main/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSink.java @@ -19,6 +19,9 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; +import org.apache.flink.cdc.common.event.SchemaChangeEventTypeFamily; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.common.sink.EventSinkProvider; import org.apache.flink.cdc.common.sink.FlinkSinkProvider; @@ -31,7 +34,10 @@ import org.apache.kafka.clients.producer.ProducerConfig; import java.time.ZoneId; +import java.util.Arrays; import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; /** A {@link DataSink} for "Kafka" connector. */ public class KafkaDataSink implements DataSink { @@ -93,8 +99,23 @@ public EventSinkProvider getEventSinkProvider() { } @Override - public MetadataApplier getMetadataApplier() { - // simply do nothing here because Kafka do not maintain the schemas. - return schemaChangeEvent -> {}; + public MetadataApplier getMetadataApplier(Set enabledEventTypes) { + return new MetadataApplier() { + @Override + public boolean acceptsSchemaEvolutionType(SchemaChangeEventType schemaChangeEventType) { + return enabledEventTypes.contains(schemaChangeEventType); + } + + @Override + public Set getSupportedSchemaEvolutionTypes() { + // All schema change events are supported. + return Arrays.stream(SchemaChangeEventTypeFamily.ALL).collect(Collectors.toSet()); + } + + @Override + public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) { + // simply do nothing here because Kafka do not maintain the schemas. + } + }; } } 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/PaimonDataSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/PaimonDataSink.java index 6c94a3a74e5..a1acacf433d 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/PaimonDataSink.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/PaimonDataSink.java @@ -18,6 +18,7 @@ package org.apache.flink.cdc.connectors.paimon.sink; import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; import org.apache.flink.cdc.common.event.TableId; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.common.sink.EventSinkProvider; @@ -31,6 +32,7 @@ import java.io.Serializable; import java.util.List; import java.util.Map; +import java.util.Set; /** A {@link DataSink} for Paimon connector that supports schema evolution. */ public class PaimonDataSink implements DataSink, Serializable { @@ -66,7 +68,7 @@ public EventSinkProvider getEventSinkProvider() { } @Override - public MetadataApplier getMetadataApplier() { - return new PaimonMetadataApplier(options, tableOptions, partitionMaps); + public MetadataApplier getMetadataApplier(Set enabledEventTypes) { + return new PaimonMetadataApplier(options, tableOptions, partitionMaps, enabledEventTypes); } } 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 f71e9da1ab8..4c0e7f0f5ca 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,16 +18,25 @@ package org.apache.flink.cdc.connectors.paimon.sink; import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnCommentEvent; import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; +import org.apache.flink.cdc.common.event.AlterTableCommentEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.DropTableEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; +import org.apache.flink.cdc.common.event.RenameTableEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; +import org.apache.flink.cdc.common.event.SchemaChangeEventVisitorVoid; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.sink.MetadataApplier; import org.apache.flink.cdc.common.types.utils.DataTypeUtils; +import org.apache.flink.shaded.guava31.com.google.common.collect.Sets; + import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.FlinkCatalogFactory; @@ -35,11 +44,14 @@ import org.apache.paimon.options.Options; import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.table.Table; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import static org.apache.flink.cdc.common.utils.Preconditions.checkArgument; import static org.apache.flink.cdc.common.utils.Preconditions.checkNotNull; @@ -50,6 +62,8 @@ */ public class PaimonMetadataApplier implements MetadataApplier { + private static final Logger LOG = LoggerFactory.getLogger(PaimonMetadataApplier.class); + // Catalog is unSerializable. private transient Catalog catalog; @@ -60,10 +74,13 @@ public class PaimonMetadataApplier implements MetadataApplier { private final Map> partitionMaps; + private final Set enabledSchemaEvolutionTypes; + public PaimonMetadataApplier(Options catalogOptions) { this.catalogOptions = catalogOptions; this.tableOptions = new HashMap<>(); this.partitionMaps = new HashMap<>(); + this.enabledSchemaEvolutionTypes = getSupportedSchemaEvolutionTypes(); } public PaimonMetadataApplier( @@ -73,6 +90,33 @@ public PaimonMetadataApplier( this.catalogOptions = catalogOptions; this.tableOptions = tableOptions; this.partitionMaps = partitionMaps; + this.enabledSchemaEvolutionTypes = getSupportedSchemaEvolutionTypes(); + } + + public PaimonMetadataApplier( + Options catalogOptions, + Map tableOptions, + Map> partitionMaps, + Set enabledSchemaEvolutionTypes) { + this.catalogOptions = catalogOptions; + this.tableOptions = tableOptions; + this.partitionMaps = partitionMaps; + this.enabledSchemaEvolutionTypes = enabledSchemaEvolutionTypes; + } + + @Override + public boolean acceptsSchemaEvolutionType(SchemaChangeEventType schemaChangeEventType) { + return enabledSchemaEvolutionTypes.contains(schemaChangeEventType); + } + + @Override + public Set getSupportedSchemaEvolutionTypes() { + return Sets.newHashSet( + SchemaChangeEventType.CREATE_TABLE, + SchemaChangeEventType.ADD_COLUMN, + SchemaChangeEventType.DROP_COLUMN, + SchemaChangeEventType.RENAME_COLUMN, + SchemaChangeEventType.ALTER_COLUMN_TYPE); } @Override @@ -80,24 +124,69 @@ public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) { if (catalog == null) { catalog = FlinkCatalogFactory.createPaimonCatalog(catalogOptions); } - try { - if (schemaChangeEvent instanceof CreateTableEvent) { - applyCreateTable((CreateTableEvent) schemaChangeEvent); - } else if (schemaChangeEvent instanceof AddColumnEvent) { - applyAddColumn((AddColumnEvent) schemaChangeEvent); - } else if (schemaChangeEvent instanceof DropColumnEvent) { - applyDropColumn((DropColumnEvent) schemaChangeEvent); - } else if (schemaChangeEvent instanceof RenameColumnEvent) { - applyRenameColumn((RenameColumnEvent) schemaChangeEvent); - } else if (schemaChangeEvent instanceof AlterColumnTypeEvent) { - applyAlterColumn((AlterColumnTypeEvent) schemaChangeEvent); - } else { - throw new UnsupportedOperationException( - "PaimonDataSink doesn't support schema change event " + schemaChangeEvent); - } - } catch (Exception e) { - throw new RuntimeException(e); - } + schemaChangeEvent.visit( + new SchemaChangeEventVisitorVoid() { + + @Override + public void visit(AddColumnEvent event) throws Exception { + applyAddColumn(event); + } + + @Override + public void visit(AlterColumnCommentEvent event) throws Exception { + throw new UnsupportedOperationException( + "PaimonDataSink doesn't support schema change event " + + schemaChangeEvent); + } + + @Override + public void visit(AlterColumnTypeEvent event) throws Exception { + applyAlterColumnType(event); + } + + @Override + public void visit(AlterTableCommentEvent event) throws Exception { + throw new UnsupportedOperationException( + "PaimonDataSink doesn't support schema change event " + + schemaChangeEvent); + } + + @Override + public void visit(CreateTableEvent event) throws Exception { + applyCreateTable(event); + } + + @Override + public void visit(DropColumnEvent event) throws Exception { + applyDropColumn(event); + } + + @Override + public void visit(DropTableEvent event) throws Exception { + throw new UnsupportedOperationException( + "PaimonDataSink doesn't support schema change event " + + schemaChangeEvent); + } + + @Override + public void visit(RenameColumnEvent event) throws Exception { + applyRenameColumn(event); + } + + @Override + public void visit(RenameTableEvent event) throws Exception { + throw new UnsupportedOperationException( + "PaimonDataSink doesn't support schema change event " + + schemaChangeEvent); + } + + @Override + public void visit(TruncateTableEvent event) throws Exception { + throw new UnsupportedOperationException( + "PaimonDataSink doesn't support schema change event " + + schemaChangeEvent); + } + }); } private void applyCreateTable(CreateTableEvent event) @@ -237,7 +326,7 @@ private void applyRenameColumn(RenameColumnEvent event) true); } - private void applyAlterColumn(AlterColumnTypeEvent event) + private void applyAlterColumnType(AlterColumnTypeEvent event) throws Catalog.ColumnAlreadyExistException, Catalog.TableNotExistException, Catalog.ColumnNotExistException { List tableChangeList = new ArrayList<>(); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonRecordEventSerializer.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonRecordEventSerializer.java index 53b63f3b599..95e90038d05 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonRecordEventSerializer.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonRecordEventSerializer.java @@ -17,12 +17,13 @@ package org.apache.flink.cdc.connectors.paimon.sink.v2; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.event.ChangeEvent; -import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DataChangeEvent; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.utils.SchemaUtils; import org.apache.paimon.catalog.Identifier; @@ -31,6 +32,7 @@ import java.time.ZoneId; import java.util.HashMap; import java.util.Map; +import java.util.Optional; /** * A {@link PaimonRecordSerializer} for converting {@link Event} into {@link PaimonEvent} for {@link @@ -56,20 +58,19 @@ public PaimonEvent serialize(Event event) { ((ChangeEvent) event).tableId().getSchemaName(), ((ChangeEvent) event).tableId().getTableName()); if (event instanceof SchemaChangeEvent) { - if (event instanceof CreateTableEvent) { - CreateTableEvent createTableEvent = (CreateTableEvent) event; - schemaMaps.put( - createTableEvent.tableId(), - new TableSchemaInfo(createTableEvent.getSchema(), zoneId)); + SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; + Tuple2 appliedSchema = + SchemaUtils.applySchemaChangeEvent( + schemaChangeEvent, + Optional.ofNullable( + schemaMaps.getOrDefault( + schemaChangeEvent.tableId(), null)) + .map(TableSchemaInfo::getSchema) + .orElse(null)); + if (appliedSchema.f1 != null) { + schemaMaps.put(appliedSchema.f0, new TableSchemaInfo(appliedSchema.f1, zoneId)); } else { - SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; - schemaMaps.put( - schemaChangeEvent.tableId(), - new TableSchemaInfo( - SchemaUtils.applySchemaChangeEvent( - schemaMaps.get(schemaChangeEvent.tableId()).getSchema(), - schemaChangeEvent), - zoneId)); + schemaMaps.remove(appliedSchema.f0, null); } return new PaimonEvent(tableId, null, true); } else if (event instanceof DataChangeEvent) { diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/EventRecordSerializationSchema.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/EventRecordSerializationSchema.java index 76b55aafc98..65ce038c2ad 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 @@ -18,8 +18,8 @@ package org.apache.flink.cdc.connectors.starrocks.sink; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.java.tuple.Tuple2; 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.Event; import org.apache.flink.cdc.common.event.SchemaChangeEvent; @@ -39,6 +39,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; /** Serializer for the input {@link Event}. It will serialize a row to a json string. */ public class EventRecordSerializationSchema implements RecordSerializationSchema { @@ -81,26 +82,25 @@ public StarRocksRowData serialize(Event record) { } private void applySchemaChangeEvent(SchemaChangeEvent event) { - TableId tableId = event.tableId(); - Schema newSchema; - if (event instanceof CreateTableEvent) { - newSchema = ((CreateTableEvent) event).getSchema(); - } else { - TableInfo tableInfo = tableInfoMap.get(tableId); - if (tableInfo == null) { - throw new RuntimeException("schema of " + tableId + " is not existed."); + Tuple2 appliedSchema = + SchemaUtils.applySchemaChangeEvent( + event, + Optional.ofNullable(tableInfoMap.getOrDefault(event.tableId(), null)) + .map(e -> e.schema) + .orElse(null)); + if (appliedSchema.f1 != null) { + TableInfo tableInfo = new TableInfo(); + tableInfo.schema = appliedSchema.f1; + tableInfo.fieldGetters = new RecordData.FieldGetter[appliedSchema.f1.getColumnCount()]; + for (int i = 0; i < appliedSchema.f1.getColumnCount(); i++) { + tableInfo.fieldGetters[i] = + StarRocksUtils.createFieldGetter( + appliedSchema.f1.getColumns().get(i).getType(), i, zoneId); } - newSchema = SchemaUtils.applySchemaChangeEvent(tableInfo.schema, event); - } - TableInfo tableInfo = new TableInfo(); - tableInfo.schema = newSchema; - tableInfo.fieldGetters = new RecordData.FieldGetter[newSchema.getColumnCount()]; - for (int i = 0; i < newSchema.getColumnCount(); i++) { - tableInfo.fieldGetters[i] = - StarRocksUtils.createFieldGetter( - newSchema.getColumns().get(i).getType(), i, zoneId); + tableInfoMap.put(appliedSchema.f0, tableInfo); + } else { + tableInfoMap.remove(appliedSchema.f0); } - tableInfoMap.put(tableId, tableInfo); } private StarRocksRowData applyDataChangeEvent(DataChangeEvent event) { diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksDataSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksDataSink.java index 4c1969ae90c..793ee2f166a 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksDataSink.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksDataSink.java @@ -18,6 +18,7 @@ package org.apache.flink.cdc.connectors.starrocks.sink; import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.common.sink.EventSinkProvider; import org.apache.flink.cdc.common.sink.FlinkSinkProvider; @@ -30,6 +31,7 @@ import java.io.Serializable; import java.time.ZoneId; +import java.util.Set; /** A {@link DataSink} for StarRocks connector that supports schema evolution. */ public class StarRocksDataSink implements DataSink, Serializable { @@ -70,12 +72,13 @@ public EventSinkProvider getEventSinkProvider() { } @Override - public MetadataApplier getMetadataApplier() { + public MetadataApplier getMetadataApplier(Set enabledEventTypes) { StarRocksCatalog catalog = new StarRocksCatalog( sinkOptions.getJdbcUrl(), sinkOptions.getUsername(), sinkOptions.getPassword()); - return new StarRocksMetadataApplier(catalog, tableCreateConfig, schemaChangeConfig); + return new StarRocksMetadataApplier( + catalog, tableCreateConfig, schemaChangeConfig, enabledEventTypes); } } 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..50bc56abfed 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksMetadataApplier.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/main/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksMetadataApplier.java @@ -18,15 +18,24 @@ package org.apache.flink.cdc.connectors.starrocks.sink; import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnCommentEvent; import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; +import org.apache.flink.cdc.common.event.AlterTableCommentEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.DropTableEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; +import org.apache.flink.cdc.common.event.RenameTableEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; +import org.apache.flink.cdc.common.event.SchemaChangeEventVisitorVoid; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.sink.MetadataApplier; +import org.apache.flink.shaded.guava31.com.google.common.collect.Sets; + import com.starrocks.connector.flink.catalog.StarRocksCatalog; import com.starrocks.connector.flink.catalog.StarRocksCatalogException; import com.starrocks.connector.flink.catalog.StarRocksColumn; @@ -36,6 +45,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Set; /** A {@code MetadataApplier} that applies metadata changes to StarRocks. */ public class StarRocksMetadataApplier implements MetadataApplier { @@ -48,6 +58,7 @@ public class StarRocksMetadataApplier implements MetadataApplier { private final TableCreateConfig tableCreateConfig; private final SchemaChangeConfig schemaChangeConfig; private boolean isOpened; + private final Set enabledSchemaEvolutionTypes; public StarRocksMetadataApplier( StarRocksCatalog catalog, @@ -57,6 +68,32 @@ public StarRocksMetadataApplier( this.tableCreateConfig = tableCreateConfig; this.schemaChangeConfig = schemaChangeConfig; this.isOpened = false; + this.enabledSchemaEvolutionTypes = getSupportedSchemaEvolutionTypes(); + } + + public StarRocksMetadataApplier( + StarRocksCatalog catalog, + TableCreateConfig tableCreateConfig, + SchemaChangeConfig schemaChangeConfig, + Set enabledSchemaEvolutionTypes) { + this.catalog = catalog; + this.tableCreateConfig = tableCreateConfig; + this.schemaChangeConfig = schemaChangeConfig; + this.isOpened = false; + this.enabledSchemaEvolutionTypes = enabledSchemaEvolutionTypes; + } + + @Override + public boolean acceptsSchemaEvolutionType(SchemaChangeEventType schemaChangeEventType) { + return enabledSchemaEvolutionTypes.contains(schemaChangeEventType); + } + + @Override + public Set getSupportedSchemaEvolutionTypes() { + return Sets.newHashSet( + SchemaChangeEventType.CREATE_TABLE, + SchemaChangeEventType.ADD_COLUMN, + SchemaChangeEventType.DROP_COLUMN); } @Override @@ -66,20 +103,69 @@ 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.visit( + new SchemaChangeEventVisitorVoid() { + + @Override + public void visit(AddColumnEvent event) { + applyAddColumn(event); + } + + @Override + public void visit(AlterColumnCommentEvent event) { + throw new UnsupportedOperationException( + "StarRocksDataSink doesn't support schema change event " + + schemaChangeEvent); + } + + @Override + public void visit(AlterColumnTypeEvent event) { + applyAlterColumnType(event); + } + + @Override + public void visit(AlterTableCommentEvent event) { + throw new UnsupportedOperationException( + "StarRocksDataSink doesn't support schema change event " + + schemaChangeEvent); + } + + @Override + public void visit(CreateTableEvent event) { + applyCreateTable(event); + } + + @Override + public void visit(DropColumnEvent event) { + applyDropColumn(event); + } + + @Override + public void visit(DropTableEvent event) { + throw new UnsupportedOperationException( + "StarRocksDataSink doesn't support schema change event " + + schemaChangeEvent); + } + + @Override + public void visit(RenameColumnEvent event) { + applyRenameColumn(event); + } + + @Override + public void visit(RenameTableEvent event) { + throw new UnsupportedOperationException( + "StarRocksDataSink doesn't support schema change event " + + schemaChangeEvent); + } + + @Override + public void visit(TruncateTableEvent event) { + throw new UnsupportedOperationException( + "StarRocksDataSink doesn't support schema change event " + + schemaChangeEvent); + } + }); } private void applyCreateTable(CreateTableEvent createTableEvent) { @@ -257,7 +343,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..e4539d4c58f 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.applySchemaChangeEvent(addColumnEvent, schema1).f1; 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.applySchemaChangeEvent(dropColumnEvent, schema2).f1; BinaryRecordDataGenerator newGenerator2 = new BinaryRecordDataGenerator( newSchema2.getColumnDataTypes().toArray(new DataType[0])); 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/StarRocksMetadataApplierITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/test/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksMetadataApplierITCase.java index c294dd42338..ed0d19b8e13 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/test/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksMetadataApplierITCase.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/test/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksMetadataApplierITCase.java @@ -26,6 +26,7 @@ import org.apache.flink.cdc.common.event.DropColumnEvent; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.RenameColumnEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventTypeFamily; import org.apache.flink.cdc.common.event.TableId; import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; import org.apache.flink.cdc.common.schema.PhysicalColumn; @@ -52,6 +53,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.stream.Collectors; import static org.apache.flink.cdc.common.pipeline.PipelineOptions.DEFAULT_SCHEMA_OPERATOR_RPC_TIMEOUT; import static org.apache.flink.cdc.connectors.starrocks.sink.StarRocksDataSinkOptions.JDBC_URL; @@ -373,7 +375,9 @@ private void runJobWithEvents(List events) throws Exception { schemaOperatorTranslator.translate( stream, DEFAULT_PARALLELISM, - starRocksSink.getMetadataApplier(), + starRocksSink.getMetadataApplier( + Arrays.stream(SchemaChangeEventTypeFamily.ALL) + .collect(Collectors.toSet())), new ArrayList<>()); DataSinkTranslator sinkTranslator = new DataSinkTranslator(); 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..da208f4e00a 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,18 @@ 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.SchemaChangeEventType; 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; @@ -35,12 +40,15 @@ import org.apache.flink.cdc.connectors.values.sink.ValuesDataSink; import org.apache.flink.cdc.connectors.values.source.ValuesDataSource; +import org.apache.flink.shaded.guava31.com.google.common.collect.Sets; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; @@ -69,12 +77,81 @@ public class ValuesDatabase { */ public static class ValuesMetadataApplier implements MetadataApplier { + private final Set enabledSchemaEvolutionTypes; + + public ValuesMetadataApplier() { + this.enabledSchemaEvolutionTypes = getSupportedSchemaEvolutionTypes(); + } + + public ValuesMetadataApplier(Set types) { + this.enabledSchemaEvolutionTypes = types; + } + + @Override + public boolean acceptsSchemaEvolutionType(SchemaChangeEventType schemaChangeEventType) { + return enabledSchemaEvolutionTypes.contains(schemaChangeEventType); + } + + @Override + public Set getSupportedSchemaEvolutionTypes() { + return Sets.newHashSet( + SchemaChangeEventType.ADD_COLUMN, + SchemaChangeEventType.ALTER_COLUMN_TYPE, + SchemaChangeEventType.CREATE_TABLE, + SchemaChangeEventType.DROP_COLUMN, + SchemaChangeEventType.RENAME_COLUMN); + } + @Override public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) { applySchemaChangeEvent(schemaChangeEvent); } } + /** + * apply SchemaChangeEvent to ValuesDatabase and print it out, throw exception if illegal + * changes occur. + */ + public static class ErrorOnChangeMetadataApplier implements MetadataApplier { + private final Set enabledSchemaEvolutionTypes; + + public ErrorOnChangeMetadataApplier() { + enabledSchemaEvolutionTypes = getSupportedSchemaEvolutionTypes(); + } + + public ErrorOnChangeMetadataApplier(Set types) { + enabledSchemaEvolutionTypes = types; + } + + @Override + public boolean acceptsSchemaEvolutionType(SchemaChangeEventType schemaChangeEventType) { + return enabledSchemaEvolutionTypes.contains(schemaChangeEventType); + } + + @Override + public Set getSupportedSchemaEvolutionTypes() { + return Collections.singleton(SchemaChangeEventType.CREATE_TABLE); + } + + @Override + public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) { + if (schemaChangeEvent instanceof CreateTableEvent) { + TableId tableId = schemaChangeEvent.tableId(); + if (!globalTables.containsKey(tableId)) { + globalTables.put( + tableId, + new ValuesTable( + tableId, ((CreateTableEvent) schemaChangeEvent).getSchema())); + } + } else { + throw new RuntimeException( + String.format( + "Rejected schema change event %s since error.on.schema.change is enabled.", + schemaChangeEvent)); + } + } + } + /** provide namespace/schema/table lists for {@link ValuesDataSource}. */ public static class ValuesMetadataAccessor implements MetadataAccessor { @@ -140,6 +217,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 +238,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"); @@ -318,6 +418,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 +518,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/factory/ValuesDataFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/factory/ValuesDataFactory.java index ee8411d2baf..671b909eb04 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/factory/ValuesDataFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/factory/ValuesDataFactory.java @@ -54,7 +54,9 @@ public DataSink createDataSink(Context context) { return new ValuesDataSink( context.getFactoryConfiguration().get(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY), context.getFactoryConfiguration().get(ValuesDataSinkOptions.PRINT_ENABLED), - context.getFactoryConfiguration().get(ValuesDataSinkOptions.SINK_API)); + context.getFactoryConfiguration().get(ValuesDataSinkOptions.SINK_API), + context.getFactoryConfiguration() + .get(ValuesDataSinkOptions.ERROR_ON_SCHEMA_CHANGE)); } @Override @@ -73,6 +75,7 @@ public Set> optionalOptions() { options.add(ValuesDataSourceOptions.EVENT_SET_ID); options.add(ValuesDataSourceOptions.FAILURE_INJECTION_INDEX); options.add(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY); + options.add(ValuesDataSinkOptions.ERROR_ON_SCHEMA_CHANGE); return options; } } 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..e32ab475e2f 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 @@ -19,13 +19,14 @@ import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.annotation.Internal; import org.apache.flink.cdc.common.data.RecordData; import org.apache.flink.cdc.common.event.ChangeEvent; -import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DataChangeEvent; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; import org.apache.flink.cdc.common.event.TableId; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.sink.DataSink; @@ -40,6 +41,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; /** A {@link DataSink} for "values" connector that supports schema evolution. */ @Internal @@ -52,10 +54,17 @@ public class ValuesDataSink implements DataSink, Serializable { private final SinkApi sinkApi; - public ValuesDataSink(boolean materializedInMemory, boolean print, SinkApi sinkApi) { + private final boolean errorOnSchemaChange; + + public ValuesDataSink( + boolean materializedInMemory, + boolean print, + SinkApi sinkApi, + boolean errorOnSchemaChange) { this.materializedInMemory = materializedInMemory; this.print = print; this.sinkApi = sinkApi; + this.errorOnSchemaChange = errorOnSchemaChange; } @Override @@ -69,8 +78,12 @@ public EventSinkProvider getEventSinkProvider() { } @Override - public MetadataApplier getMetadataApplier() { - return new ValuesDatabase.ValuesMetadataApplier(); + public MetadataApplier getMetadataApplier(Set enabledEventTypes) { + if (errorOnSchemaChange) { + return new ValuesDatabase.ErrorOnChangeMetadataApplier(enabledEventTypes); + } else { + return new ValuesDatabase.ValuesMetadataApplier(enabledEventTypes); + } } /** an e2e {@link Sink} implementation that print all {@link DataChangeEvent} out. */ @@ -131,20 +144,18 @@ public ValuesSinkWriter( public void write(Event event, Context context) { if (event instanceof SchemaChangeEvent) { SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; - TableId tableId = schemaChangeEvent.tableId(); - if (event instanceof CreateTableEvent) { - Schema schema = ((CreateTableEvent) event).getSchema(); - schemaMaps.put(tableId, schema); - fieldGetterMaps.put(tableId, SchemaUtils.createFieldGetters(schema)); + Tuple2 appliedSchema = + SchemaUtils.applySchemaChangeEvent( + schemaChangeEvent, + schemaMaps.getOrDefault(schemaChangeEvent.tableId(), null)); + + if (appliedSchema.f1 != null) { + schemaMaps.put(appliedSchema.f0, appliedSchema.f1); + fieldGetterMaps.put( + appliedSchema.f0, SchemaUtils.createFieldGetters(appliedSchema.f1)); } else { - if (!schemaMaps.containsKey(tableId)) { - throw new RuntimeException("schema of " + tableId + " is not existed."); - } - Schema schema = - SchemaUtils.applySchemaChangeEvent( - schemaMaps.get(tableId), schemaChangeEvent); - schemaMaps.put(tableId, schema); - fieldGetterMaps.put(tableId, SchemaUtils.createFieldGetters(schema)); + schemaMaps.remove(appliedSchema.f0); + fieldGetterMaps.remove(appliedSchema.f0); } } else if (materializedInMemory && event instanceof DataChangeEvent) { ValuesDatabase.applyDataChangeEvent((DataChangeEvent) event); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/sink/ValuesDataSinkFunction.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/sink/ValuesDataSinkFunction.java index f4876d5e04a..bb367efbbf1 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 @@ -18,9 +18,9 @@ package org.apache.flink.cdc.connectors.values.sink; import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.data.RecordData; import org.apache.flink.cdc.common.event.ChangeEvent; -import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DataChangeEvent; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.SchemaChangeEvent; @@ -59,20 +59,18 @@ public ValuesDataSinkFunction(boolean materializedInMemory, boolean print) { public void invoke(Event event, Context context) throws Exception { if (event instanceof SchemaChangeEvent) { SchemaChangeEvent schemaChangeEvent = (SchemaChangeEvent) event; - TableId tableId = schemaChangeEvent.tableId(); - if (event instanceof CreateTableEvent) { - Schema schema = ((CreateTableEvent) event).getSchema(); - schemaMaps.put(tableId, schema); - fieldGetterMaps.put(tableId, SchemaUtils.createFieldGetters(schema)); + Tuple2 appliedSchema = + SchemaUtils.applySchemaChangeEvent( + schemaChangeEvent, + schemaMaps.getOrDefault(schemaChangeEvent.tableId(), null)); + + if (appliedSchema.f1 != null) { + schemaMaps.put(appliedSchema.f0, appliedSchema.f1); + fieldGetterMaps.put( + appliedSchema.f0, SchemaUtils.createFieldGetters(appliedSchema.f1)); } else { - if (!schemaMaps.containsKey(tableId)) { - throw new RuntimeException("schema of " + tableId + " is not existed."); - } - Schema schema = - SchemaUtils.applySchemaChangeEvent( - schemaMaps.get(tableId), schemaChangeEvent); - schemaMaps.put(tableId, schema); - fieldGetterMaps.put(tableId, SchemaUtils.createFieldGetters(schema)); + schemaMaps.remove(appliedSchema.f0); + fieldGetterMaps.remove(appliedSchema.f0); } } else if (materializedInMemory && event instanceof DataChangeEvent) { ValuesDatabase.applyDataChangeEvent((DataChangeEvent) event); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/sink/ValuesDataSinkOptions.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/sink/ValuesDataSinkOptions.java index 11b132a2da8..4830d102be0 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/sink/ValuesDataSinkOptions.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/src/main/java/org/apache/flink/cdc/connectors/values/sink/ValuesDataSinkOptions.java @@ -42,4 +42,11 @@ public class ValuesDataSinkOptions { .defaultValue(ValuesDataSink.SinkApi.SINK_V2) .withDescription( "The sink api on which the sink is based: SinkFunction or SinkV2."); + + public static final ConfigOption ERROR_ON_SCHEMA_CHANGE = + ConfigOptions.key("error.on.schema.change") + .booleanType() + .defaultValue(false) + .withDescription( + "True if a runtime error should be thrown when handling schema change events."); } 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 28db063ef87..4898d423165 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 @@ -55,6 +55,7 @@ public class MysqlE2eITCase extends PipelineTestEnvironment { protected static final String MYSQL_TEST_PASSWORD = "mysqlpw"; protected static final String MYSQL_DRIVER_CLASS = "com.mysql.cj.jdbc.Driver"; protected static final String INTER_CONTAINER_MYSQL_ALIAS = "mysql"; + protected static final long EVENT_WAITING_TIMEOUT = 60000L; @ClassRule public static final MySqlContainer MYSQL = @@ -117,13 +118,11 @@ public void testSyncWholeDatabase() throws Exception { waitUntilSpecificEvent( String.format( "DataChangeEvent{tableId=%s.customers, before=[], after=[104, user_4, Shanghai, 123567891234], op=INSERT, meta=()}", - mysqlInventoryDatabase.getDatabaseName()), - 60000L); + mysqlInventoryDatabase.getDatabaseName())); waitUntilSpecificEvent( 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); + mysqlInventoryDatabase.getDatabaseName())); List expectedEvents = Arrays.asList( String.format( @@ -191,17 +190,22 @@ public void testSyncWholeDatabase() throws Exception { waitUntilSpecificEvent( String.format( "DataChangeEvent{tableId=%s.products, before=[106, hammer, 16oz carpenter's hammer, 1.0, null, null, null], after=[106, hammer, 18oz carpenter hammer, 1.0, null, null, null], op=UPDATE, meta=()}", - mysqlInventoryDatabase.getDatabaseName()), - 20000L); + mysqlInventoryDatabase.getDatabaseName())); // modify table schema stat.execute("ALTER TABLE products ADD COLUMN new_col INT;"); 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) { @@ -212,8 +216,7 @@ public void testSyncWholeDatabase() throws Exception { waitUntilSpecificEvent( 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); + mysqlInventoryDatabase.getDatabaseName())); expectedEvents = Arrays.asList( @@ -230,29 +233,47 @@ 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) throws Exception { for (String event : expectedEvents) { - waitUntilSpecificEvent(event, 6000L); + waitUntilSpecificEvent(event); } } - private void waitUntilSpecificEvent(String event, long timeout) throws Exception { + private void waitUntilSpecificEvent(String event) throws Exception { boolean result = false; - long endTimeout = System.currentTimeMillis() + timeout; + long endTimeout = System.currentTimeMillis() + MysqlE2eITCase.EVENT_WAITING_TIMEOUT; while (System.currentTimeMillis() < endTimeout) { String stdout = taskManagerConsumer.toUtf8String(); if (stdout.contains(event)) { diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/SchemaEvolveE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/SchemaEvolveE2eITCase.java new file mode 100644 index 00000000000..d4c8a91a8ce --- /dev/null +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/SchemaEvolveE2eITCase.java @@ -0,0 +1,682 @@ +/* + * 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.pipeline.tests; + +import org.apache.flink.cdc.common.test.utils.TestUtils; +import org.apache.flink.cdc.connectors.mysql.testutils.MySqlContainer; +import org.apache.flink.cdc.connectors.mysql.testutils.MySqlVersion; +import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase; +import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.containers.output.ToStringConsumer; + +import java.nio.file.Path; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.Statement; +import java.time.Duration; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** E2e tests for Schema Evolution cases. */ +public class SchemaEvolveE2eITCase extends PipelineTestEnvironment { + private static final Logger LOG = LoggerFactory.getLogger(SchemaEvolveE2eITCase.class); + + // ------------------------------------------------------------------------------------------ + // MySQL Variables (we always use MySQL as the data source for easier verifying) + // ------------------------------------------------------------------------------------------ + protected static final String MYSQL_TEST_USER = "mysqluser"; + protected static final String MYSQL_TEST_PASSWORD = "mysqlpw"; + protected static final String MYSQL_DRIVER_CLASS = "com.mysql.cj.jdbc.Driver"; + protected static final String INTER_CONTAINER_MYSQL_ALIAS = "mysql"; + protected static final long EVENT_WAITING_TIMEOUT = 60000L; + + @ClassRule + public static final MySqlContainer MYSQL = + (MySqlContainer) + new MySqlContainer( + MySqlVersion.V8_0) // v8 support both ARM and AMD architectures + .withConfigurationOverride("docker/mysql/my.cnf") + .withSetupSQL("docker/mysql/setup.sql") + .withDatabaseName("flink-test") + .withUsername("flinkuser") + .withPassword("flinkpw") + .withNetwork(NETWORK) + .withNetworkAliases(INTER_CONTAINER_MYSQL_ALIAS) + .withLogConsumer(new Slf4jLogConsumer(LOG)); + + protected final UniqueDatabase schemaEvolveDatabase = + new UniqueDatabase(MYSQL, "schema_evolve", MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + + @Before + public void before() throws Exception { + super.before(); + schemaEvolveDatabase.createAndInitialize(); + } + + @After + public void after() { + super.after(); + schemaEvolveDatabase.dropDatabase(); + } + + private void validateSnapshotData(String tableName) throws Exception { + List expected = + Stream.of( + "CreateTableEvent{tableId=%s.%s, schema=columns={`id` INT NOT NULL,`name` VARCHAR(17),`age` INT}, primaryKeys=id, options=()}", + "DataChangeEvent{tableId=%s.%s, before=[], after=[1008, Alice, 21], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.%s, before=[], after=[1009, Bob, 20], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.%s, before=[], after=[1010, Carol, 19], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.%s, before=[], after=[1011, Derrida, 18], op=INSERT, meta=()}") + .map( + s -> + String.format( + s, + schemaEvolveDatabase.getDatabaseName(), + tableName)) + .collect(Collectors.toList()); + + validateResult(expected, taskManagerConsumer); + } + + private void waitForIncrementalStage(String tableName, Statement stmt) throws Exception { + stmt.execute("INSERT INTO members VALUES (0, '__fence__', 0);"); + + // Ensure we change schema after incremental stage + waitUntilSpecificEvent( + String.format( + "DataChangeEvent{tableId=%s.%s, before=[], after=[0, __fence__, 0], op=INSERT, meta=()}", + schemaEvolveDatabase.getDatabaseName(), tableName), + taskManagerConsumer); + } + + @Test + public void testSchemaEvolve() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: %s\n" + + " port: 3306\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.members\n" + + " server-id: 5400-5404\n" + + " server-time-zone: UTC\n" + + "\n" + + "sink:\n" + + " type: values\n" + + "\n" + + "pipeline:\n" + + " schema.change.behavior: evolve\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + schemaEvolveDatabase.getDatabaseName()); + Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); + Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); + Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); + submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + waitUntilJobRunning(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + validateSnapshotData("members"); + + LOG.info("Starting schema evolution"); + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), + MYSQL.getDatabasePort(), + schemaEvolveDatabase.getDatabaseName()); + + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stmt = conn.createStatement()) { + + waitForIncrementalStage("members", stmt); + + // triggers AddColumnEvent + stmt.execute("ALTER TABLE members ADD COLUMN gender TINYINT AFTER age;"); + stmt.execute("INSERT INTO members VALUES (1012, 'Eve', 17, 0);"); + + // triggers AlterColumnTypeEvent and RenameColumnEvent + stmt.execute("ALTER TABLE members CHANGE COLUMN age precise_age DOUBLE;"); + + // triggers RenameColumnEvent + stmt.execute("ALTER TABLE members RENAME COLUMN gender TO biological_sex;"); + + // triggers DropColumnEvent + stmt.execute("ALTER TABLE members DROP COLUMN biological_sex"); + stmt.execute("INSERT INTO members VALUES (1013, 'Fiona', 16);"); + } + + List expected = + Stream.of( + "AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`gender` TINYINT, position=AFTER, existedColumnName=age}]}", + "DataChangeEvent{tableId=%s.members, before=[], after=[1012, Eve, 17, 0], op=INSERT, meta=()}", + "AlterColumnTypeEvent{tableId=%s.members, typeMapping={age=DOUBLE}, oldTypeMapping={age=INT}}", + "RenameColumnEvent{tableId=%s.members, nameMapping={age=precise_age}}", + "RenameColumnEvent{tableId=%s.members, nameMapping={gender=biological_sex}}", + "DropColumnEvent{tableId=%s.members, droppedColumnNames=[biological_sex]}", + "DataChangeEvent{tableId=%s.members, before=[], after=[1013, Fiona, 16.0], op=INSERT, meta=()}") + .map(s -> String.format(s, schemaEvolveDatabase.getDatabaseName())) + .collect(Collectors.toList()); + + validateResult(expected, taskManagerConsumer); + } + + @Test + public void testSchemaEvolveWithIncompatibleChanges() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: %s\n" + + " port: 3306\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.(members|new_members)\n" + + " server-id: 5400-5404\n" + + " server-time-zone: UTC\n" + + "\n" + + "sink:\n" + + " type: values\n" + + "route:\n" + + " - source-table: %s.(members|new_members)\n" + + " sink-table: %s.merged\n" + + "\n" + + "pipeline:\n" + + " schema.change.behavior: evolve\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + schemaEvolveDatabase.getDatabaseName(), + schemaEvolveDatabase.getDatabaseName(), + schemaEvolveDatabase.getDatabaseName()); + Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); + Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); + Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); + submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + waitUntilJobRunning(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + validateSnapshotData("merged"); + + LOG.info("Starting schema evolution"); + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), + MYSQL.getDatabasePort(), + schemaEvolveDatabase.getDatabaseName()); + + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stmt = conn.createStatement()) { + waitForIncrementalStage("merged", stmt); + + // incompatible type INT and VARCHAR cannot be merged + stmt.execute("ALTER TABLE members CHANGE COLUMN age age VARCHAR(17);"); + } + + waitUntilSpecificEvent( + "java.lang.IllegalStateException: Incompatible types: \"INT\" and \"VARCHAR(17)\"", + taskManagerConsumer); + + // Ensure that job was terminated + waitUntilSpecificEvent( + "org.apache.flink.runtime.JobException: Recovery is suppressed by NoRestartBackoffTimeStrategy", + jobManagerConsumer); + } + + @Test + public void testSchemaEvolveWithException() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: %s\n" + + " port: 3306\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.members\n" + + " server-id: 5400-5404\n" + + " server-time-zone: UTC\n" + + "\n" + + "sink:\n" + + " type: values\n" + + " error.on.schema.change: true\n" + + "\n" + + "pipeline:\n" + + " schema.change.behavior: evolve\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + schemaEvolveDatabase.getDatabaseName()); + Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); + Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); + Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); + submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + waitUntilJobRunning(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + validateSnapshotData("members"); + + LOG.info("Starting schema evolution"); + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), + MYSQL.getDatabasePort(), + schemaEvolveDatabase.getDatabaseName()); + + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stmt = conn.createStatement()) { + waitForIncrementalStage("members", stmt); + stmt.execute("ALTER TABLE members ADD COLUMN gender TINYINT AFTER age;"); + } + + waitUntilSpecificEvent( + String.format( + "AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`gender` TINYINT, position=AFTER, existedColumnName=age}]}", + schemaEvolveDatabase.getDatabaseName()), + taskManagerConsumer); + + validateResult( + Arrays.asList( + String.format( + "Failed to apply schema change event AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`gender` TINYINT, position=AFTER, existedColumnName=age}]}.", + schemaEvolveDatabase.getDatabaseName()), + String.format( + "java.lang.RuntimeException: Rejected schema change event AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`gender` TINYINT, position=AFTER, existedColumnName=age}]} since error.on.schema.change is enabled.", + schemaEvolveDatabase.getDatabaseName()), + "org.apache.flink.runtime.JobException: Recovery is suppressed by NoRestartBackoffTimeStrategy"), + jobManagerConsumer); + } + + @Test + public void testSchemaTryEvolveWithException() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: %s\n" + + " port: 3306\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.members\n" + + " server-id: 5400-5404\n" + + " server-time-zone: UTC\n" + + "\n" + + "sink:\n" + + " type: values\n" + + " error.on.schema.change: true\n" + + "\n" + + "pipeline:\n" + + " schema.change.behavior: try_evolve\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + schemaEvolveDatabase.getDatabaseName()); + Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); + Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); + Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); + submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + waitUntilJobRunning(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + validateSnapshotData("members"); + LOG.info("Starting schema evolution"); + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), + MYSQL.getDatabasePort(), + schemaEvolveDatabase.getDatabaseName()); + + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stmt = conn.createStatement()) { + + waitForIncrementalStage("members", stmt); + + stmt.execute("ALTER TABLE members ADD COLUMN gender TINYINT AFTER age;"); + stmt.execute("INSERT INTO members VALUES (1012, 'Eve', 17, 0);"); + stmt.execute("UPDATE members SET name = 'Eva' WHERE id = 1012;"); + stmt.execute("DELETE FROM members WHERE id = 1012;"); + } + + List expected = + Stream.of( + // Add column never succeeded, so age column will not appear. + "DataChangeEvent{tableId=%s.members, before=[], after=[1012, Eve, 17], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.members, before=[1012, Eve, 17], after=[1012, Eva, 17], op=UPDATE, meta=()}", + "DataChangeEvent{tableId=%s.members, before=[1012, Eva, 17], after=[], op=DELETE, meta=()}") + .map(s -> String.format(s, schemaEvolveDatabase.getDatabaseName())) + .collect(Collectors.toList()); + + validateResult(expected, taskManagerConsumer); + + waitUntilSpecificEvent( + String.format( + "Failed to apply schema change AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`gender` TINYINT, position=AFTER, existedColumnName=age}]} to table %s.members.", + schemaEvolveDatabase.getDatabaseName(), + schemaEvolveDatabase.getDatabaseName()), + jobManagerConsumer); + + waitUntilSpecificEvent( + String.format( + "Rejected schema change event AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`gender` TINYINT, position=AFTER, existedColumnName=age}]} since error.on.schema.change is enabled.", + schemaEvolveDatabase.getDatabaseName()), + jobManagerConsumer); + } + + @Test + public void testSchemaIgnore() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: %s\n" + + " port: 3306\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.members\n" + + " server-id: 5400-5404\n" + + " server-time-zone: UTC\n" + + "\n" + + "sink:\n" + + " type: values\n" + + "\n" + + "pipeline:\n" + + " schema.change.behavior: ignore\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + schemaEvolveDatabase.getDatabaseName()); + Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); + Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); + Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); + submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + + waitUntilJobRunning(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + validateSnapshotData("members"); + + LOG.info("Starting schema evolution"); + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), + MYSQL.getDatabasePort(), + schemaEvolveDatabase.getDatabaseName()); + + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stmt = conn.createStatement()) { + + waitForIncrementalStage("members", stmt); + + // triggers AddColumnEvent + stmt.execute("ALTER TABLE members ADD COLUMN gender TINYINT AFTER age;"); + stmt.execute("INSERT INTO members VALUES (1012, 'Eve', 17, 0);"); + + // triggers AlterColumnTypeEvent and RenameColumnEvent + stmt.execute("ALTER TABLE members CHANGE COLUMN age precise_age DOUBLE;"); + + // triggers RenameColumnEvent + stmt.execute("ALTER TABLE members RENAME COLUMN gender TO biological_sex;"); + + // triggers DropColumnEvent + stmt.execute("ALTER TABLE members DROP COLUMN biological_sex"); + stmt.execute("INSERT INTO members VALUES (1013, 'Fiona', 16);"); + } + + List expected = + Stream.of( + "DataChangeEvent{tableId=%s.members, before=[], after=[1012, Eve, 17], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.members, before=[], after=[1013, Fiona, null], op=INSERT, meta=()}") + .map(s -> String.format(s, schemaEvolveDatabase.getDatabaseName())) + .collect(Collectors.toList()); + + validateResult(expected, taskManagerConsumer); + } + + @Test + public void testSchemaException() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: %s\n" + + " port: 3306\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.members\n" + + " server-id: 5400-5404\n" + + " server-time-zone: UTC\n" + + "\n" + + "sink:\n" + + " type: values\n" + + "\n" + + "pipeline:\n" + + " schema.change.behavior: exception\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + schemaEvolveDatabase.getDatabaseName()); + Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); + Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); + Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); + submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + + waitUntilJobRunning(Duration.ofSeconds(30)); + + LOG.info("Pipeline job is running"); + validateSnapshotData("members"); + + LOG.info("Starting schema evolution"); + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), + MYSQL.getDatabasePort(), + schemaEvolveDatabase.getDatabaseName()); + + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stmt = conn.createStatement()) { + waitForIncrementalStage("members", stmt); + + // triggers AddColumnEvent + stmt.execute("ALTER TABLE members ADD COLUMN gender TINYINT AFTER age;"); + } + + waitUntilSpecificEvent( + String.format( + "java.lang.RuntimeException: Refused to apply schema change event AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`gender` TINYINT, position=AFTER, existedColumnName=age}]} in EXCEPTION mode.", + schemaEvolveDatabase.getDatabaseName()), + taskManagerConsumer); + } + + @Test + public void testUnexpectedBehavior() { + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: %s\n" + + " port: 3306\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.members\n" + + " server-id: 5400-5404\n" + + " server-time-zone: UTC\n" + + "\n" + + "sink:\n" + + " type: values\n" + + "\n" + + "pipeline:\n" + + " schema.change.behavior: unexpected\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + schemaEvolveDatabase.getDatabaseName()); + Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); + Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); + Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); + + // Submitting job should fail given an unknown schema change behavior configuration + Assert.assertThrows( + AssertionError.class, + () -> submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar)); + } + + @Test + public void testFineGrainedSchemaEvolution() throws Exception { + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: %s\n" + + " port: 3306\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.members\n" + + " server-id: 5400-5404\n" + + " server-time-zone: UTC\n" + + "\n" + + "sink:\n" + + " type: values\n" + + " exclude.schema.changes:\n" + + " - drop\n" + + "\n" + + "pipeline:\n" + + " schema.change.behavior: evolve\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + schemaEvolveDatabase.getDatabaseName()); + Path mysqlCdcJar = TestUtils.getResource("mysql-cdc-pipeline-connector.jar"); + Path valuesCdcJar = TestUtils.getResource("values-cdc-pipeline-connector.jar"); + Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); + submitPipelineJob(pipelineJob, mysqlCdcJar, valuesCdcJar, mysqlDriverJar); + waitUntilJobRunning(Duration.ofSeconds(30)); + LOG.info("Pipeline job is running"); + validateSnapshotData("members"); + + LOG.info("Starting schema evolution"); + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", + MYSQL.getHost(), + MYSQL.getDatabasePort(), + schemaEvolveDatabase.getDatabaseName()); + + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stmt = conn.createStatement()) { + waitForIncrementalStage("members", stmt); + + // triggers AddColumnEvent + stmt.execute("ALTER TABLE members ADD COLUMN gender TINYINT AFTER age;"); + stmt.execute("INSERT INTO members VALUES (1012, 'Eve', 17, 0);"); + + // triggers AlterColumnTypeEvent and RenameColumnEvent + stmt.execute("ALTER TABLE members CHANGE COLUMN age precise_age DOUBLE;"); + + // triggers RenameColumnEvent + stmt.execute("ALTER TABLE members RENAME COLUMN gender TO biological_sex;"); + + // triggers DropColumnEvent + stmt.execute("ALTER TABLE members DROP COLUMN biological_sex;"); + stmt.execute("INSERT INTO members VALUES (1013, 'Fiona', 16);"); + } + + List expected = + Stream.of( + "AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`gender` TINYINT, position=AFTER, existedColumnName=age}]}", + "DataChangeEvent{tableId=%s.members, before=[], after=[1012, Eve, 17, 0], op=INSERT, meta=()}", + "AlterColumnTypeEvent{tableId=%s.members, typeMapping={age=DOUBLE}, oldTypeMapping={age=INT}}", + "RenameColumnEvent{tableId=%s.members, nameMapping={age=precise_age}}", + "RenameColumnEvent{tableId=%s.members, nameMapping={gender=biological_sex}}", + "DataChangeEvent{tableId=%s.members, before=[], after=[1013, Fiona, 16.0, null], op=INSERT, meta=()}") + .map(s -> String.format(s, schemaEvolveDatabase.getDatabaseName())) + .collect(Collectors.toList()); + + validateResult(expected, taskManagerConsumer); + + waitUntilSpecificEvent( + String.format( + "Ignored schema change DropColumnEvent{tableId=%s.members, droppedColumnNames=[biological_sex]} to table %s.members.", + schemaEvolveDatabase.getDatabaseName(), + schemaEvolveDatabase.getDatabaseName()), + jobManagerConsumer); + } + + private void validateResult(List expectedEvents, ToStringConsumer consumer) + throws Exception { + for (String event : expectedEvents) { + waitUntilSpecificEvent(event, consumer); + } + } + + private void waitUntilSpecificEvent(String event, ToStringConsumer consumer) throws Exception { + boolean result = false; + long endTimeout = System.currentTimeMillis() + SchemaEvolveE2eITCase.EVENT_WAITING_TIMEOUT; + while (System.currentTimeMillis() < endTimeout) { + String stdout = consumer.toUtf8String(); + if (stdout.contains(event)) { + result = true; + break; + } + Thread.sleep(1000); + } + if (!result) { + throw new TimeoutException( + "failed to get specific event: " + + event + + " from stdout: " + + consumer.toUtf8String()); + } + } +} diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/TransformE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/TransformE2eITCase.java index 2f896d3344e..ff61a88a2ed 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/TransformE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/TransformE2eITCase.java @@ -56,6 +56,7 @@ public class TransformE2eITCase extends PipelineTestEnvironment { protected static final String MYSQL_TEST_PASSWORD = "mysqlpw"; protected static final String MYSQL_DRIVER_CLASS = "com.mysql.cj.jdbc.Driver"; protected static final String INTER_CONTAINER_MYSQL_ALIAS = "mysql"; + protected static final long EVENT_WAITING_TIMEOUT = 60000L; @ClassRule public static final MySqlContainer MYSQL = @@ -131,14 +132,12 @@ public void testHeteroSchemaTransform() throws Exception { waitUntilSpecificEvent( String.format( "DataChangeEvent{tableId=%s.terminus, before=[], after=[1011, 11], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - 60000L); + transformRenameDatabase.getDatabaseName())); waitUntilSpecificEvent( String.format( "DataChangeEvent{tableId=%s.terminus, before=[], after=[2014, 14], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - 60000L); + transformRenameDatabase.getDatabaseName())); List expectedEvents = Arrays.asList( @@ -187,20 +186,17 @@ public void testHeteroSchemaTransform() throws Exception { waitUntilSpecificEvent( String.format( "DataChangeEvent{tableId=%s.terminus, before=[], after=[3007, 7], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - 20000L); + transformRenameDatabase.getDatabaseName())); waitUntilSpecificEvent( String.format( "DataChangeEvent{tableId=%s.terminus, before=[1009, 8.1], after=[1009, 100], op=UPDATE, meta=()}", - transformRenameDatabase.getDatabaseName()), - 20000L); + transformRenameDatabase.getDatabaseName())); waitUntilSpecificEvent( String.format( "DataChangeEvent{tableId=%s.terminus, before=[2011, 11], after=[], op=DELETE, meta=()}", - transformRenameDatabase.getDatabaseName()), - 20000L); + transformRenameDatabase.getDatabaseName())); String stdout = taskManagerConsumer.toUtf8String(); System.out.println(stdout); @@ -240,6 +236,10 @@ public void testMultipleHittingTable() throws Exception { waitUntilJobRunning(Duration.ofSeconds(30)); LOG.info("Pipeline job is running"); + waitUntilSpecificEvent( + String.format( + "CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` STRING,`PRICEALPHA` INT,`UID` INT,`NEWVERSION` STRING}, primaryKeys=ID, options=()}", + transformRenameDatabase.getDatabaseName())); List expectedEvents = Arrays.asList( String.format( @@ -296,34 +296,28 @@ public void testMultipleHittingTable() throws Exception { waitUntilSpecificEvent( String.format( "DataChangeEvent{tableId=%s.TABLEALPHA, before=[1009, 8.1, 0, 2009, 8.1], after=[1009, 100, 0, 2009, 100], op=UPDATE, meta=()}", - transformRenameDatabase.getDatabaseName()), - 6000L); + transformRenameDatabase.getDatabaseName())); waitUntilSpecificEvent( String.format( "DataChangeEvent{tableId=%s.TABLEALPHA, before=[], after=[3007, 7, 79, 4007, 7], op=INSERT, meta=()}", - transformRenameDatabase.getDatabaseName()), - 6000L); + transformRenameDatabase.getDatabaseName())); waitUntilSpecificEvent( String.format( "DataChangeEvent{tableId=%s.TABLEBETA, before=[2011, 11, Big Sur, 3011, 11], after=[], op=DELETE, meta=()}", - transformRenameDatabase.getDatabaseName()), - 6000L); - - String stdout = taskManagerConsumer.toUtf8String(); - System.out.println(stdout); + transformRenameDatabase.getDatabaseName())); } private void validateResult(List expectedEvents) throws Exception { for (String event : expectedEvents) { - waitUntilSpecificEvent(event, 6000L); + waitUntilSpecificEvent(event); } } - private void waitUntilSpecificEvent(String event, long timeout) throws Exception { + private void waitUntilSpecificEvent(String event) throws Exception { boolean result = false; - long endTimeout = System.currentTimeMillis() + timeout; + long endTimeout = System.currentTimeMillis() + TransformE2eITCase.EVENT_WAITING_TIMEOUT; while (System.currentTimeMillis() < endTimeout) { String stdout = taskManagerConsumer.toUtf8String(); if (stdout.contains(event)) { diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java index 65c0a202e51..1143012a617 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java @@ -267,6 +267,7 @@ private static String getFlinkProperties(String flinkVersion) { return String.join( "\n", Arrays.asList( + "restart-strategy.type: off", "jobmanager.rpc.address: jobmanager", "taskmanager.numberOfTaskSlots: 10", "parallelism.default: 4", diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/schema_evolve.sql b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/schema_evolve.sql new file mode 100644 index 00000000000..8bb7cfa79a0 --- /dev/null +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/resources/ddl/schema_evolve.sql @@ -0,0 +1,37 @@ +-- 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. + +DROP TABLE IF EXISTS members; + +CREATE TABLE members ( + id INT NOT NULL, + name VARCHAR(17), + age INT, + PRIMARY KEY (id) +); + +INSERT INTO members VALUES (1008, 'Alice', 21); +INSERT INTO members VALUES (1009, 'Bob', 20); +INSERT INTO members VALUES (1010, 'Carol', 19); +INSERT INTO members VALUES (1011, 'Derrida', 18); + +DROP TABLE IF EXISTS new_members; + +CREATE TABLE new_members ( + id INT NOT NULL, + name VARCHAR(17), + age INT, + PRIMARY KEY (id) +); diff --git a/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/SchemaManagerMigrationMock.java b/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/SchemaManagerMigrationMock.java index c946b9f5f46..7774604d7ed 100644 --- a/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/SchemaManagerMigrationMock.java +++ b/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/SchemaManagerMigrationMock.java @@ -18,6 +18,7 @@ package org.apache.flink.cdc.migration.tests; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.types.DataTypes; import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaManager; @@ -43,7 +44,10 @@ public SchemaManager generateDummyObject() { schemaVersions.put(1, DUMMY_SCHEMA); schemaVersions.put(2, DUMMY_SCHEMA); schemaVersions.put(3, DUMMY_SCHEMA); - return new SchemaManager(Collections.singletonMap(DUMMY_TABLE_ID, schemaVersions)); + return new SchemaManager( + Collections.singletonMap(DUMMY_TABLE_ID, schemaVersions), + Collections.singletonMap(DUMMY_TABLE_ID, schemaVersions), + SchemaChangeBehavior.EVOLVE); } @Override diff --git a/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/SchemaRegistryMigrationMock.java b/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/SchemaRegistryMigrationMock.java index 93269abece8..bb19223e64e 100644 --- a/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/SchemaRegistryMigrationMock.java +++ b/flink-cdc-migration-tests/flink-cdc-release-snapshot/src/main/java/org/apache/flink/cdc/migration/tests/SchemaRegistryMigrationMock.java @@ -18,9 +18,14 @@ package org.apache.flink.cdc.migration.tests; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; +import org.apache.flink.cdc.common.event.SchemaChangeEventTypeFamily; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.schema.Selectors; +import org.apache.flink.cdc.common.sink.MetadataApplier; import org.apache.flink.cdc.common.types.DataTypes; import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaDerivation; import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaManager; @@ -28,11 +33,14 @@ import java.lang.reflect.Field; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; /** Dummy classes for migration test. Called via reflection. */ public class SchemaRegistryMigrationMock implements MigrationMockBase { @@ -51,11 +59,36 @@ public SchemaManager generateDummySchemaManager() { schemaVersions.put(1, DUMMY_SCHEMA); schemaVersions.put(2, DUMMY_SCHEMA); schemaVersions.put(3, DUMMY_SCHEMA); - return new SchemaManager(Collections.singletonMap(DUMMY_TABLE_ID, schemaVersions)); + return new SchemaManager( + Collections.singletonMap(DUMMY_TABLE_ID, schemaVersions), + Collections.singletonMap(DUMMY_TABLE_ID, schemaVersions), + SchemaChangeBehavior.EVOLVE); } public SchemaRegistry generateSchemaRegistry() { - return new SchemaRegistry("Dummy Name", null, e -> {}, new ArrayList<>()); + return new SchemaRegistry( + "Dummy Name", + null, + new MetadataApplier() { + @Override + public boolean acceptsSchemaEvolutionType( + SchemaChangeEventType schemaChangeEventType) { + return true; + } + + @Override + public Set getSupportedSchemaEvolutionTypes() { + return Arrays.stream(SchemaChangeEventTypeFamily.ALL) + .collect(Collectors.toSet()); + } + + @Override + public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) { + // Do nothing + } + }, + new ArrayList<>(), + SchemaChangeBehavior.EVOLVE); } private SchemaManager getSchemaManager(SchemaRegistry schemaRegistry) throws Exception { 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..88218cff74a 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 @@ -19,13 +19,16 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.annotation.VisibleForTesting; import org.apache.flink.cdc.common.data.RecordData; import org.apache.flink.cdc.common.data.StringData; import org.apache.flink.cdc.common.event.DataChangeEvent; 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.SchemaChangeEventType; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.schema.Selectors; @@ -34,13 +37,17 @@ import org.apache.flink.cdc.common.types.DataTypeRoot; import org.apache.flink.cdc.common.utils.ChangeEventUtils; import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaRegistry; +import org.apache.flink.cdc.runtime.operators.schema.event.ApplyEvolvedSchemaChangeRequest; +import org.apache.flink.cdc.runtime.operators.schema.event.ApplyUpstreamSchemaChangeRequest; import org.apache.flink.cdc.runtime.operators.schema.event.CoordinationResponseUtils; import org.apache.flink.cdc.runtime.operators.schema.event.RefreshPendingListsRequest; import org.apache.flink.cdc.runtime.operators.schema.event.ReleaseUpstreamRequest; +import org.apache.flink.cdc.runtime.operators.schema.event.ReleaseUpstreamResponse; import org.apache.flink.cdc.runtime.operators.schema.event.SchemaChangeProcessingResponse; import org.apache.flink.cdc.runtime.operators.schema.event.SchemaChangeRequest; import org.apache.flink.cdc.runtime.operators.schema.event.SchemaChangeResponse; import org.apache.flink.cdc.runtime.operators.schema.event.SchemaChangeResultRequest; +import org.apache.flink.cdc.runtime.operators.schema.metrics.SchemaOperatorMetrics; import org.apache.flink.cdc.runtime.operators.sink.SchemaEvolutionClient; import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator; import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway; @@ -93,20 +100,47 @@ public class SchemaOperator extends AbstractStreamOperator private transient List> routes; private transient TaskOperatorEventGateway toCoordinator; private transient SchemaEvolutionClient schemaEvolutionClient; - private transient LoadingCache cachedSchemas; + private transient LoadingCache upstreamSchema; + private transient LoadingCache evolvedSchema; + private transient LoadingCache schemaDivergesMap; private final long rpcTimeOutInMillis; + private final SchemaChangeBehavior schemaChangeBehavior; + private transient SchemaOperatorMetrics schemaOperatorMetrics; + + @VisibleForTesting public SchemaOperator(List> routingRules) { this.routingRules = routingRules; this.chainingStrategy = ChainingStrategy.ALWAYS; this.rpcTimeOutInMillis = DEFAULT_SCHEMA_OPERATOR_RPC_TIMEOUT.toMillis(); + this.schemaChangeBehavior = SchemaChangeBehavior.EVOLVE; } + @VisibleForTesting public SchemaOperator(List> routingRules, Duration rpcTimeOut) { this.routingRules = routingRules; this.chainingStrategy = ChainingStrategy.ALWAYS; this.rpcTimeOutInMillis = rpcTimeOut.toMillis(); + this.schemaChangeBehavior = SchemaChangeBehavior.EVOLVE; + } + + public SchemaOperator( + List> routingRules, + Duration rpcTimeOut, + SchemaChangeBehavior schemaChangeBehavior) { + this.routingRules = routingRules; + this.chainingStrategy = ChainingStrategy.ALWAYS; + this.rpcTimeOutInMillis = rpcTimeOut.toMillis(); + this.schemaChangeBehavior = schemaChangeBehavior; + } + + @Override + public void open() throws Exception { + super.open(); + schemaOperatorMetrics = + new SchemaOperatorMetrics( + getRuntimeContext().getMetricGroup(), schemaChangeBehavior); } @Override @@ -130,14 +164,34 @@ public void setup( }) .collect(Collectors.toList()); schemaEvolutionClient = new SchemaEvolutionClient(toCoordinator, getOperatorID()); - cachedSchemas = + evolvedSchema = CacheBuilder.newBuilder() .expireAfterAccess(CACHE_EXPIRE_DURATION) .build( new CacheLoader() { @Override public Schema load(TableId tableId) { - return getLatestSchema(tableId); + return getLatestEvolvedSchema(tableId); + } + }); + upstreamSchema = + CacheBuilder.newBuilder() + .expireAfterAccess(CACHE_EXPIRE_DURATION) + .build( + new CacheLoader() { + @Override + public Schema load(TableId tableId) throws Exception { + return getLatestUpstreamSchema(tableId); + } + }); + schemaDivergesMap = + CacheBuilder.newBuilder() + .expireAfterAccess(CACHE_EXPIRE_DURATION) + .build( + new CacheLoader() { + @Override + public Boolean load(TableId tableId) throws Exception { + return checkSchemaDiverges(tableId); } }); } @@ -160,75 +214,105 @@ public void initializeState(StateInitializationContext context) throws Exception public void processElement(StreamRecord streamRecord) throws InterruptedException, TimeoutException { Event event = streamRecord.getValue(); - // Schema changes if (event instanceof SchemaChangeEvent) { - TableId tableId = ((SchemaChangeEvent) event).tableId(); - LOG.info( - "Table {} received SchemaChangeEvent and start to be blocked.", - tableId.toString()); - handleSchemaChangeEvent(tableId, (SchemaChangeEvent) event); - // Update caches - cachedSchemas.put(tableId, getLatestSchema(tableId)); - getRoutedTable(tableId) - .ifPresent(routed -> cachedSchemas.put(routed, getLatestSchema(routed))); - return; + processSchemaChangeEvents((SchemaChangeEvent) event); + } else if (event instanceof DataChangeEvent) { + processDataChangeEvents(streamRecord, (DataChangeEvent) event); + } else { + throw new RuntimeException("Unknown event type in Stream record: " + event); + } + } + + private void processSchemaChangeEvents(SchemaChangeEvent event) + throws InterruptedException, TimeoutException { + TableId tableId = event.tableId(); + LOG.info("Table {} received SchemaChangeEvent and start to be blocked.", tableId); + handleSchemaChangeEvent(tableId, event); + // Update caches + upstreamSchema.put(tableId, getLatestUpstreamSchema(tableId)); + schemaDivergesMap.put(tableId, checkSchemaDiverges(tableId)); + + Optional optionalRoutedTable = getRoutedTable(tableId); + if (optionalRoutedTable.isPresent()) { + TableId routedTableId = optionalRoutedTable.get(); + evolvedSchema.put(routedTableId, getLatestEvolvedSchema(routedTableId)); + } else { + evolvedSchema.put(tableId, getLatestEvolvedSchema(tableId)); } + } - // Data changes - DataChangeEvent dataChangeEvent = (DataChangeEvent) event; - TableId tableId = dataChangeEvent.tableId(); + private void processDataChangeEvents(StreamRecord streamRecord, DataChangeEvent event) { + TableId tableId = event.tableId(); Optional optionalRoutedTable = getRoutedTable(tableId); if (optionalRoutedTable.isPresent()) { output.collect( new StreamRecord<>( - maybeFillInNullForEmptyColumns( - dataChangeEvent, optionalRoutedTable.get()))); + normalizeSchemaChangeEvents(event, optionalRoutedTable.get(), false))); + } else if (Boolean.FALSE.equals(schemaDivergesMap.getIfPresent(tableId))) { + output.collect(new StreamRecord<>(normalizeSchemaChangeEvents(event, true))); } else { output.collect(streamRecord); } } - // ---------------------------------------------------------------------------------- + private DataChangeEvent normalizeSchemaChangeEvents( + DataChangeEvent event, boolean tolerantMode) { + return normalizeSchemaChangeEvents(event, event.tableId(), tolerantMode); + } - private DataChangeEvent maybeFillInNullForEmptyColumns( - DataChangeEvent originalEvent, TableId routedTableId) { + private DataChangeEvent normalizeSchemaChangeEvents( + DataChangeEvent event, TableId renamedTableId, boolean tolerantMode) { try { - Schema originalSchema = cachedSchemas.get(originalEvent.tableId()); - Schema routedTableSchema = cachedSchemas.get(routedTableId); - if (originalSchema.equals(routedTableSchema)) { - return ChangeEventUtils.recreateDataChangeEvent(originalEvent, routedTableId); + Schema originalSchema = upstreamSchema.get(event.tableId()); + Schema evolvedTableSchema = evolvedSchema.get(renamedTableId); + if (originalSchema.equals(evolvedTableSchema)) { + return ChangeEventUtils.recreateDataChangeEvent(event, renamedTableId); } - switch (originalEvent.op()) { + switch (event.op()) { case INSERT: return DataChangeEvent.insertEvent( - routedTableId, + renamedTableId, regenerateRecordData( - originalEvent.after(), originalSchema, routedTableSchema), - originalEvent.meta()); + event.after(), + originalSchema, + evolvedTableSchema, + tolerantMode), + event.meta()); case UPDATE: return DataChangeEvent.updateEvent( - routedTableId, + renamedTableId, regenerateRecordData( - originalEvent.before(), originalSchema, routedTableSchema), + event.before(), + originalSchema, + evolvedTableSchema, + tolerantMode), regenerateRecordData( - originalEvent.after(), originalSchema, routedTableSchema), - originalEvent.meta()); + event.after(), + originalSchema, + evolvedTableSchema, + tolerantMode), + event.meta()); case DELETE: return DataChangeEvent.deleteEvent( - routedTableId, + renamedTableId, regenerateRecordData( - originalEvent.before(), originalSchema, routedTableSchema), - originalEvent.meta()); + event.before(), + originalSchema, + evolvedTableSchema, + tolerantMode), + event.meta()); case REPLACE: return DataChangeEvent.replaceEvent( - routedTableId, + renamedTableId, regenerateRecordData( - originalEvent.after(), originalSchema, routedTableSchema), - originalEvent.meta()); + event.after(), + originalSchema, + evolvedTableSchema, + tolerantMode), + event.meta()); default: throw new IllegalArgumentException( - String.format( - "Unrecognized operation type \"%s\"", originalEvent.op())); + String.format("Unrecognized operation type \"%s\"", event.op())); } } catch (Exception e) { throw new IllegalStateException("Unable to fill null for empty columns", e); @@ -236,7 +320,10 @@ private DataChangeEvent maybeFillInNullForEmptyColumns( } private RecordData regenerateRecordData( - RecordData recordData, Schema originalSchema, Schema routedTableSchema) { + RecordData recordData, + Schema originalSchema, + Schema routedTableSchema, + boolean tolerantMode) { // Regenerate record data List fieldGetters = new ArrayList<>(); for (Column column : routedTableSchema.getColumns()) { @@ -252,7 +339,9 @@ private RecordData regenerateRecordData( if (originalSchema.getColumn(columnName).get().getType().equals(column.getType())) { fieldGetters.add(fieldGetter); } else { - fieldGetters.add(new TypeCoercionFieldGetter(column.getType(), fieldGetter)); + fieldGetters.add( + new TypeCoercionFieldGetter( + column.getType(), fieldGetter, tolerantMode)); } } } @@ -276,17 +365,81 @@ private Optional getRoutedTable(TableId originalTableId) { private void handleSchemaChangeEvent(TableId tableId, SchemaChangeEvent schemaChangeEvent) throws InterruptedException, TimeoutException { + + if (schemaChangeBehavior == SchemaChangeBehavior.EXCEPTION + && schemaChangeEvent.getType() != SchemaChangeEventType.CREATE_TABLE) { + // CreateTableEvent should be applied even in EXCEPTION mode + throw new RuntimeException( + String.format( + "Refused to apply schema change event %s in EXCEPTION mode.", + schemaChangeEvent)); + } + // The request will need to send a FlushEvent or block until flushing finished SchemaChangeResponse response = requestSchemaChange(tableId, schemaChangeEvent); + if (!response.getSchemaChangeEvents().isEmpty()) { LOG.info( "Sending the FlushEvent for table {} in subtask {}.", tableId, getRuntimeContext().getIndexOfThisSubtask()); + output.collect(new StreamRecord<>(new FlushEvent(tableId))); - response.getSchemaChangeEvents().forEach(e -> output.collect(new StreamRecord<>(e))); + List expectedSchemaChangeEvents = response.getSchemaChangeEvents(); + schemaOperatorMetrics.increaseSchemaChangeEvents(expectedSchemaChangeEvents.size()); + // The request will block until flushing finished in each sink writer - requestReleaseUpstream(); + ReleaseUpstreamResponse schemaEvolveResponse = requestReleaseUpstream(); + List finishedSchemaChangeEvents = + schemaEvolveResponse.getFinishedSchemaChangeEvents(); + List> failedSchemaChangeEvents = + schemaEvolveResponse.getFailedSchemaChangeEvents(); + List ignoredSchemaChangeEvents = + schemaEvolveResponse.getIgnoredSchemaChangeEvents(); + + if (schemaChangeBehavior == SchemaChangeBehavior.EVOLVE + || schemaChangeBehavior == SchemaChangeBehavior.EXCEPTION) { + if (schemaEvolveResponse.hasException()) { + throw new RuntimeException( + String.format( + "Failed to apply schema change event %s.\nExceptions: %s", + schemaChangeEvent, + schemaEvolveResponse.getPrintableFailedSchemaChangeEvents())); + } + } else if (schemaChangeBehavior == SchemaChangeBehavior.TRY_EVOLVE + || schemaChangeBehavior == SchemaChangeBehavior.IGNORE) { + if (schemaEvolveResponse.hasException()) { + schemaEvolveResponse + .getFailedSchemaChangeEvents() + .forEach( + e -> + LOG.warn( + "Failed to apply event {}, but keeps running in TRY_EVOLVE mode. Caused by: {}", + e.f0, + e.f1)); + } + } else { + throw new IllegalStateException( + "Unexpected schema change behavior: " + schemaChangeBehavior); + } + + // Update evolved schema changes based on apply results + requestApplyEvolvedSchemaChanges(tableId, finishedSchemaChangeEvents); + finishedSchemaChangeEvents.forEach(e -> output.collect(new StreamRecord<>(e))); + + LOG.info( + "Applied schema change event {} to downstream. Among {} total evolved events, {} succeeded, {} failed, and {} ignored.", + schemaChangeEvent, + expectedSchemaChangeEvents.size(), + finishedSchemaChangeEvents.size(), + failedSchemaChangeEvents.size(), + ignoredSchemaChangeEvents.size()); + + schemaOperatorMetrics.increaseFinishedSchemaChangeEvents( + finishedSchemaChangeEvents.size()); + schemaOperatorMetrics.increaseFailedSchemaChangeEvents(failedSchemaChangeEvents.size()); + schemaOperatorMetrics.increaseIgnoredSchemaChangeEvents( + ignoredSchemaChangeEvents.size()); } } @@ -295,7 +448,18 @@ private SchemaChangeResponse requestSchemaChange( return sendRequestToCoordinator(new SchemaChangeRequest(tableId, schemaChangeEvent)); } - private void requestReleaseUpstream() throws InterruptedException, TimeoutException { + private void requestApplyUpstreamSchemaChanges( + TableId tableId, SchemaChangeEvent schemaChangeEvent) { + sendRequestToCoordinator(new ApplyUpstreamSchemaChangeRequest(tableId, schemaChangeEvent)); + } + + private void requestApplyEvolvedSchemaChanges( + TableId tableId, List schemaChangeEvents) { + sendRequestToCoordinator(new ApplyEvolvedSchemaChangeRequest(tableId, schemaChangeEvents)); + } + + private ReleaseUpstreamResponse requestReleaseUpstream() + throws InterruptedException, TimeoutException { CoordinationResponse coordinationResponse = sendRequestToCoordinator(new ReleaseUpstreamRequest()); long nextRpcTimeOutMillis = System.currentTimeMillis() + rpcTimeOutInMillis; @@ -307,6 +471,7 @@ private void requestReleaseUpstream() throws InterruptedException, TimeoutExcept throw new TimeoutException("TimeOut when requesting release upstream"); } } + return ((ReleaseUpstreamResponse) coordinationResponse); } private @@ -322,9 +487,9 @@ RESPONSE sendRequestToCoordinator(REQUEST request) { } } - private Schema getLatestSchema(TableId tableId) { + private Schema getLatestEvolvedSchema(TableId tableId) { try { - Optional optionalSchema = schemaEvolutionClient.getLatestSchema(tableId); + Optional optionalSchema = schemaEvolutionClient.getLatestEvolvedSchema(tableId); if (!optionalSchema.isPresent()) { throw new IllegalStateException( String.format("Schema doesn't exist for table \"%s\"", tableId)); @@ -336,6 +501,30 @@ private Schema getLatestSchema(TableId tableId) { } } + private Schema getLatestUpstreamSchema(TableId tableId) { + try { + Optional optionalSchema = + schemaEvolutionClient.getLatestUpstreamSchema(tableId); + if (!optionalSchema.isPresent()) { + throw new IllegalStateException( + String.format("Schema doesn't exist for table \"%s\"", tableId)); + } + return optionalSchema.get(); + } catch (Exception e) { + throw new IllegalStateException( + String.format("Unable to get latest schema for table \"%s\"", tableId)); + } + } + + private Boolean checkSchemaDiverges(TableId tableId) { + try { + return getLatestEvolvedSchema(tableId).equals(getLatestUpstreamSchema(tableId)); + } catch (IllegalStateException e) { + // schema fetch failed, regard it as diverged + return true; + } + } + private static class NullFieldGetter implements RecordData.FieldGetter { @Nullable @Override @@ -347,11 +536,22 @@ public Object getFieldOrNull(RecordData recordData) { private static class TypeCoercionFieldGetter implements RecordData.FieldGetter { private final DataType destinationType; private final RecordData.FieldGetter originalFieldGetter; + private final boolean tolerantMode; public TypeCoercionFieldGetter( - DataType destinationType, RecordData.FieldGetter originalFieldGetter) { + DataType destinationType, + RecordData.FieldGetter originalFieldGetter, + boolean tolerantMode) { this.destinationType = destinationType; this.originalFieldGetter = originalFieldGetter; + this.tolerantMode = tolerantMode; + } + + private Object fail(IllegalArgumentException e) throws IllegalArgumentException { + if (tolerantMode) { + return null; + } + throw e; } @Nullable @@ -372,38 +572,42 @@ public Object getFieldOrNull(RecordData recordData) { // INT return ((Integer) originalField).longValue(); } else { - throw new IllegalArgumentException( - String.format( - "Cannot fit type \"%s\" into a BIGINT column. " - + "Currently only TINYINT / SMALLINT / INT can be accepted by a BIGINT column", - originalField.getClass())); + return fail( + new IllegalArgumentException( + String.format( + "Cannot fit type \"%s\" into a BIGINT column. " + + "Currently only TINYINT / SMALLINT / INT can be accepted by a BIGINT column", + originalField.getClass()))); } } else if (destinationType.is(DataTypeFamily.APPROXIMATE_NUMERIC)) { if (originalField instanceof Float) { // FLOAT return ((Float) originalField).doubleValue(); } else { - throw new IllegalArgumentException( - String.format( - "Cannot fit type \"%s\" into a DOUBLE column. " - + "Currently only FLOAT can be accepted by a DOUBLE column", - originalField.getClass())); + return fail( + new IllegalArgumentException( + String.format( + "Cannot fit type \"%s\" into a DOUBLE column. " + + "Currently only FLOAT can be accepted by a DOUBLE column", + originalField.getClass()))); } } else if (destinationType.is(DataTypeRoot.VARCHAR)) { if (originalField instanceof StringData) { return originalField; } else { - throw new IllegalArgumentException( - String.format( - "Cannot fit type \"%s\" into a STRING column. " - + "Currently only CHAR / VARCHAR can be accepted by a STRING column", - originalField.getClass())); + return fail( + new IllegalArgumentException( + String.format( + "Cannot fit type \"%s\" into a STRING column. " + + "Currently only CHAR / VARCHAR can be accepted by a STRING column", + originalField.getClass()))); } } else { - throw new IllegalArgumentException( - String.format( - "Column type \"%s\" doesn't support type coercion", - destinationType)); + return fail( + new IllegalArgumentException( + String.format( + "Column type \"%s\" doesn't support type coercion", + destinationType))); } } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/SchemaOperatorFactory.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/SchemaOperatorFactory.java index ecf50017156..bf559871f3c 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/SchemaOperatorFactory.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/SchemaOperatorFactory.java @@ -21,6 +21,7 @@ import org.apache.flink.cdc.common.annotation.Internal; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; import org.apache.flink.cdc.common.sink.MetadataApplier; import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaRegistryProvider; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -41,19 +42,23 @@ public class SchemaOperatorFactory extends SimpleOperatorFactory private final MetadataApplier metadataApplier; private final List> routingRules; + private final SchemaChangeBehavior schemaChangeBehavior; public SchemaOperatorFactory( MetadataApplier metadataApplier, List> routingRules, - Duration rpcTimeOut) { - super(new SchemaOperator(routingRules, rpcTimeOut)); + Duration rpcTimeOut, + SchemaChangeBehavior schemaChangeBehavior) { + super(new SchemaOperator(routingRules, rpcTimeOut, schemaChangeBehavior)); this.metadataApplier = metadataApplier; this.routingRules = routingRules; + this.schemaChangeBehavior = schemaChangeBehavior; } @Override public OperatorCoordinator.Provider getCoordinatorProvider( String operatorName, OperatorID operatorID) { - return new SchemaRegistryProvider(operatorID, operatorName, metadataApplier, routingRules); + return new SchemaRegistryProvider( + operatorID, operatorName, metadataApplier, routingRules, schemaChangeBehavior); } } 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..de420939ccf 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaDerivation.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaDerivation.java @@ -19,12 +19,18 @@ 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.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.SchemaChangeEventVisitor; 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.PhysicalColumn; import org.apache.flink.cdc.common.schema.Schema; @@ -83,31 +89,69 @@ public List applySchemaChange(SchemaChangeEvent schemaChangeE // 1-to-1 mapping. Replace the table ID directly SchemaChangeEvent derivedSchemaChangeEvent = ChangeEventUtils.recreateSchemaChangeEvent(schemaChangeEvent, derivedTable); - schemaManager.applySchemaChange(derivedSchemaChangeEvent); + schemaManager.applyUpstreamSchemaChange(derivedSchemaChangeEvent); return Collections.singletonList(derivedSchemaChangeEvent); } // 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)); - } + Schema derivedTableSchema = schemaManager.getLatestUpstreamSchema(derivedTable).get(); + + return schemaChangeEvent.visit( + new SchemaChangeEventVisitor>() { + + @Override + public List visit(AddColumnEvent event) { + return handleAddColumnEvent(event, derivedTableSchema, derivedTable); + } + + @Override + public List visit(AlterColumnCommentEvent event) { + return Collections.emptyList(); + } + + @Override + public List visit(AlterColumnTypeEvent event) { + return handleAlterColumnTypeEvent( + event, derivedTableSchema, derivedTable); + } + + @Override + public List visit(AlterTableCommentEvent event) { + return Collections.emptyList(); + } + + @Override + public List visit(CreateTableEvent event) { + return handleCreateTableEvent(event, derivedTableSchema, derivedTable); + } + + @Override + public List visit(DropColumnEvent event) { + // Column drop shouldn't be spread to route destination. + return Collections.emptyList(); + } + + @Override + public List visit(DropTableEvent event) { + // Column drop shouldn't be spread to route destination. + return Collections.emptyList(); + } + + @Override + public List visit(RenameColumnEvent event) { + return handleRenameColumnEvent(event, derivedTableSchema, derivedTable); + } + + @Override + public List visit(RenameTableEvent event) { + return Collections.emptyList(); + } + + @Override + public List visit(TruncateTableEvent event) { + return Collections.emptyList(); + } + }); } // No routes are matched @@ -182,7 +226,7 @@ private List handleRenameColumnEvent( AddColumnEvent derivedSchemaChangeEvent = new AddColumnEvent(derivedTable, newColumns); schemaChangeEvents.add(derivedSchemaChangeEvent); } - schemaChangeEvents.forEach(schemaManager::applySchemaChange); + schemaChangeEvents.forEach(schemaManager::applyUpstreamSchemaChange); return schemaChangeEvents; } @@ -214,7 +258,7 @@ private List handleAlterColumnTypeEvent( new AlterColumnTypeEvent(derivedTable, typeDifference); schemaChangeEvents.add(derivedSchemaChangeEvent); } - schemaChangeEvents.forEach(schemaManager::applySchemaChange); + schemaChangeEvents.forEach(schemaManager::applyUpstreamSchemaChange); return schemaChangeEvents; } @@ -253,7 +297,7 @@ private List handleAddColumnEvent( if (!newTypeMapping.isEmpty()) { schemaChangeEvents.add(new AlterColumnTypeEvent(derivedTable, newTypeMapping)); } - schemaChangeEvents.forEach(schemaManager::applySchemaChange); + schemaChangeEvents.forEach(schemaManager::applyUpstreamSchemaChange); return schemaChangeEvents; } @@ -289,7 +333,7 @@ private List handleCreateTableEvent( if (!newTypeMapping.isEmpty()) { schemaChangeEvents.add(new AlterColumnTypeEvent(derivedTable, newTypeMapping)); } - schemaChangeEvents.forEach(schemaManager::applySchemaChange); + schemaChangeEvents.forEach(schemaManager::applyUpstreamSchemaChange); 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 99335180d9c..595105780da 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 @@ -17,10 +17,12 @@ package org.apache.flink.cdc.runtime.operators.schema.coordinator; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.annotation.Internal; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.utils.SchemaUtils; import org.apache.flink.cdc.runtime.serializer.TableIdSerializer; @@ -48,47 +50,97 @@ 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 { private static final Logger LOG = LoggerFactory.getLogger(SchemaManager.class); private static final int INITIAL_SCHEMA_VERSION = 0; private static final int VERSIONS_TO_KEEP = 3; + private final SchemaChangeBehavior behavior; // Serializer for checkpointing public static final Serializer SERIALIZER = new Serializer(); // Schema management - private final Map> tableSchemas; + private final Map> upstreamSchemas; + + // Schema management + private final Map> evolvedSchemas; public SchemaManager() { - tableSchemas = new HashMap<>(); + evolvedSchemas = new HashMap<>(); + upstreamSchemas = new HashMap<>(); + behavior = SchemaChangeBehavior.EVOLVE; + } + + public SchemaManager(SchemaChangeBehavior behavior) { + evolvedSchemas = new HashMap<>(); + upstreamSchemas = new HashMap<>(); + this.behavior = behavior; + } + + public SchemaManager( + Map> upstreamSchemas, + Map> evolvedSchemas, + SchemaChangeBehavior behavior) { + this.evolvedSchemas = evolvedSchemas; + this.upstreamSchemas = upstreamSchemas; + this.behavior = behavior; + } + + public SchemaChangeBehavior getBehavior() { + return behavior; } - public SchemaManager(Map> tableSchemas) { - this.tableSchemas = tableSchemas; + public final boolean schemaExists( + Map> schemaMap, TableId tableId) { + return schemaMap.containsKey(tableId) && !schemaMap.get(tableId).isEmpty(); } - /** Check if schema exists for the specified table ID. */ - public final boolean schemaExists(TableId tableId) { - return tableSchemas.containsKey(tableId) && !tableSchemas.get(tableId).isEmpty(); + public final boolean upstreamSchemaExists(TableId tableId) { + return schemaExists(upstreamSchemas, tableId); } - /** Get the latest schema of the specified table. */ - public Optional getLatestSchema(TableId tableId) { - return getLatestSchemaVersion(tableId) - .map(version -> tableSchemas.get(tableId).get(version)); + public final boolean evolvedSchemaExists(TableId tableId) { + return schemaExists(evolvedSchemas, tableId); + } + + /** Get the latest evolved schema of the specified table. */ + public Optional getLatestEvolvedSchema(TableId tableId) { + return getLatestSchemaVersion(evolvedSchemas, tableId) + .map(version -> evolvedSchemas.get(tableId).get(version)); + } + + /** Get the latest upstream schema of the specified table. */ + public Optional getLatestUpstreamSchema(TableId tableId) { + return getLatestSchemaVersion(upstreamSchemas, tableId) + .map(version -> upstreamSchemas.get(tableId).get(version)); } /** Get schema at the specified version of a table. */ - public Schema getSchema(TableId tableId, int version) { + public Schema getEvolvedSchema(TableId tableId, int version) { checkArgument( - tableSchemas.containsKey(tableId), - "Unable to find schema for table \"%s\"", + evolvedSchemas.containsKey(tableId), + "Unable to find evolved schema for table \"%s\"", tableId); - SortedMap versionedSchemas = tableSchemas.get(tableId); + SortedMap versionedSchemas = evolvedSchemas.get(tableId); + checkArgument( + versionedSchemas.containsKey(version), + "Schema version %s does not exist for table \"%s\"", + version, + tableId); + return versionedSchemas.get(version); + } + + /** Get schema at the specified version of a table. */ + public Schema getUpstreamSchema(TableId tableId, int version) { + checkArgument( + upstreamSchemas.containsKey(tableId), + "Unable to find upstream schema for table \"%s\"", + tableId); + SortedMap versionedSchemas = upstreamSchemas.get(tableId); checkArgument( versionedSchemas.containsKey(version), "Schema version %s does not exist for table \"%s\"", @@ -98,20 +150,49 @@ public Schema getSchema(TableId tableId, int version) { } /** Apply schema change to a table. */ - public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) { - if (schemaChangeEvent instanceof CreateTableEvent) { - handleCreateTableEvent(((CreateTableEvent) schemaChangeEvent)); + public void applyUpstreamSchemaChange(SchemaChangeEvent schemaChangeEvent) { + Optional optionalSchema = getLatestUpstreamSchema(schemaChangeEvent.tableId()); + if (!(schemaChangeEvent instanceof CreateTableEvent)) { + checkArgument( + optionalSchema.isPresent(), + "Unable to apply SchemaChangeEvent for table \"%s\" without existing schema", + schemaChangeEvent.tableId()); + } + + LOG.info("Handling upstream schema change event: {}", schemaChangeEvent); + Tuple2 appliedSchema = + SchemaUtils.applySchemaChangeEvent(schemaChangeEvent, optionalSchema.orElse(null)); + + if (appliedSchema.f1 != null) { + registerNewSchema(upstreamSchemas, appliedSchema.f0, appliedSchema.f1); } else { - Optional optionalSchema = getLatestSchema(schemaChangeEvent.tableId()); + dropSchema(upstreamSchemas, appliedSchema.f0); + } + } + + /** Apply schema change to a table. */ + public void applyEvolvedSchemaChange(SchemaChangeEvent schemaChangeEvent) { + Optional optionalSchema = getLatestEvolvedSchema(schemaChangeEvent.tableId()); + if (!(schemaChangeEvent instanceof CreateTableEvent)) { checkArgument( optionalSchema.isPresent(), "Unable to apply SchemaChangeEvent for table \"%s\" without existing schema", schemaChangeEvent.tableId()); + } else { + checkArgument( + !optionalSchema.isPresent(), + "Unable to apply CreateTableEvent to an existing schema for table \"%s\"", + schemaChangeEvent.tableId()); + } - LOG.info("Handling schema change event: {}", schemaChangeEvent); - registerNewSchema( - schemaChangeEvent.tableId(), - SchemaUtils.applySchemaChangeEvent(optionalSchema.get(), schemaChangeEvent)); + LOG.info("Handling upstream schema change event: {}", schemaChangeEvent); + Tuple2 appliedSchema = + SchemaUtils.applySchemaChangeEvent(schemaChangeEvent, optionalSchema.orElse(null)); + + if (appliedSchema.f1 != null) { + registerNewSchema(evolvedSchemas, appliedSchema.f0, appliedSchema.f1); + } else { + dropSchema(evolvedSchemas, appliedSchema.f0); } } @@ -124,39 +205,35 @@ public boolean equals(Object o) { return false; } SchemaManager that = (SchemaManager) o; - return Objects.equals(tableSchemas, that.tableSchemas); + return Objects.equals(upstreamSchemas, that.upstreamSchemas) + && Objects.equals(evolvedSchemas, that.evolvedSchemas); } @Override public int hashCode() { - return Objects.hash(tableSchemas); + return Objects.hash(upstreamSchemas, evolvedSchemas); } // -------------------------------- Helper functions ------------------------------------- - private Optional getLatestSchemaVersion(TableId tableId) { - if (!tableSchemas.containsKey(tableId)) { + private Optional getLatestSchemaVersion( + final Map> schemaMap, TableId tableId) { + if (!schemaMap.containsKey(tableId)) { return Optional.empty(); } try { - return Optional.of(tableSchemas.get(tableId).lastKey()); + return Optional.of(schemaMap.get(tableId).lastKey()); } catch (NoSuchElementException e) { return Optional.empty(); } } - private void handleCreateTableEvent(CreateTableEvent event) { - checkArgument( - !schemaExists(event.tableId()), - "Unable to apply CreateTableEvent to an existing schema for table \"%s\"", - event.tableId()); - LOG.info("Handling schema change event: {}", event); - registerNewSchema(event.tableId(), event.getSchema()); - } - - private void registerNewSchema(TableId tableId, Schema newSchema) { - if (schemaExists(tableId)) { - SortedMap versionedSchemas = tableSchemas.get(tableId); + private void registerNewSchema( + final Map> schemaMap, + TableId tableId, + Schema newSchema) { + if (schemaExists(schemaMap, tableId)) { + SortedMap versionedSchemas = schemaMap.get(tableId); Integer latestVersion = versionedSchemas.lastKey(); versionedSchemas.put(latestVersion + 1, newSchema); if (versionedSchemas.size() > VERSIONS_TO_KEEP) { @@ -165,14 +242,19 @@ private void registerNewSchema(TableId tableId, Schema newSchema) { } else { TreeMap versionedSchemas = new TreeMap<>(); versionedSchemas.put(INITIAL_SCHEMA_VERSION, newSchema); - tableSchemas.putIfAbsent(tableId, versionedSchemas); + schemaMap.putIfAbsent(tableId, versionedSchemas); } } + private void dropSchema( + final Map> schemaMap, TableId tableId) { + schemaMap.remove(tableId, null); + } + /** Serializer for {@link SchemaManager}. */ public static class Serializer implements SimpleVersionedSerializer { - public static final int CURRENT_VERSION = 1; + public static final int CURRENT_VERSION = 2; @Override public int getVersion() { @@ -181,71 +263,93 @@ public int getVersion() { @Override public byte[] serialize(SchemaManager schemaManager) throws IOException { - TableIdSerializer tableIdSerializer = TableIdSerializer.INSTANCE; - SchemaSerializer schemaSerializer = SchemaSerializer.INSTANCE; try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream out = new DataOutputStream(baos)) { - // Number of tables - out.writeInt(schemaManager.tableSchemas.size()); - for (Map.Entry> tableSchema : - schemaManager.tableSchemas.entrySet()) { - // Table ID - TableId tableId = tableSchema.getKey(); - tableIdSerializer.serialize(tableId, new DataOutputViewStreamWrapper(out)); - - // Schema with versions - SortedMap versionedSchemas = tableSchema.getValue(); - out.writeInt(versionedSchemas.size()); - for (Map.Entry versionedSchema : versionedSchemas.entrySet()) { - // Version - Integer version = versionedSchema.getKey(); - out.writeInt(version); - // Schema - Schema schema = versionedSchema.getValue(); - schemaSerializer.serialize(schema, new DataOutputViewStreamWrapper(out)); - } - } + serializeSchemaMap(schemaManager.evolvedSchemas, out); + serializeSchemaMap(schemaManager.upstreamSchemas, out); + out.writeUTF(schemaManager.getBehavior().name()); return baos.toByteArray(); } } @Override public SchemaManager deserialize(int version, byte[] serialized) throws IOException { - switch (version) { - case 0: - case 1: - TableIdSerializer tableIdSerializer = TableIdSerializer.INSTANCE; - SchemaSerializer schemaSerializer = SchemaSerializer.INSTANCE; - try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); - DataInputStream in = new DataInputStream(bais)) { - // Total schema length - int numTables = in.readInt(); - Map> tableSchemas = - new HashMap<>(numTables); - for (int i = 0; i < numTables; i++) { - // Table ID - TableId tableId = - tableIdSerializer.deserialize( - new DataInputViewStreamWrapper(in)); - // Schema with versions - int numVersions = in.readInt(); - SortedMap versionedSchemas = - new TreeMap<>(Integer::compareTo); - for (int j = 0; j < numVersions; j++) { - // Version - int schemaVersion = in.readInt(); - Schema schema = - schemaSerializer.deserialize( - version, new DataInputViewStreamWrapper(in)); - versionedSchemas.put(schemaVersion, schema); - } - tableSchemas.put(tableId, versionedSchemas); + try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(bais)) { + switch (version) { + case 0: + case 1: + { + Map> schemas = + deserializeSchemaMap(version, in); + // In legacy mode, upstream schema and evolved schema never differs + return new SchemaManager(schemas, schemas, SchemaChangeBehavior.EVOLVE); + } + case 2: + { + Map> evolvedSchemas = + deserializeSchemaMap(version, in); + Map> upstreamSchemas = + deserializeSchemaMap(version, in); + SchemaChangeBehavior behavior = + SchemaChangeBehavior.valueOf(in.readUTF()); + return new SchemaManager(upstreamSchemas, evolvedSchemas, behavior); } - return new SchemaManager(tableSchemas); - } - default: - throw new IOException("Unrecognized serialization version " + version); + default: + throw new RuntimeException("Unknown serialize version: " + version); + } + } + } + } + + private static void serializeSchemaMap( + Map> schemaMap, DataOutputStream out) + throws IOException { + TableIdSerializer tableIdSerializer = TableIdSerializer.INSTANCE; + SchemaSerializer schemaSerializer = SchemaSerializer.INSTANCE; + // Number of tables + out.writeInt(schemaMap.size()); + for (Map.Entry> tableSchema : schemaMap.entrySet()) { + // Table ID + TableId tableId = tableSchema.getKey(); + tableIdSerializer.serialize(tableId, new DataOutputViewStreamWrapper(out)); + + // Schema with versions + SortedMap versionedSchemas = tableSchema.getValue(); + out.writeInt(versionedSchemas.size()); + for (Map.Entry versionedSchema : versionedSchemas.entrySet()) { + // Version + Integer version = versionedSchema.getKey(); + out.writeInt(version); + // Schema + Schema schema = versionedSchema.getValue(); + schemaSerializer.serialize(schema, new DataOutputViewStreamWrapper(out)); + } + } + } + + private static Map> deserializeSchemaMap( + int version, DataInputStream in) throws IOException { + TableIdSerializer tableIdSerializer = TableIdSerializer.INSTANCE; + SchemaSerializer schemaSerializer = SchemaSerializer.INSTANCE; + // Total schema length + int numTables = in.readInt(); + Map> tableSchemas = new HashMap<>(numTables); + for (int i = 0; i < numTables; i++) { + // Table ID + TableId tableId = tableIdSerializer.deserialize(new DataInputViewStreamWrapper(in)); + // Schema with versions + int numVersions = in.readInt(); + SortedMap versionedSchemas = new TreeMap<>(Integer::compareTo); + for (int j = 0; j < numVersions; j++) { + // Version + int schemaVersion = in.readInt(); + Schema schema = + schemaSerializer.deserialize(version, new DataInputViewStreamWrapper(in)); + versionedSchemas.put(schemaVersion, schema); } + tableSchemas.put(tableId, versionedSchemas); } + return tableSchemas; } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaRegistry.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaRegistry.java index 2c718dcec43..68f5875a7da 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaRegistry.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaRegistry.java @@ -19,12 +19,19 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; import org.apache.flink.cdc.common.schema.Selectors; import org.apache.flink.cdc.common.sink.MetadataApplier; import org.apache.flink.cdc.runtime.operators.schema.SchemaOperator; +import org.apache.flink.cdc.runtime.operators.schema.event.ApplyEvolvedSchemaChangeRequest; +import org.apache.flink.cdc.runtime.operators.schema.event.ApplyEvolvedSchemaChangeResponse; +import org.apache.flink.cdc.runtime.operators.schema.event.ApplyUpstreamSchemaChangeRequest; +import org.apache.flink.cdc.runtime.operators.schema.event.ApplyUpstreamSchemaChangeResponse; import org.apache.flink.cdc.runtime.operators.schema.event.FlushSuccessEvent; -import org.apache.flink.cdc.runtime.operators.schema.event.GetSchemaRequest; -import org.apache.flink.cdc.runtime.operators.schema.event.GetSchemaResponse; +import org.apache.flink.cdc.runtime.operators.schema.event.GetEvolvedSchemaRequest; +import org.apache.flink.cdc.runtime.operators.schema.event.GetEvolvedSchemaResponse; +import org.apache.flink.cdc.runtime.operators.schema.event.GetUpstreamSchemaRequest; +import org.apache.flink.cdc.runtime.operators.schema.event.GetUpstreamSchemaResponse; import org.apache.flink.cdc.runtime.operators.schema.event.RefreshPendingListsRequest; import org.apache.flink.cdc.runtime.operators.schema.event.ReleaseUpstreamRequest; import org.apache.flink.cdc.runtime.operators.schema.event.SchemaChangeRequest; @@ -96,24 +103,37 @@ public class SchemaRegistry implements OperatorCoordinator, CoordinationRequestH private SchemaRegistryRequestHandler requestHandler; /** Schema manager for tracking schemas of all tables. */ - private SchemaManager schemaManager = new SchemaManager(); + private SchemaManager schemaManager; private SchemaDerivation schemaDerivation; + private SchemaChangeBehavior schemaChangeBehavior; + public SchemaRegistry( String operatorName, OperatorCoordinator.Context context, MetadataApplier metadataApplier, List> routes) { + this(operatorName, context, metadataApplier, routes, SchemaChangeBehavior.EVOLVE); + } + + public SchemaRegistry( + String operatorName, + OperatorCoordinator.Context context, + MetadataApplier metadataApplier, + List> routes, + SchemaChangeBehavior schemaChangeBehavior) { this.context = context; this.operatorName = operatorName; this.failedReasons = new HashMap<>(); this.metadataApplier = metadataApplier; this.routes = routes; - schemaManager = new SchemaManager(); - schemaDerivation = new SchemaDerivation(schemaManager, routes, new HashMap<>()); - requestHandler = - new SchemaRegistryRequestHandler(metadataApplier, schemaManager, schemaDerivation); + this.schemaManager = new SchemaManager(schemaChangeBehavior); + this.schemaDerivation = new SchemaDerivation(schemaManager, routes, new HashMap<>()); + this.requestHandler = + new SchemaRegistryRequestHandler( + metadataApplier, schemaManager, schemaDerivation, schemaChangeBehavior); + this.schemaChangeBehavior = schemaChangeBehavior; } @Override @@ -177,9 +197,22 @@ public CompletableFuture handleCoordinationRequest( return requestHandler.handleSchemaChangeRequest(schemaChangeRequest); } else if (request instanceof ReleaseUpstreamRequest) { return requestHandler.handleReleaseUpstreamRequest(); - } else if (request instanceof GetSchemaRequest) { + } else if (request instanceof GetEvolvedSchemaRequest) { return CompletableFuture.completedFuture( - wrap(handleGetSchemaRequest(((GetSchemaRequest) request)))); + wrap(handleGetEvolvedSchemaRequest(((GetEvolvedSchemaRequest) request)))); + } else if (request instanceof GetUpstreamSchemaRequest) { + return CompletableFuture.completedFuture( + wrap(handleGetUpstreamSchemaRequest((GetUpstreamSchemaRequest) request))); + } else if (request instanceof ApplyUpstreamSchemaChangeRequest) { + return CompletableFuture.completedFuture( + wrap( + handleApplyUpstreamSchemaChangeRequest( + (ApplyUpstreamSchemaChangeRequest) request))); + } else if (request instanceof ApplyEvolvedSchemaChangeRequest) { + return CompletableFuture.completedFuture( + wrap( + handleApplyEvolvedSchemaChangeRequest( + (ApplyEvolvedSchemaChangeRequest) request))); } else if (request instanceof SchemaChangeResultRequest) { return requestHandler.getSchemaChangeResult(); } else if (request instanceof RefreshPendingListsRequest) { @@ -198,6 +231,7 @@ public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData try (ByteArrayInputStream bais = new ByteArrayInputStream(checkpointData); DataInputStream in = new DataInputStream(bais)) { int schemaManagerSerializerVersion = in.readInt(); + switch (schemaManagerSerializerVersion) { case 0: { @@ -211,10 +245,14 @@ public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData new SchemaDerivation(schemaManager, routes, Collections.emptyMap()); requestHandler = new SchemaRegistryRequestHandler( - metadataApplier, schemaManager, schemaDerivation); + metadataApplier, + schemaManager, + schemaDerivation, + schemaManager.getBehavior()); break; } case 1: + case 2: { int length = in.readInt(); byte[] serializedSchemaManager = new byte[length]; @@ -228,7 +266,10 @@ public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData new SchemaDerivation(schemaManager, routes, derivationMapping); requestHandler = new SchemaRegistryRequestHandler( - metadataApplier, schemaManager, schemaDerivation); + metadataApplier, + schemaManager, + schemaDerivation, + schemaManager.getBehavior()); break; } default: @@ -258,22 +299,62 @@ public void executionAttemptReady( // do nothing } - private GetSchemaResponse handleGetSchemaRequest(GetSchemaRequest getSchemaRequest) { - LOG.info("Handling schema request: {}", getSchemaRequest); - int schemaVersion = getSchemaRequest.getSchemaVersion(); - TableId tableId = getSchemaRequest.getTableId(); - if (schemaVersion == GetSchemaRequest.LATEST_SCHEMA_VERSION) { - return new GetSchemaResponse(schemaManager.getLatestSchema(tableId).orElse(null)); + private GetEvolvedSchemaResponse handleGetEvolvedSchemaRequest( + GetEvolvedSchemaRequest getEvolvedSchemaRequest) { + LOG.info("Handling evolved schema request: {}", getEvolvedSchemaRequest); + int schemaVersion = getEvolvedSchemaRequest.getSchemaVersion(); + TableId tableId = getEvolvedSchemaRequest.getTableId(); + if (schemaVersion == GetEvolvedSchemaRequest.LATEST_SCHEMA_VERSION) { + return new GetEvolvedSchemaResponse( + schemaManager.getLatestEvolvedSchema(tableId).orElse(null)); } else { try { - return new GetSchemaResponse(schemaManager.getSchema(tableId, schemaVersion)); + return new GetEvolvedSchemaResponse( + schemaManager.getEvolvedSchema(tableId, schemaVersion)); } catch (IllegalArgumentException iae) { LOG.warn( - "Some client is requesting an non-existed schema for table {} with version {}", + "Some client is requesting an non-existed evolved schema for table {} with version {}", tableId, schemaVersion); - return new GetSchemaResponse(null); + return new GetEvolvedSchemaResponse(null); } } } + + private GetUpstreamSchemaResponse handleGetUpstreamSchemaRequest( + GetUpstreamSchemaRequest getUpstreamSchemaRequest) { + LOG.info("Handling upstream schema request: {}", getUpstreamSchemaRequest); + int schemaVersion = getUpstreamSchemaRequest.getSchemaVersion(); + TableId tableId = getUpstreamSchemaRequest.getTableId(); + if (schemaVersion == GetUpstreamSchemaRequest.LATEST_SCHEMA_VERSION) { + return new GetUpstreamSchemaResponse( + schemaManager.getLatestUpstreamSchema(tableId).orElse(null)); + } else { + try { + return new GetUpstreamSchemaResponse( + schemaManager.getUpstreamSchema(tableId, schemaVersion)); + } catch (IllegalArgumentException iae) { + LOG.warn( + "Some client is requesting an non-existed upstream schema for table {} with version {}", + tableId, + schemaVersion); + return new GetUpstreamSchemaResponse(null); + } + } + } + + private ApplyUpstreamSchemaChangeResponse handleApplyUpstreamSchemaChangeRequest( + ApplyUpstreamSchemaChangeRequest applyUpstreamSchemaChangeRequest) { + schemaManager.applyUpstreamSchemaChange( + applyUpstreamSchemaChangeRequest.getSchemaChangeEvent()); + return new ApplyUpstreamSchemaChangeResponse(); + } + + private ApplyEvolvedSchemaChangeResponse handleApplyEvolvedSchemaChangeRequest( + ApplyEvolvedSchemaChangeRequest applyEvolvedSchemaChangeRequest) { + applyEvolvedSchemaChangeRequest + .getSchemaChangeEvent() + .forEach(schemaManager::applyEvolvedSchemaChange); + return new ApplyEvolvedSchemaChangeResponse(); + } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaRegistryProvider.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaRegistryProvider.java index 1f6e7aaf57a..524b09dca3b 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaRegistryProvider.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaRegistryProvider.java @@ -20,6 +20,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.annotation.Internal; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; import org.apache.flink.cdc.common.schema.Selectors; import org.apache.flink.cdc.common.sink.MetadataApplier; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -37,16 +38,19 @@ public class SchemaRegistryProvider implements OperatorCoordinator.Provider { private final String operatorName; private final MetadataApplier metadataApplier; private final List> routingRules; + private final SchemaChangeBehavior schemaChangeBehavior; public SchemaRegistryProvider( OperatorID operatorID, String operatorName, MetadataApplier metadataApplier, - List> routingRules) { + List> routingRules, + SchemaChangeBehavior schemaChangeBehavior) { this.operatorID = operatorID; this.operatorName = operatorName; this.metadataApplier = metadataApplier; this.routingRules = routingRules; + this.schemaChangeBehavior = schemaChangeBehavior; } @Override @@ -69,6 +73,7 @@ public OperatorCoordinator create(OperatorCoordinator.Context context) throws Ex return new Tuple2<>(selectors, replaceBy); }) .collect(Collectors.toList()); - return new SchemaRegistry(operatorName, context, metadataApplier, routes); + return new SchemaRegistry( + operatorName, context, metadataApplier, routes, schemaChangeBehavior); } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaRegistryRequestHandler.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaRegistryRequestHandler.java index 1ee06a7e53f..87bc06adb58 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaRegistryRequestHandler.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaRegistryRequestHandler.java @@ -17,10 +17,14 @@ package org.apache.flink.cdc.runtime.operators.schema.coordinator; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cdc.common.annotation.Internal; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; +import org.apache.flink.cdc.common.event.SchemaChangeEventWithPreSchema; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; import org.apache.flink.cdc.common.sink.MetadataApplier; import org.apache.flink.cdc.runtime.operators.schema.event.RefreshPendingListsResponse; import org.apache.flink.cdc.runtime.operators.schema.event.ReleaseUpstreamRequest; @@ -69,28 +73,38 @@ public class SchemaRegistryRequestHandler implements Closeable { * sink writers. */ private final List pendingSchemaChanges; + + private final List finishedSchemaChanges; + private final List> failedSchemaChanges; + private final List ignoredSchemaChanges; + /** Sink writers which have sent flush success events for the request. */ private final Set flushedSinkWriters; /** Status of the execution of current schema change request. */ private boolean isSchemaChangeApplying; - /** Actual exception if failed to apply schema change. */ - private Exception schemaChangeException; /** Executor service to execute schema change. */ private final ExecutorService schemaChangeThreadPool; + private final SchemaChangeBehavior schemaChangeBehavior; + public SchemaRegistryRequestHandler( MetadataApplier metadataApplier, SchemaManager schemaManager, - SchemaDerivation schemaDerivation) { + SchemaDerivation schemaDerivation, + SchemaChangeBehavior schemaChangeBehavior) { this.metadataApplier = metadataApplier; this.activeSinkWriters = new HashSet<>(); this.flushedSinkWriters = new HashSet<>(); this.pendingSchemaChanges = new LinkedList<>(); + this.finishedSchemaChanges = new LinkedList<>(); + this.failedSchemaChanges = new LinkedList<>(); + this.ignoredSchemaChanges = new LinkedList<>(); this.schemaManager = schemaManager; this.schemaDerivation = schemaDerivation; - schemaChangeThreadPool = Executors.newSingleThreadExecutor(); - isSchemaChangeApplying = false; + this.schemaChangeThreadPool = Executors.newSingleThreadExecutor(); + this.isSchemaChangeApplying = false; + this.schemaChangeBehavior = schemaChangeBehavior; } /** @@ -102,21 +116,40 @@ public SchemaRegistryRequestHandler( private void applySchemaChange( TableId tableId, List derivedSchemaChangeEvents) { isSchemaChangeApplying = true; - schemaChangeException = null; - try { - for (SchemaChangeEvent changeEvent : derivedSchemaChangeEvents) { - metadataApplier.applySchemaChange(changeEvent); - LOG.debug("Apply schema change {} to table {}.", changeEvent, tableId); + finishedSchemaChanges.clear(); + failedSchemaChanges.clear(); + ignoredSchemaChanges.clear(); + + for (SchemaChangeEvent changeEvent : derivedSchemaChangeEvents) { + if (changeEvent.getType() != SchemaChangeEventType.CREATE_TABLE) { + if (schemaChangeBehavior == SchemaChangeBehavior.IGNORE) { + ignoredSchemaChanges.add(changeEvent); + continue; + } } - PendingSchemaChange waitFlushSuccess = pendingSchemaChanges.get(0); - if (RECEIVED_RELEASE_REQUEST.equals(waitFlushSuccess.getStatus())) { - startNextSchemaChangeRequest(); + if (!metadataApplier.acceptsSchemaEvolutionType(changeEvent.getType())) { + LOG.info("Ignored schema change {} to table {}.", changeEvent, tableId); + ignoredSchemaChanges.add(changeEvent); + } else { + try { + metadataApplier.applySchemaChange(changeEvent); + LOG.debug("Applied schema change {} to table {}.", changeEvent, tableId); + finishedSchemaChanges.add(changeEvent); + } catch (Throwable t) { + LOG.error( + "Failed to apply schema change {} to table {}. Caused by: {}", + changeEvent, + tableId, + t); + failedSchemaChanges.add(Tuple2.of(changeEvent, t)); + } } - } catch (Exception e) { - this.schemaChangeException = e; - } finally { - this.isSchemaChangeApplying = false; } + PendingSchemaChange waitFlushSuccess = pendingSchemaChanges.get(0); + if (RECEIVED_RELEASE_REQUEST.equals(waitFlushSuccess.getStatus())) { + startNextSchemaChangeRequest(); + } + isSchemaChangeApplying = false; } /** @@ -130,14 +163,27 @@ public CompletableFuture handleSchemaChangeRequest( LOG.info( "Received schema change event request from table {}. Start to buffer requests for others.", request.getTableId().toString()); - if (request.getSchemaChangeEvent() instanceof CreateTableEvent - && schemaManager.schemaExists(request.getTableId())) { + SchemaChangeEvent event = request.getSchemaChangeEvent(); + if (event instanceof CreateTableEvent + && schemaManager.upstreamSchemaExists(request.getTableId())) { return CompletableFuture.completedFuture( wrap(new SchemaChangeResponse(Collections.emptyList()))); } - schemaManager.applySchemaChange(request.getSchemaChangeEvent()); + schemaManager.applyUpstreamSchemaChange(event); List derivedSchemaChangeEvents = - schemaDerivation.applySchemaChange(request.getSchemaChangeEvent()); + schemaDerivation.applySchemaChange(event); + + derivedSchemaChangeEvents.forEach( + e -> { + if (e instanceof SchemaChangeEventWithPreSchema) { + SchemaChangeEventWithPreSchema pe = (SchemaChangeEventWithPreSchema) e; + if (!pe.hasPreSchema()) { + schemaManager + .getLatestEvolvedSchema(pe.tableId()) + .ifPresent(pe::fillPreSchema); + } + } + }); CompletableFuture response = CompletableFuture.completedFuture( wrap(new SchemaChangeResponse(derivedSchemaChangeEvents))); @@ -195,15 +241,20 @@ public void flushSuccess(TableId tableId, int sinkSubtask) throws InterruptedExc schemaChangeThreadPool.submit( () -> applySchemaChange(tableId, waitFlushSuccess.derivedSchemaChangeEvents)); Thread.sleep(1000); - if (schemaChangeException != null) { - throw new RuntimeException("failed to apply schema change.", schemaChangeException); - } + if (isSchemaChangeApplying) { waitFlushSuccess .getResponseFuture() .complete(wrap(new SchemaChangeProcessingResponse())); } else { - waitFlushSuccess.getResponseFuture().complete(wrap(new ReleaseUpstreamResponse())); + waitFlushSuccess + .getResponseFuture() + .complete( + wrap( + new ReleaseUpstreamResponse( + finishedSchemaChanges, + failedSchemaChanges, + ignoredSchemaChanges))); } } } @@ -215,13 +266,13 @@ private void startNextSchemaChangeRequest() { PendingSchemaChange pendingSchemaChange = pendingSchemaChanges.get(0); SchemaChangeRequest request = pendingSchemaChange.changeRequest; if (request.getSchemaChangeEvent() instanceof CreateTableEvent - && schemaManager.schemaExists(request.getTableId())) { + && schemaManager.upstreamSchemaExists(request.getTableId())) { pendingSchemaChange .getResponseFuture() .complete(wrap(new SchemaChangeResponse(Collections.emptyList()))); pendingSchemaChanges.remove(0); } else { - schemaManager.applySchemaChange(request.getSchemaChangeEvent()); + schemaManager.applyUpstreamSchemaChange(request.getSchemaChangeEvent()); List derivedSchemaChangeEvents = schemaDerivation.applySchemaChange(request.getSchemaChangeEvent()); pendingSchemaChange @@ -243,13 +294,16 @@ public CompletableFuture refreshPendingLists() { } public CompletableFuture getSchemaChangeResult() { - if (schemaChangeException != null) { - throw new RuntimeException("failed to apply schema change.", schemaChangeException); - } if (isSchemaChangeApplying) { return CompletableFuture.supplyAsync(() -> wrap(new SchemaChangeProcessingResponse())); } else { - return CompletableFuture.supplyAsync(() -> wrap(new ReleaseUpstreamResponse())); + return CompletableFuture.supplyAsync( + () -> + wrap( + new ReleaseUpstreamResponse( + finishedSchemaChanges, + failedSchemaChanges, + ignoredSchemaChanges))); } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ApplyEvolvedSchemaChangeRequest.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ApplyEvolvedSchemaChangeRequest.java new file mode 100644 index 00000000000..f6798af5196 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ApplyEvolvedSchemaChangeRequest.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * 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.operators.schema.event; + +import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.runtime.operators.schema.SchemaOperator; +import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaRegistry; +import org.apache.flink.runtime.operators.coordination.CoordinationRequest; + +import java.util.List; +import java.util.Objects; + +/** + * The request from {@link SchemaOperator} to {@link SchemaRegistry} to request apply evolved schema + * changes. + */ +public class ApplyEvolvedSchemaChangeRequest implements CoordinationRequest { + + private static final long serialVersionUID = 1L; + + /** The sender of the request. */ + private final TableId tableId; + /** The schema changes. */ + private final List schemaChangeEvent; + + public ApplyEvolvedSchemaChangeRequest( + TableId tableId, List schemaChangeEvent) { + this.tableId = tableId; + this.schemaChangeEvent = schemaChangeEvent; + } + + public TableId getTableId() { + return tableId; + } + + public List getSchemaChangeEvent() { + return schemaChangeEvent; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof ApplyEvolvedSchemaChangeRequest)) { + return false; + } + ApplyEvolvedSchemaChangeRequest that = (ApplyEvolvedSchemaChangeRequest) o; + return Objects.equals(tableId, that.tableId) + && Objects.equals(schemaChangeEvent, that.schemaChangeEvent); + } + + @Override + public int hashCode() { + return Objects.hash(tableId, schemaChangeEvent); + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ApplyEvolvedSchemaChangeResponse.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ApplyEvolvedSchemaChangeResponse.java new file mode 100644 index 00000000000..6f87ce9544a --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ApplyEvolvedSchemaChangeResponse.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.runtime.operators.schema.event; + +import org.apache.flink.cdc.runtime.operators.schema.SchemaOperator; +import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaRegistry; +import org.apache.flink.runtime.operators.coordination.CoordinationResponse; + +/** + * The response from {@link SchemaOperator} to {@link SchemaRegistry} to request apply evolved + * schema changes. + */ +public class ApplyEvolvedSchemaChangeResponse implements CoordinationResponse { + + private static final long serialVersionUID = 1L; +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ApplyUpstreamSchemaChangeRequest.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ApplyUpstreamSchemaChangeRequest.java new file mode 100644 index 00000000000..c24075f174c --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ApplyUpstreamSchemaChangeRequest.java @@ -0,0 +1,71 @@ +/* + * 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.operators.schema.event; + +import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.runtime.operators.schema.SchemaOperator; +import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaRegistry; +import org.apache.flink.runtime.operators.coordination.CoordinationRequest; + +import java.util.Objects; + +/** + * The request from {@link SchemaOperator} to {@link SchemaRegistry} to request apply evolved schema + * changes. + */ +public class ApplyUpstreamSchemaChangeRequest implements CoordinationRequest { + + private static final long serialVersionUID = 1L; + + /** The sender of the request. */ + private final TableId tableId; + /** The schema changes. */ + private final SchemaChangeEvent schemaChangeEvent; + + public ApplyUpstreamSchemaChangeRequest(TableId tableId, SchemaChangeEvent schemaChangeEvent) { + this.tableId = tableId; + this.schemaChangeEvent = schemaChangeEvent; + } + + public TableId getTableId() { + return tableId; + } + + public SchemaChangeEvent getSchemaChangeEvent() { + return schemaChangeEvent; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof ApplyUpstreamSchemaChangeRequest)) { + return false; + } + ApplyUpstreamSchemaChangeRequest that = (ApplyUpstreamSchemaChangeRequest) o; + return Objects.equals(tableId, that.tableId) + && Objects.equals(schemaChangeEvent, that.schemaChangeEvent); + } + + @Override + public int hashCode() { + return Objects.hash(tableId, schemaChangeEvent); + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ApplyUpstreamSchemaChangeResponse.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ApplyUpstreamSchemaChangeResponse.java new file mode 100644 index 00000000000..8d201a40417 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ApplyUpstreamSchemaChangeResponse.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.runtime.operators.schema.event; + +import org.apache.flink.cdc.runtime.operators.schema.SchemaOperator; +import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaRegistry; +import org.apache.flink.runtime.operators.coordination.CoordinationResponse; + +/** + * The response from {@link SchemaOperator} to {@link SchemaRegistry} to request apply evolved + * schema changes. + */ +public class ApplyUpstreamSchemaChangeResponse implements CoordinationResponse { + + private static final long serialVersionUID = 1L; +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetSchemaRequest.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetEvolvedSchemaRequest.java similarity index 84% rename from flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetSchemaRequest.java rename to flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetEvolvedSchemaRequest.java index aac440ba59c..f5b7a58607a 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetSchemaRequest.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetEvolvedSchemaRequest.java @@ -24,17 +24,17 @@ /** Request to {@link SchemaRegistry} for getting schema of a table. */ @Internal -public class GetSchemaRequest implements CoordinationRequest { +public class GetEvolvedSchemaRequest implements CoordinationRequest { public static final int LATEST_SCHEMA_VERSION = -1; private final TableId tableId; private final int schemaVersion; - public static GetSchemaRequest ofLatestSchema(TableId tableId) { - return new GetSchemaRequest(tableId, LATEST_SCHEMA_VERSION); + public static GetEvolvedSchemaRequest ofLatestSchema(TableId tableId) { + return new GetEvolvedSchemaRequest(tableId, LATEST_SCHEMA_VERSION); } - public GetSchemaRequest(TableId tableId, int schemaVersion) { + public GetEvolvedSchemaRequest(TableId tableId, int schemaVersion) { this.tableId = tableId; this.schemaVersion = schemaVersion; } @@ -49,7 +49,7 @@ public int getSchemaVersion() { @Override public String toString() { - return "GetSchemaRequest{" + return "GetEvolvedSchemaRequest{" + "tableId=" + tableId + ", schemaVersion=" diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetSchemaResponse.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetEvolvedSchemaResponse.java similarity index 89% rename from flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetSchemaResponse.java rename to flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetEvolvedSchemaResponse.java index aec9fd4de48..81fef92d7e9 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetSchemaResponse.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetEvolvedSchemaResponse.java @@ -26,12 +26,12 @@ import java.util.Optional; -/** Coordination response from {@link SchemaRegistry} for {@link GetSchemaRequest}. */ +/** Coordination response from {@link SchemaRegistry} for {@link GetEvolvedSchemaRequest}. */ @Internal -public class GetSchemaResponse implements CoordinationResponse { +public class GetEvolvedSchemaResponse implements CoordinationResponse { @Nullable private final Schema schema; - public GetSchemaResponse(@Nullable Schema schema) { + public GetEvolvedSchemaResponse(@Nullable Schema schema) { this.schema = schema; } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetUpstreamSchemaRequest.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetUpstreamSchemaRequest.java new file mode 100644 index 00000000000..eb66a1ae419 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetUpstreamSchemaRequest.java @@ -0,0 +1,59 @@ +/* + * 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.operators.schema.event; + +import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaRegistry; +import org.apache.flink.runtime.operators.coordination.CoordinationRequest; + +/** Request to {@link SchemaRegistry} for getting upstream schema of a table. */ +@Internal +public class GetUpstreamSchemaRequest implements CoordinationRequest { + public static final int LATEST_SCHEMA_VERSION = -1; + + private final TableId tableId; + private final int schemaVersion; + + public static GetUpstreamSchemaRequest ofLatestSchema(TableId tableId) { + return new GetUpstreamSchemaRequest(tableId, LATEST_SCHEMA_VERSION); + } + + public GetUpstreamSchemaRequest(TableId tableId, int schemaVersion) { + this.tableId = tableId; + this.schemaVersion = schemaVersion; + } + + public TableId getTableId() { + return tableId; + } + + public int getSchemaVersion() { + return schemaVersion; + } + + @Override + public String toString() { + return "GetUpstreamSchemaRequest{" + + "tableId=" + + tableId + + ", schemaVersion=" + + schemaVersion + + '}'; + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetUpstreamSchemaResponse.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetUpstreamSchemaResponse.java new file mode 100644 index 00000000000..c44ac2e06a7 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetUpstreamSchemaResponse.java @@ -0,0 +1,41 @@ +/* + * 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.operators.schema.event; + +import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.schema.Schema; +import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaRegistry; +import org.apache.flink.runtime.operators.coordination.CoordinationResponse; + +import javax.annotation.Nullable; + +import java.util.Optional; + +/** Coordination response from {@link SchemaRegistry} for {@link GetUpstreamSchemaRequest}. */ +@Internal +public class GetUpstreamSchemaResponse implements CoordinationResponse { + @Nullable private final Schema schema; + + public GetUpstreamSchemaResponse(@Nullable Schema schema) { + this.schema = schema; + } + + public Optional getSchema() { + return Optional.ofNullable(schema); + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ReleaseUpstreamResponse.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ReleaseUpstreamResponse.java index 0b0043235ef..f577f1120fd 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ReleaseUpstreamResponse.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ReleaseUpstreamResponse.java @@ -17,10 +17,16 @@ package org.apache.flink.cdc.runtime.operators.schema.event; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.runtime.operators.schema.SchemaOperator; import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaRegistry; import org.apache.flink.runtime.operators.coordination.CoordinationResponse; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + /** * The response for {@link ReleaseUpstreamRequest} from {@link SchemaRegistry} to {@link * SchemaOperator}. @@ -28,4 +34,77 @@ public class ReleaseUpstreamResponse implements CoordinationResponse { private static final long serialVersionUID = 1L; + + /** + * Whether the SchemaOperator need to buffer data and the SchemaOperatorCoordinator need to wait + * for flushing. + */ + private final List finishedSchemaChangeEvents; + + private final List> failedSchemaChangeEvents; + + private final List ignoredSchemaChangeEvents; + + public ReleaseUpstreamResponse( + List finishedSchemaChangeEvents, + List> failedSchemaChangeEvents, + List ignoredSchemaChangeEvents) { + this.finishedSchemaChangeEvents = finishedSchemaChangeEvents; + this.failedSchemaChangeEvents = failedSchemaChangeEvents; + this.ignoredSchemaChangeEvents = ignoredSchemaChangeEvents; + } + + public List getFinishedSchemaChangeEvents() { + return finishedSchemaChangeEvents; + } + + public List> getFailedSchemaChangeEvents() { + return failedSchemaChangeEvents; + } + + public List getIgnoredSchemaChangeEvents() { + return ignoredSchemaChangeEvents; + } + + public String getPrintableFailedSchemaChangeEvents() { + return failedSchemaChangeEvents.stream() + .map(e -> "Failed to apply " + e.f0 + ". Caused by: " + e.f1) + .collect(Collectors.joining("\n")); + } + + public boolean hasException() { + return !failedSchemaChangeEvents.isEmpty(); + } + + @Override + public String toString() { + return "ReleaseUpstreamResponse{" + + "finishedSchemaChangeEvents=" + + finishedSchemaChangeEvents + + ", failedSchemaChangeEvents=" + + failedSchemaChangeEvents + + ", ignoredSchemaChangeEvents=" + + ignoredSchemaChangeEvents + + '}'; + } + + @Override + public boolean equals(Object object) { + if (this == object) { + return true; + } + if (object == null || getClass() != object.getClass()) { + return false; + } + ReleaseUpstreamResponse that = (ReleaseUpstreamResponse) object; + return Objects.equals(finishedSchemaChangeEvents, that.finishedSchemaChangeEvents) + && Objects.equals(failedSchemaChangeEvents, that.failedSchemaChangeEvents) + && Objects.equals(ignoredSchemaChangeEvents, that.ignoredSchemaChangeEvents); + } + + @Override + public int hashCode() { + return Objects.hash( + finishedSchemaChangeEvents, failedSchemaChangeEvents, ignoredSchemaChangeEvents); + } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/metrics/SchemaOperatorMetrics.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/metrics/SchemaOperatorMetrics.java new file mode 100644 index 00000000000..f650b6cacdb --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/metrics/SchemaOperatorMetrics.java @@ -0,0 +1,86 @@ +/* + * 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.operators.schema.metrics; + +import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; +import org.apache.flink.cdc.runtime.operators.schema.SchemaOperator; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.MetricGroup; + +import java.util.HashMap; +import java.util.Map; + +/** A collection class for handling metrics in {@link SchemaOperator}. */ +public class SchemaOperatorMetrics { + + /** Current schema change behavior. */ + public static final String SCHEMA_CHANGE_BEHAVIOR = "schemaChangeBehavior"; + + public static final Map SCHEMA_CHANGE_BEHAVIOR_INTEGER_MAP = + new HashMap() { + { + put(SchemaChangeBehavior.IGNORE, 0); + put(SchemaChangeBehavior.TRY_EVOLVE, 1); + put(SchemaChangeBehavior.EVOLVE, 2); + put(SchemaChangeBehavior.EXCEPTION, 3); + } + }; + + /** Total count of schema change events received. */ + public static final String NUM_SCHEMA_CHANGE_EVENTS = "numSchemaChangeEvents"; + + /** Number of successfully applied schema change events. */ + public static final String NUM_FINISHED_SCHEMA_CHANGE_EVENTS = "numFinishedSchemaChangeEvents"; + + /** Number of schema change events that failed to apply. */ + public static final String NUM_FAILED_SCHEMA_CHANGE_EVENTS = "numFailedSchemaChangeEvents"; + + /** Number of schema change events ignored. */ + public static final String NUM_IGNORED_SCHEMA_CHANGE_EVENTS = "numIgnoredSchemaChangeEvents"; + + private final Counter numSchemaChangeEventsCounter; + private final Counter numFinishedSchemaChangeEventsCounter; + private final Counter numFailedSchemaChangeEventsCounter; + private final Counter numIgnoredSchemaChangeEventsCounter; + + public SchemaOperatorMetrics(MetricGroup metricGroup, SchemaChangeBehavior behavior) { + numSchemaChangeEventsCounter = metricGroup.counter(NUM_SCHEMA_CHANGE_EVENTS); + numFinishedSchemaChangeEventsCounter = + metricGroup.counter(NUM_FINISHED_SCHEMA_CHANGE_EVENTS); + numFailedSchemaChangeEventsCounter = metricGroup.counter(NUM_FAILED_SCHEMA_CHANGE_EVENTS); + numIgnoredSchemaChangeEventsCounter = metricGroup.counter(NUM_IGNORED_SCHEMA_CHANGE_EVENTS); + metricGroup.gauge( + SCHEMA_CHANGE_BEHAVIOR, () -> SCHEMA_CHANGE_BEHAVIOR_INTEGER_MAP.get(behavior)); + } + + public void increaseSchemaChangeEvents(long count) { + numSchemaChangeEventsCounter.inc(count); + } + + public void increaseFinishedSchemaChangeEvents(long count) { + numFinishedSchemaChangeEventsCounter.inc(count); + } + + public void increaseFailedSchemaChangeEvents(long count) { + numFailedSchemaChangeEventsCounter.inc(count); + } + + public void increaseIgnoredSchemaChangeEvents(long count) { + numIgnoredSchemaChangeEventsCounter.inc(count); + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkFunctionOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkFunctionOperator.java index 438c3f302d5..3767ba32123 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkFunctionOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkFunctionOperator.java @@ -117,7 +117,7 @@ private void handleFlushEvent(FlushEvent event) throws Exception { } private void emitLatestSchema(TableId tableId) throws Exception { - Optional schema = schemaEvolutionClient.getLatestSchema(tableId); + Optional schema = schemaEvolutionClient.getLatestEvolvedSchema(tableId); if (schema.isPresent()) { // request and process CreateTableEvent because SinkFunction need to retrieve // Schema to deserialize RecordData after resuming job. diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperator.java index bdb384cbb79..554d7970272 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperator.java @@ -203,7 +203,7 @@ private void handleFlushEvent(FlushEvent event) throws Exception { } private void emitLatestSchema(TableId tableId) throws Exception { - Optional schema = schemaEvolutionClient.getLatestSchema(tableId); + Optional schema = schemaEvolutionClient.getLatestEvolvedSchema(tableId); if (schema.isPresent()) { // request and process CreateTableEvent because SinkWriter need to retrieve // Schema to deserialize RecordData after resuming job. diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/SchemaEvolutionClient.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/SchemaEvolutionClient.java index 4ef5a4dea7c..72846b78ce0 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/SchemaEvolutionClient.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/SchemaEvolutionClient.java @@ -22,8 +22,10 @@ import org.apache.flink.cdc.runtime.operators.schema.SchemaOperator; import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaRegistry; import org.apache.flink.cdc.runtime.operators.schema.event.FlushSuccessEvent; -import org.apache.flink.cdc.runtime.operators.schema.event.GetSchemaRequest; -import org.apache.flink.cdc.runtime.operators.schema.event.GetSchemaResponse; +import org.apache.flink.cdc.runtime.operators.schema.event.GetEvolvedSchemaRequest; +import org.apache.flink.cdc.runtime.operators.schema.event.GetEvolvedSchemaResponse; +import org.apache.flink.cdc.runtime.operators.schema.event.GetUpstreamSchemaRequest; +import org.apache.flink.cdc.runtime.operators.schema.event.GetUpstreamSchemaResponse; import org.apache.flink.cdc.runtime.operators.schema.event.SinkWriterRegisterEvent; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway; @@ -63,15 +65,27 @@ public void notifyFlushSuccess(int subtask, TableId tableId) throws IOException schemaOperatorID, new SerializedValue<>(new FlushSuccessEvent(subtask, tableId))); } - public Optional getLatestSchema(TableId tableId) throws Exception { - GetSchemaResponse getSchemaResponse = + public Optional getLatestEvolvedSchema(TableId tableId) throws Exception { + GetEvolvedSchemaResponse getEvolvedSchemaResponse = unwrap( toCoordinator .sendRequestToCoordinator( schemaOperatorID, new SerializedValue<>( - GetSchemaRequest.ofLatestSchema(tableId))) + GetEvolvedSchemaRequest.ofLatestSchema(tableId))) .get()); - return getSchemaResponse.getSchema(); + return getEvolvedSchemaResponse.getSchema(); + } + + public Optional getLatestUpstreamSchema(TableId tableId) throws Exception { + GetUpstreamSchemaResponse getUpstreamSchemaResponse = + unwrap( + toCoordinator + .sendRequestToCoordinator( + schemaOperatorID, + new SerializedValue<>( + GetUpstreamSchemaRequest.ofLatestSchema(tableId))) + .get()); + return getUpstreamSchemaResponse.getSchema(); } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TableChangeInfo.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TableChangeInfo.java index 7fab9c65ad7..fa04fa50a5f 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TableChangeInfo.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/TableChangeInfo.java @@ -109,7 +109,7 @@ public static TableChangeInfo of( /** Serializer for {@link TableChangeInfo}. */ public static class Serializer implements SimpleVersionedSerializer { - public static final int CURRENT_VERSION = 1; + public static final int CURRENT_VERSION = 2; @Override public int getVersion() { 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 20479c87322..532b8f17991 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 @@ -21,7 +21,6 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple4; import org.apache.flink.cdc.common.data.binary.BinaryRecordData; -import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DataChangeEvent; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.SchemaChangeEvent; @@ -195,24 +194,35 @@ public void processElement(StreamRecord element) throws Exception { } private SchemaChangeEvent cacheSchema(SchemaChangeEvent event) throws Exception { - TableId tableId = event.tableId(); - Schema newSchema; - if (event instanceof CreateTableEvent) { - newSchema = ((CreateTableEvent) event).getSchema(); + Tuple2 appliedSchema; + appliedSchema = + SchemaUtils.applySchemaChangeEvent( + event, + getTableInfoFromSchemaEvolutionClientNullable(event.tableId()) + .map(TableInfo::getSchema) + .orElse(null)); + if (appliedSchema.f1 != null) { + transformSchema(appliedSchema.f0, appliedSchema.f1); + tableInfoMap.put(appliedSchema.f0, TableInfo.of(appliedSchema.f0, appliedSchema.f1)); } else { - newSchema = - SchemaUtils.applySchemaChangeEvent( - getTableInfoFromSchemaEvolutionClient(tableId).getSchema(), event); + tableInfoMap.remove(appliedSchema.f0); } - transformSchema(tableId, newSchema); - tableInfoMap.put(tableId, TableInfo.of(tableId, newSchema)); return event; } + private Optional getTableInfoFromSchemaEvolutionClientNullable(TableId tableId) { + try { + return Optional.of(getTableInfoFromSchemaEvolutionClient(tableId)); + } catch (Throwable t) { + return Optional.empty(); + } + } + private TableInfo getTableInfoFromSchemaEvolutionClient(TableId tableId) throws Exception { TableInfo tableInfo = tableInfoMap.get(tableId); if (tableInfo == null) { - Optional schemaOptional = schemaEvolutionClient.getLatestSchema(tableId); + Optional schemaOptional = + schemaEvolutionClient.getLatestUpstreamSchema(tableId); if (schemaOptional.isPresent()) { tableInfo = TableInfo.of(tableId, schemaOptional.get()); } else { 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..5ca5484cb5e 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 @@ -23,11 +23,18 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple5; import org.apache.flink.cdc.common.data.binary.BinaryRecordData; +import org.apache.flink.cdc.common.event.AlterTableCommentEvent; +import org.apache.flink.cdc.common.event.ColumnSchemaChangeEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DataChangeEvent; +import org.apache.flink.cdc.common.event.DropTableEvent; import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.event.RenameTableEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TableSchemaChangeEvent; +import org.apache.flink.cdc.common.event.TableSchemaChangeEventVisitorVoid; +import org.apache.flink.cdc.common.event.TruncateTableEvent; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.schema.Selectors; import org.apache.flink.cdc.common.utils.SchemaUtils; @@ -164,11 +171,37 @@ public void close() throws Exception { @Override public void processElement(StreamRecord element) throws Exception { Event event = element.getValue(); - if (event instanceof CreateTableEvent) { - event = cacheCreateTable((CreateTableEvent) event); - output.collect(new StreamRecord<>(event)); - } else if (event instanceof SchemaChangeEvent) { - event = cacheChangeSchema((SchemaChangeEvent) event); + if (event instanceof TableSchemaChangeEvent) { + ((TableSchemaChangeEvent) event) + .visit( + new TableSchemaChangeEventVisitorVoid() { + @Override + public void visit(AlterTableCommentEvent event) { + output.collect(new StreamRecord<>(event)); + } + + @Override + public void visit(CreateTableEvent event) { + output.collect(new StreamRecord<>(cacheCreateTable(event))); + } + + @Override + public void visit(DropTableEvent event) { + output.collect(new StreamRecord<>(event)); + } + + @Override + public void visit(RenameTableEvent event) { + output.collect(new StreamRecord<>(cacheRenameTable(event))); + } + + @Override + public void visit(TruncateTableEvent event) { + output.collect(new StreamRecord<>(event)); + } + }); + } else if (event instanceof ColumnSchemaChangeEvent) { + cacheChangeSchema((ColumnSchemaChangeEvent) event); output.collect(new StreamRecord<>(event)); } else if (event instanceof DataChangeEvent) { output.collect(new StreamRecord<>(processDataChangeEvent(((DataChangeEvent) event)))); @@ -184,14 +217,25 @@ private SchemaChangeEvent cacheCreateTable(CreateTableEvent event) { return event; } - private SchemaChangeEvent cacheChangeSchema(SchemaChangeEvent event) { + private SchemaChangeEvent cacheRenameTable(RenameTableEvent event) { TableId tableId = event.tableId(); TableChangeInfo tableChangeInfo = tableChangeInfoMap.get(tableId); - Schema originalSchema = - SchemaUtils.applySchemaChangeEvent(tableChangeInfo.getOriginalSchema(), event); - Schema newSchema = - SchemaUtils.applySchemaChangeEvent(tableChangeInfo.getTransformedSchema(), event); - tableChangeInfoMap.put(tableId, TableChangeInfo.of(tableId, originalSchema, newSchema)); + tableChangeInfoMap.put(event.newTableId(), tableChangeInfo); + return event; + } + + private ColumnSchemaChangeEvent cacheChangeSchema(ColumnSchemaChangeEvent event) { + TableId tableId = event.tableId(); + TableChangeInfo tableChangeInfo = tableChangeInfoMap.get(tableId); + Tuple2 originalSchema = + SchemaUtils.applySchemaChangeEvent(event, tableChangeInfo.getOriginalSchema()); + Tuple2 newSchema = + SchemaUtils.applySchemaChangeEvent(event, tableChangeInfo.getTransformedSchema()); + if (newSchema.f1 != null) { + tableChangeInfoMap.put( + newSchema.f0, + TableChangeInfo.of(originalSchema.f0, originalSchema.f1, newSchema.f1)); + } return event; } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/PrePartitionOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/PrePartitionOperator.java index 4097abe737e..915b9206131 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/PrePartitionOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/PrePartitionOperator.java @@ -113,7 +113,7 @@ private void broadcastEvent(Event toBroadcast) { private Schema loadLatestSchemaFromRegistry(TableId tableId) { Optional schema; try { - schema = schemaEvolutionClient.getLatestSchema(tableId); + schema = schemaEvolutionClient.getLatestUpstreamSchema(tableId); } catch (Exception e) { throw new RuntimeException( String.format("Failed to request latest schema for table \"%s\"", tableId), e); 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..11c71c1bcb8 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/SchemaChangeEventSerializer.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/SchemaChangeEventSerializer.java @@ -21,12 +21,20 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.cdc.common.event.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnCommentEvent; import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; +import org.apache.flink.cdc.common.event.AlterTableCommentEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.DropTableEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; +import org.apache.flink.cdc.common.event.RenameTableEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; +import org.apache.flink.cdc.common.event.SchemaChangeEventVisitor; +import org.apache.flink.cdc.common.event.SchemaChangeEventVisitorVoid; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.event.TruncateTableEvent; import org.apache.flink.cdc.runtime.serializer.EnumSerializer; import org.apache.flink.cdc.runtime.serializer.TypeSerializerSingleton; import org.apache.flink.core.memory.DataInputView; @@ -34,6 +42,17 @@ import java.io.IOException; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ADD_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ALTER_COLUMN_COMMENT; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ALTER_COLUMN_TYPE; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.ALTER_TABLE_COMMENT; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.CREATE_TABLE; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.DROP_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.DROP_TABLE; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.RENAME_COLUMN; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.RENAME_TABLE; +import static org.apache.flink.cdc.common.event.SchemaChangeEventType.TRUNCATE_TABLE; + /** A {@link TypeSerializer} for {@link SchemaChangeEvent}. */ public final class SchemaChangeEventSerializer extends TypeSerializerSingleton { @@ -42,8 +61,8 @@ public final class SchemaChangeEventSerializer extends TypeSerializerSingleton enumSerializer = - new EnumSerializer<>(SchemaChangeEventClass.class); + private final EnumSerializer enumSerializer = + new EnumSerializer<>(SchemaChangeEventType.class); @Override public boolean isImmutableType() { @@ -52,24 +71,74 @@ public boolean isImmutableType() { @Override public SchemaChangeEvent createInstance() { - return () -> TableId.tableId("unknown", "unknown", "unknown"); + return new SchemaChangeEvent() { + @Override + public SchemaChangeEventType getType() { + return null; + } + + @Override + public TableId tableId() { + return TableId.tableId("unknown", "unknown", "unknown"); + } + }; } @Override public SchemaChangeEvent copy(SchemaChangeEvent from) { - if (from instanceof AlterColumnTypeEvent) { - return AlterColumnTypeEventSerializer.INSTANCE.copy((AlterColumnTypeEvent) from); - } else if (from instanceof CreateTableEvent) { - return CreateTableEventSerializer.INSTANCE.copy((CreateTableEvent) from); - } else if (from instanceof RenameColumnEvent) { - return RenameColumnEventSerializer.INSTANCE.copy((RenameColumnEvent) from); - } else if (from instanceof AddColumnEvent) { - return AddColumnEventSerializer.INSTANCE.copy((AddColumnEvent) from); - } else if (from instanceof DropColumnEvent) { - return DropColumnEventSerializer.INSTANCE.copy((DropColumnEvent) from); - } else { - throw new IllegalArgumentException("Unknown schema change event: " + from); - } + return from.visit( + new SchemaChangeEventVisitor() { + + @Override + public SchemaChangeEvent visit(AddColumnEvent event) { + return AddColumnEventSerializer.INSTANCE.copy(event); + } + + @Override + public SchemaChangeEvent visit(AlterColumnCommentEvent event) { + return AlterColumnCommentEventSerializer.INSTANCE.copy(event); + } + + @Override + public SchemaChangeEvent visit(AlterColumnTypeEvent event) { + return AlterColumnTypeEventSerializer.INSTANCE.copy(event); + } + + @Override + public SchemaChangeEvent visit(AlterTableCommentEvent event) { + return AlterTableCommentEventSerializer.INSTANCE.copy(event); + } + + @Override + public SchemaChangeEvent visit(CreateTableEvent event) { + return CreateTableEventSerializer.INSTANCE.copy(event); + } + + @Override + public SchemaChangeEvent visit(DropColumnEvent event) { + return DropColumnEventSerializer.INSTANCE.copy(event); + } + + @Override + public SchemaChangeEvent visit(DropTableEvent event) { + return DropTableEventSerializer.INSTANCE.copy(event); + } + + @Override + public SchemaChangeEvent visit(RenameColumnEvent event) { + return RenameColumnEventSerializer.INSTANCE.copy(event); + } + + @Override + public SchemaChangeEvent visit(RenameTableEvent event) { + return RenameTableEventSerializer.INSTANCE.copy(event); + } + + @Override + public SchemaChangeEvent visit(TruncateTableEvent event) { + return TruncateTableEventSerializer.INSTANCE.copy(event); + } + }); } @Override @@ -84,31 +153,76 @@ 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); - } + + record.visit( + new SchemaChangeEventVisitorVoid() { + + @Override + public void visit(AddColumnEvent event) throws Exception { + enumSerializer.serialize(ADD_COLUMN, target); + AddColumnEventSerializer.INSTANCE.serialize(event, target); + } + + @Override + public void visit(AlterColumnCommentEvent event) throws Exception { + enumSerializer.serialize(ALTER_COLUMN_COMMENT, target); + AlterColumnCommentEventSerializer.INSTANCE.serialize(event, target); + } + + @Override + public void visit(AlterColumnTypeEvent event) throws Exception { + enumSerializer.serialize(ALTER_COLUMN_TYPE, target); + AlterColumnTypeEventSerializer.INSTANCE.serialize(event, target); + } + + @Override + public void visit(AlterTableCommentEvent event) throws Exception { + enumSerializer.serialize(ALTER_TABLE_COMMENT, target); + AlterTableCommentEventSerializer.INSTANCE.serialize(event, target); + } + + @Override + public void visit(CreateTableEvent event) throws Exception { + enumSerializer.serialize(CREATE_TABLE, target); + CreateTableEventSerializer.INSTANCE.serialize(event, target); + } + + @Override + public void visit(DropColumnEvent event) throws Exception { + enumSerializer.serialize(DROP_COLUMN, target); + DropColumnEventSerializer.INSTANCE.serialize(event, target); + } + + @Override + public void visit(DropTableEvent event) throws Exception { + enumSerializer.serialize(DROP_TABLE, target); + DropTableEventSerializer.INSTANCE.serialize(event, target); + } + + @Override + public void visit(RenameColumnEvent event) throws Exception { + enumSerializer.serialize(RENAME_COLUMN, target); + RenameColumnEventSerializer.INSTANCE.serialize(event, target); + } + + @Override + public void visit(RenameTableEvent event) throws Exception { + enumSerializer.serialize(RENAME_TABLE, target); + RenameTableEventSerializer.INSTANCE.serialize(event, target); + } + + @Override + public void visit(TruncateTableEvent event) throws Exception { + enumSerializer.serialize(TRUNCATE_TABLE, target); + TruncateTableEventSerializer.INSTANCE.serialize(event, target); + } + }); } @Override public SchemaChangeEvent deserialize(DataInputView source) throws IOException { - SchemaChangeEventClass schemaChangeEventClass = enumSerializer.deserialize(source); - switch (schemaChangeEventClass) { + SchemaChangeEventType schemaChangeEventType = enumSerializer.deserialize(source); + switch (schemaChangeEventType) { case ADD_COLUMN: return AddColumnEventSerializer.INSTANCE.deserialize(source); case DROP_COLUMN: @@ -119,9 +233,17 @@ 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); + "Unknown schema change event class: " + schemaChangeEventType); } } @@ -150,12 +272,4 @@ public SchemaChangeEventSerializerSnapshot() { super(() -> INSTANCE); } } - - enum SchemaChangeEventClass { - ALTER_COLUMN_TYPE, - RENAME_COLUMN, - ADD_COLUMN, - DROP_COLUMN, - CREATE_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/main/java/org/apache/flink/cdc/runtime/serializer/schema/SchemaSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/schema/SchemaSerializer.java index 8e2a4fa02f9..60646cafe24 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/schema/SchemaSerializer.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/schema/SchemaSerializer.java @@ -89,7 +89,7 @@ public void serialize(Schema record, DataOutputView target) throws IOException { stringSerializer.serialize(record.comment(), target); } - private static final int CURRENT_VERSION = 1; + private static final int CURRENT_VERSION = 2; @Override public Schema deserialize(DataInputView source) throws IOException { @@ -106,6 +106,7 @@ public Schema deserialize(int version, DataInputView source) throws IOException .comment(stringSerializer.deserialize(source)) .build(); case 1: + case 2: return Schema.newBuilder() .setColumns(columnsSerializer.deserialize(source)) .primaryKey(primaryKeysSerializer.deserialize(source)) diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/SchemaEvolveTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/SchemaEvolveTest.java new file mode 100644 index 00000000000..5f6e0fe6f46 --- /dev/null +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/SchemaEvolveTest.java @@ -0,0 +1,1694 @@ +/* + * 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.operators.schema; + +import org.apache.flink.calcite.shaded.com.google.common.collect.ImmutableMap; +import org.apache.flink.cdc.common.data.RecordData; +import org.apache.flink.cdc.common.data.binary.BinaryStringData; +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.DataChangeEvent; +import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.event.FlushEvent; +import org.apache.flink.cdc.common.event.RenameColumnEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; +import org.apache.flink.cdc.common.event.SchemaChangeEventTypeFamily; +import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; +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 org.apache.flink.cdc.common.types.DataTypes; +import org.apache.flink.cdc.common.types.RowType; +import org.apache.flink.cdc.runtime.testutils.operators.EventOperatorTestHarness; +import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +import org.apache.flink.shaded.guava31.com.google.common.collect.Sets; + +import org.apache.commons.collections.ListUtils; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +/** Unit tests for the {@link SchemaOperator} to handle evolved schema. */ +public class SchemaEvolveTest { + + private static final DataType TINYINT = DataTypes.TINYINT(); + private static final DataType SMALLINT = DataTypes.SMALLINT(); + private static final DataType INT = DataTypes.INT(); + private static final DataType BIGINT = DataTypes.BIGINT(); + private static final DataType FLOAT = DataTypes.FLOAT(); + private static final DataType DOUBLE = DataTypes.DOUBLE(); + private static final DataType STRING = DataTypes.STRING(); + + private static final TableId CUSTOMERS_TABLE_ID = + TableId.tableId("my_company", "my_branch", "customers"); + + /** Tests common evolve schema changes without exceptions. */ + @Test + public void testEvolveSchema() throws Exception { + TableId tableId = CUSTOMERS_TABLE_ID; + Schema schemaV1 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("name", STRING) + .physicalColumn("age", SMALLINT) + .primaryKey("id") + .build(); + + SchemaChangeBehavior behavior = SchemaChangeBehavior.EVOLVE; + + SchemaOperator schemaOperator = + new SchemaOperator(new ArrayList<>(), Duration.ofSeconds(30), behavior); + EventOperatorTestHarness harness = + new EventOperatorTestHarness<>(schemaOperator, 17, Duration.ofSeconds(3), behavior); + harness.open(); + + // Test CreateTableEvent + { + List createAndInsertDataEvents = + Arrays.asList( + new CreateTableEvent(tableId, schemaV1), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 1, STRING, "Alice", SMALLINT, (short) 17)), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 2, STRING, "Bob", SMALLINT, (short) 18)), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 3, STRING, "Carol", SMALLINT, (short) 19))); + + processEvent(schemaOperator, createAndInsertDataEvents); + + Assertions.assertThat( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + createAndInsertDataEvents)) + .isEqualTo( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())); + + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV1); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV1); + + harness.clearOutputRecords(); + } + + // Test AddColumnEvent + { + List addColumnEvents = + Arrays.asList( + new AddColumnEvent( + tableId, + Arrays.asList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "score", INT, "Score data")), + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "height", DOUBLE, "Height data")))), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 4, + STRING, + "Derrida", + SMALLINT, + (short) 20, + INT, + 100, + DOUBLE, + 173.25)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 5, + STRING, + "Eve", + SMALLINT, + (short) 21, + INT, + 97, + DOUBLE, + 160.))); + processEvent(schemaOperator, addColumnEvents); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + addColumnEvents)); + + Schema schemaV2 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("name", STRING) + .physicalColumn("age", SMALLINT) + .physicalColumn("score", INT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV2); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV2); + + harness.clearOutputRecords(); + } + + // Test RenameColumnEvent + { + List renameColumnEvents = + Arrays.asList( + new RenameColumnEvent( + tableId, ImmutableMap.of("name", "namae", "age", "toshi")), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 6, + STRING, + "Fiona", + SMALLINT, + (short) 22, + INT, + 100, + DOUBLE, + 173.25)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 7, + STRING, + "Gloria", + SMALLINT, + (short) 23, + INT, + 97, + DOUBLE, + 160.))); + + processEvent(schemaOperator, renameColumnEvents); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + renameColumnEvents)); + + Schema schemaV3 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING) + .physicalColumn("toshi", SMALLINT) + .physicalColumn("score", INT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV3); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV3); + + harness.clearOutputRecords(); + } + + // Test AlterColumnTypeEvent + { + List alterColumnTypeEvents = + Arrays.asList( + new AlterColumnTypeEvent( + tableId, + ImmutableMap.of("score", BIGINT, "toshi", FLOAT), + ImmutableMap.of("score", INT, "toshi", SMALLINT)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 8, STRING, "Helen", FLOAT, 22f, BIGINT, 100L, + DOUBLE, 173.25)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 9, STRING, "Iva", FLOAT, 23f, BIGINT, 97L, DOUBLE, + 160.))); + + processEvent(schemaOperator, alterColumnTypeEvents); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + alterColumnTypeEvents)); + + Schema schemaV4 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING) + .physicalColumn("toshi", FLOAT) + .physicalColumn("score", BIGINT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV4); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV4); + + harness.clearOutputRecords(); + } + + // Test DropColumnEvent + { + List dropColumnEvents = + Arrays.asList( + new DropColumnEvent(tableId, Arrays.asList("score", "height")), + DataChangeEvent.insertEvent( + tableId, buildRecord(INT, 12, STRING, "Jane", FLOAT, 11f)), + DataChangeEvent.insertEvent( + tableId, buildRecord(INT, 13, STRING, "Kryo", FLOAT, 23f))); + + processEvent(schemaOperator, dropColumnEvents); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + dropColumnEvents)); + + Schema schemaV5 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING) + .physicalColumn("toshi", FLOAT) + .primaryKey("id") + .build(); + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV5); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV5); + + harness.clearOutputRecords(); + } + harness.close(); + } + + /** Tests try-evolve behavior without exceptions. */ + @Test + public void testTryEvolveSchema() throws Exception { + TableId tableId = CUSTOMERS_TABLE_ID; + Schema schemaV1 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("name", STRING) + .physicalColumn("age", SMALLINT) + .primaryKey("id") + .build(); + + SchemaChangeBehavior behavior = SchemaChangeBehavior.TRY_EVOLVE; + + SchemaOperator schemaOperator = + new SchemaOperator(new ArrayList<>(), Duration.ofSeconds(30), behavior); + EventOperatorTestHarness harness = + new EventOperatorTestHarness<>(schemaOperator, 17, Duration.ofSeconds(3), behavior); + harness.open(); + + // Test CreateTableEvent + { + List createAndInsertDataEvents = + Arrays.asList( + new CreateTableEvent(tableId, schemaV1), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 1, STRING, "Alice", SMALLINT, (short) 17)), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 2, STRING, "Bob", SMALLINT, (short) 18)), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 3, STRING, "Carol", SMALLINT, (short) 19))); + + processEvent(schemaOperator, createAndInsertDataEvents); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + createAndInsertDataEvents)); + + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV1); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV1); + + harness.clearOutputRecords(); + } + + // Test AddColumnEvent + { + List addColumnEvents = + Arrays.asList( + new AddColumnEvent( + tableId, + Arrays.asList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "score", INT, "Score data")), + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "height", DOUBLE, "Height data")))), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 4, + STRING, + "Derrida", + SMALLINT, + (short) 20, + INT, + 100, + DOUBLE, + 173.25)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 5, + STRING, + "Eve", + SMALLINT, + (short) 21, + INT, + 97, + DOUBLE, + 160.))); + processEvent(schemaOperator, addColumnEvents); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + addColumnEvents)); + + Schema schemaV2 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("name", STRING) + .physicalColumn("age", SMALLINT) + .physicalColumn("score", INT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV2); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV2); + + harness.clearOutputRecords(); + } + + // Test RenameColumnEvent + { + List renameColumnEvents = + Arrays.asList( + new RenameColumnEvent( + tableId, ImmutableMap.of("name", "namae", "age", "toshi")), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 6, + STRING, + "Fiona", + SMALLINT, + (short) 22, + INT, + 100, + DOUBLE, + 173.25)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 7, + STRING, + "Gloria", + SMALLINT, + (short) 23, + INT, + 97, + DOUBLE, + 160.))); + + processEvent(schemaOperator, renameColumnEvents); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + renameColumnEvents)); + + Schema schemaV3 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING) + .physicalColumn("toshi", SMALLINT) + .physicalColumn("score", INT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV3); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV3); + + harness.clearOutputRecords(); + } + + // Test AlterColumnTypeEvent + { + List alterColumnTypeEvents = + Arrays.asList( + new AlterColumnTypeEvent( + tableId, + ImmutableMap.of("score", BIGINT, "toshi", FLOAT), + ImmutableMap.of("score", INT, "toshi", SMALLINT)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 8, STRING, "Helen", FLOAT, 22f, BIGINT, 100L, + DOUBLE, 173.25)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 9, STRING, "Iva", FLOAT, 23f, BIGINT, 97L, DOUBLE, + 160.))); + + processEvent(schemaOperator, alterColumnTypeEvents); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + alterColumnTypeEvents)); + + Schema schemaV4 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING) + .physicalColumn("toshi", FLOAT) + .physicalColumn("score", BIGINT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV4); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV4); + + harness.clearOutputRecords(); + } + + // Test DropColumnEvent + { + List dropColumnEvents = + Arrays.asList( + new DropColumnEvent(tableId, Arrays.asList("score", "height")), + DataChangeEvent.insertEvent( + tableId, buildRecord(INT, 12, STRING, "Jane", FLOAT, 11f)), + DataChangeEvent.insertEvent( + tableId, buildRecord(INT, 13, STRING, "Kryo", FLOAT, 23f))); + + processEvent(schemaOperator, dropColumnEvents); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + dropColumnEvents)); + + Schema schemaV5 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING) + .physicalColumn("toshi", FLOAT) + .primaryKey("id") + .build(); + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV5); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV5); + + harness.clearOutputRecords(); + } + harness.close(); + } + + /** Tests evolve schema changes when schema change behavior is set to EXCEPTION. */ + @Test + public void testExceptionEvolveSchema() throws Exception { + TableId tableId = CUSTOMERS_TABLE_ID; + Schema schemaV1 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("name", STRING) + .physicalColumn("age", SMALLINT) + .primaryKey("id") + .build(); + + SchemaChangeBehavior behavior = SchemaChangeBehavior.EXCEPTION; + + SchemaOperator schemaOperator = + new SchemaOperator(new ArrayList<>(), Duration.ofSeconds(30), behavior); + EventOperatorTestHarness harness = + new EventOperatorTestHarness<>(schemaOperator, 17, Duration.ofSeconds(3), behavior); + harness.open(); + + // Test CreateTableEvent + { + List createAndInsertDataEvents = + Arrays.asList( + new CreateTableEvent(tableId, schemaV1), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 1, STRING, "Alice", SMALLINT, (short) 17)), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 2, STRING, "Bob", SMALLINT, (short) 18)), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 3, STRING, "Carol", SMALLINT, (short) 19))); + + processEvent(schemaOperator, createAndInsertDataEvents); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + createAndInsertDataEvents)); + + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV1); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV1); + + harness.clearOutputRecords(); + } + + // Test AddColumnEvent (expected to fail) + { + List addColumnEvents = + Collections.singletonList( + new AddColumnEvent( + tableId, + Arrays.asList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "score", INT, "Score data")), + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "height", DOUBLE, "Height data"))))); + Assertions.assertThatThrownBy(() -> processEvent(schemaOperator, addColumnEvents)); + + // No schema change events should be sent to downstream + Assertions.assertThat(harness.getOutputRecords()).isEmpty(); + } + + // Test RenameColumnEvent (expected to fail) + { + List addColumnEvents = + Collections.singletonList( + new RenameColumnEvent( + tableId, ImmutableMap.of("name", "namae", "age", "toshi"))); + Assertions.assertThatThrownBy(() -> processEvent(schemaOperator, addColumnEvents)); + + // No schema change events should be sent to downstream + Assertions.assertThat(harness.getOutputRecords()).isEmpty(); + } + + // Test AlterColumnTypeEvent (expected to fail) + { + List addColumnEvents = + Collections.singletonList( + new AlterColumnTypeEvent( + tableId, ImmutableMap.of("score", BIGINT, "toshi", FLOAT))); + Assertions.assertThatThrownBy(() -> processEvent(schemaOperator, addColumnEvents)); + + // No schema change events should be sent to downstream + Assertions.assertThat(harness.getOutputRecords()).isEmpty(); + } + + // Test DropColumnEvent (expected to fail) + { + List addColumnEvents = + Collections.singletonList( + new DropColumnEvent(tableId, Arrays.asList("score", "height"))); + Assertions.assertThatThrownBy(() -> processEvent(schemaOperator, addColumnEvents)); + + // No schema change events should be sent to downstream + Assertions.assertThat(harness.getOutputRecords()).isEmpty(); + } + + harness.close(); + } + + /** Tests evolve schema changes when schema change behavior is set to IGNORE. */ + @Test + public void testIgnoreEvolveSchema() throws Exception { + TableId tableId = CUSTOMERS_TABLE_ID; + Schema schemaV1 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("name", STRING) + .physicalColumn("age", SMALLINT) + .primaryKey("id") + .build(); + + SchemaChangeBehavior behavior = SchemaChangeBehavior.IGNORE; + + SchemaOperator schemaOperator = + new SchemaOperator(new ArrayList<>(), Duration.ofSeconds(30), behavior); + EventOperatorTestHarness harness = + new EventOperatorTestHarness<>(schemaOperator, 17, Duration.ofSeconds(3), behavior); + harness.open(); + + // Test CreateTableEvent + { + List createAndInsertDataEvents = + Arrays.asList( + new CreateTableEvent(tableId, schemaV1), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 1, STRING, "Alice", SMALLINT, (short) 17)), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 2, STRING, "Bob", SMALLINT, (short) 18)), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 3, STRING, "Carol", SMALLINT, (short) 19))); + + processEvent(schemaOperator, createAndInsertDataEvents); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + createAndInsertDataEvents)); + + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV1); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV1); + + harness.clearOutputRecords(); + } + + // Test AddColumnEvent (should be ignored) + { + List addColumnEvents = + Arrays.asList( + new AddColumnEvent( + tableId, + Arrays.asList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "score", INT, "Score data")), + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "height", DOUBLE, "Height data")))), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 4, + STRING, + "Derrida", + SMALLINT, + (short) 20, + INT, + 100, + DOUBLE, + 173.25)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 5, + STRING, + "Eve", + SMALLINT, + (short) 21, + INT, + 97, + DOUBLE, + 160.))); + processEvent(schemaOperator, addColumnEvents); + + List expectedEvents = + Arrays.asList( + new FlushEvent(tableId), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 4, STRING, "Derrida", SMALLINT, (short) 20)), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 5, STRING, "Eve", SMALLINT, (short) 21))); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo(expectedEvents); + + Schema schemaV2 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("name", STRING) + .physicalColumn("age", SMALLINT) + .physicalColumn("score", INT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + + // Downstream schema should not evolve in IGNORE mode + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV2); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV1); + + harness.clearOutputRecords(); + } + + // Test RenameColumnEvent (should be ignored) + { + List renameColumnEvents = + Arrays.asList( + new RenameColumnEvent( + tableId, ImmutableMap.of("name", "namae", "score", "sukoa")), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 6, + STRING, + "Fiona", + SMALLINT, + (short) 22, + INT, + 100, + DOUBLE, + 173.25)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 7, + STRING, + "Gloria", + SMALLINT, + (short) 23, + INT, + 97, + DOUBLE, + 160.))); + + processEvent(schemaOperator, renameColumnEvents); + + List expectedEvents = + Arrays.asList( + new FlushEvent(tableId), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 6, STRING, null, SMALLINT, (short) 22)), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 7, STRING, null, SMALLINT, (short) 23))); + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo(expectedEvents); + + Schema schemaV3 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING) + .physicalColumn("age", SMALLINT) + .physicalColumn("sukoa", INT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV3); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV1); + + harness.clearOutputRecords(); + } + + // Test AlterColumnTypeEvent (should be ignored) + { + List alterColumnTypeEvents = + Arrays.asList( + new AlterColumnTypeEvent( + tableId, ImmutableMap.of("sukoa", BIGINT, "age", FLOAT)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 8, STRING, "Helen", FLOAT, 22f, BIGINT, 100L, + DOUBLE, 173.25)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 9, STRING, "Iva", FLOAT, 23f, BIGINT, 97L, DOUBLE, + 160.))); + + processEvent(schemaOperator, alterColumnTypeEvents); + + List expectedEvents = + Arrays.asList( + new FlushEvent(tableId), + DataChangeEvent.insertEvent( + tableId, buildRecord(INT, 8, STRING, null, SMALLINT, null)), + DataChangeEvent.insertEvent( + tableId, buildRecord(INT, 9, STRING, null, SMALLINT, null))); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo(expectedEvents); + + Schema schemaV4 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING) + .physicalColumn("age", FLOAT) + .physicalColumn("sukoa", BIGINT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV4); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV1); + + harness.clearOutputRecords(); + } + + // Test DropColumnEvent (should be ignored) + { + List dropColumnEvents = + Arrays.asList( + new DropColumnEvent(tableId, Arrays.asList("sukoa", "height")), + DataChangeEvent.insertEvent( + tableId, buildRecord(INT, 12, STRING, "Jane", FLOAT, 11f)), + DataChangeEvent.insertEvent( + tableId, buildRecord(INT, 13, STRING, "Kryo", FLOAT, 23f))); + + processEvent(schemaOperator, dropColumnEvents); + + List expectedEvents = + Arrays.asList( + new FlushEvent(tableId), + DataChangeEvent.insertEvent( + tableId, buildRecord(INT, 12, STRING, null, DOUBLE, null)), + DataChangeEvent.insertEvent( + tableId, buildRecord(INT, 13, STRING, null, DOUBLE, null))); + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo(expectedEvents); + + Schema schemaV5 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING) + .physicalColumn("age", FLOAT) + .primaryKey("id") + .build(); + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV5); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV1); + + harness.clearOutputRecords(); + } + harness.close(); + } + + /** Tests common evolve schema changes without exceptions. */ + @Test + public void testEvolveSchemaWithFailure() throws Exception { + TableId tableId = CUSTOMERS_TABLE_ID; + Schema schemaV1 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("name", STRING) + .physicalColumn("age", SMALLINT) + .primaryKey("id") + .build(); + + SchemaChangeBehavior behavior = SchemaChangeBehavior.EVOLVE; + + SchemaOperator schemaOperator = + new SchemaOperator(new ArrayList<>(), Duration.ofSeconds(30), behavior); + EventOperatorTestHarness harness = + new EventOperatorTestHarness<>( + schemaOperator, + 17, + Duration.ofSeconds(3), + behavior, + Arrays.stream(SchemaChangeEventTypeFamily.ALL).collect(Collectors.toSet()), + Sets.newHashSet( + SchemaChangeEventType.ADD_COLUMN, + SchemaChangeEventType.RENAME_COLUMN)); + + harness.open(); + + // Test CreateTableEvent + List createAndInsertDataEvents = + Arrays.asList( + new CreateTableEvent(tableId, schemaV1), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 1, STRING, "Alice", SMALLINT, (short) 17)), + DataChangeEvent.insertEvent( + tableId, buildRecord(INT, 2, STRING, "Bob", SMALLINT, (short) 18)), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 3, STRING, "Carol", SMALLINT, (short) 19))); + + processEvent(schemaOperator, createAndInsertDataEvents); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + createAndInsertDataEvents)); + + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV1); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV1); + + harness.clearOutputRecords(); + + // Test AddColumnEvent (should fail) + List addColumnEvents = + Collections.singletonList( + new AddColumnEvent( + tableId, + Arrays.asList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn("score", INT, "Score data")), + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "height", DOUBLE, "Height data"))))); + Assertions.assertThatThrownBy(() -> processEvent(schemaOperator, addColumnEvents)) + .isExactlyInstanceOf(RuntimeException.class) + .hasMessageContaining("Failed to apply schema change"); + harness.close(); + } + + /** Tests evolve schema changes when schema change behavior is set to TRY_EVOLVE. */ + @Test + public void testTryEvolveSchemaWithFailure() throws Exception { + TableId tableId = CUSTOMERS_TABLE_ID; + Schema schemaV1 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("name", STRING) + .physicalColumn("age", SMALLINT) + .primaryKey("id") + .build(); + + SchemaChangeBehavior behavior = SchemaChangeBehavior.TRY_EVOLVE; + + SchemaOperator schemaOperator = + new SchemaOperator(new ArrayList<>(), Duration.ofSeconds(30), behavior); + + // All types of schema change events will be sent to the sink + // AddColumn and RenameColumn events will always fail + EventOperatorTestHarness harness = + new EventOperatorTestHarness<>( + schemaOperator, + 17, + Duration.ofSeconds(3), + behavior, + Arrays.stream(SchemaChangeEventTypeFamily.ALL).collect(Collectors.toSet()), + Sets.newHashSet( + SchemaChangeEventType.ALTER_COLUMN_TYPE, + SchemaChangeEventType.DROP_COLUMN)); + + harness.open(); + + // Test CreateTableEvent + { + List createAndInsertDataEvents = + Arrays.asList( + new CreateTableEvent(tableId, schemaV1), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 1, STRING, "Alice", SMALLINT, (short) 17)), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 2, STRING, "Bob", SMALLINT, (short) 18)), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 3, STRING, "Carol", SMALLINT, (short) 19))); + + processEvent(schemaOperator, createAndInsertDataEvents); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + createAndInsertDataEvents)); + + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV1); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV1); + + harness.clearOutputRecords(); + } + + // Test AddColumnEvent + { + List addColumnEvents = + Arrays.asList( + new AddColumnEvent( + tableId, + Arrays.asList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "score", INT, "Score data")), + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "height", DOUBLE, "Height data")))), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 4, + STRING, + "Derrida", + SMALLINT, + (short) 20, + INT, + 100, + DOUBLE, + 173.25)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 5, + STRING, + "Eve", + SMALLINT, + (short) 21, + INT, + 97, + DOUBLE, + 160.))); + processEvent(schemaOperator, addColumnEvents); + + List expectedEvents = new ArrayList<>(); + expectedEvents.add(new FlushEvent(tableId)); + expectedEvents.addAll(addColumnEvents); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo(expectedEvents); + + Schema schemaV2 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("name", STRING) + .physicalColumn("age", SMALLINT) + .physicalColumn("score", INT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + + // Downstream schema should not evolve in IGNORE mode + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV2); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV2); + + harness.clearOutputRecords(); + } + + // Test RenameColumnEvent + { + List renameColumnEvents = + Arrays.asList( + new RenameColumnEvent( + tableId, ImmutableMap.of("name", "namae", "score", "sukoa")), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 6, + STRING, + "Fiona", + SMALLINT, + (short) 22, + INT, + 100, + DOUBLE, + 173.25)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 7, + STRING, + "Gloria", + SMALLINT, + (short) 23, + INT, + 97, + DOUBLE, + 160.))); + + processEvent(schemaOperator, renameColumnEvents); + + List expectedEvents = new ArrayList<>(); + expectedEvents.add(new FlushEvent(tableId)); + expectedEvents.addAll(renameColumnEvents); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo(expectedEvents); + + Schema schemaV3 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING) + .physicalColumn("age", SMALLINT) + .physicalColumn("sukoa", INT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV3); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV3); + + harness.clearOutputRecords(); + } + + // Test AlterColumnTypeEvent (should fail) + { + List alterColumnTypeEvents = + Arrays.asList( + new AlterColumnTypeEvent( + tableId, ImmutableMap.of("sukoa", BIGINT, "age", FLOAT)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 8, STRING, "Helen", FLOAT, 22f, BIGINT, 100L, + DOUBLE, 173.25)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 9, STRING, "Iva", FLOAT, 23f, BIGINT, 97L, DOUBLE, + 160.))); + + processEvent(schemaOperator, alterColumnTypeEvents); + + List expectedEvents = + Arrays.asList( + new FlushEvent(tableId), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 8, STRING, "Helen", SMALLINT, null, INT, null, + DOUBLE, 173.25)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 9, STRING, "Iva", SMALLINT, null, INT, null, + DOUBLE, 160.))); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo(expectedEvents); + + Schema schemaV4 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING) + .physicalColumn("age", FLOAT) + .physicalColumn("sukoa", BIGINT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + + Schema schemaV4E = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING) + .physicalColumn("age", SMALLINT) + .physicalColumn("sukoa", INT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV4); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV4E); + + harness.clearOutputRecords(); + } + + // Test DropColumnEvent (should fail) + { + List dropColumnEvents = + Arrays.asList( + new DropColumnEvent(tableId, Arrays.asList("sukoa", "height")), + DataChangeEvent.insertEvent( + tableId, buildRecord(INT, 12, STRING, "Jane", FLOAT, 11f)), + DataChangeEvent.insertEvent( + tableId, buildRecord(INT, 13, STRING, "Kryo", FLOAT, 23f))); + + processEvent(schemaOperator, dropColumnEvents); + + List expectedEvents = + Arrays.asList( + new FlushEvent(tableId), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 12, STRING, "Jane", SMALLINT, null, INT, null, + DOUBLE, null)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 13, STRING, "Kryo", SMALLINT, null, INT, null, + DOUBLE, null))); + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo(expectedEvents); + + Schema schemaV5 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING) + .physicalColumn("age", FLOAT) + .primaryKey("id") + .build(); + Schema schemaV5E = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING) + .physicalColumn("age", SMALLINT) + .physicalColumn("sukoa", INT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV5); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV5E); + + harness.clearOutputRecords(); + } + harness.close(); + } + + /** Tests fine-grained schema change configurations. */ + @Test + public void testFineGrainedSchemaEvolves() throws Exception { + TableId tableId = CUSTOMERS_TABLE_ID; + Schema schemaV1 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("name", STRING) + .physicalColumn("age", SMALLINT) + .primaryKey("id") + .build(); + + SchemaChangeBehavior behavior = SchemaChangeBehavior.EVOLVE; + + SchemaOperator schemaOperator = + new SchemaOperator(new ArrayList<>(), Duration.ofSeconds(30), behavior); + + // All types of schema change events will be sent to the sink + // AddColumn and RenameColumn events will always fail + EventOperatorTestHarness harness = + new EventOperatorTestHarness<>( + schemaOperator, + 17, + Duration.ofSeconds(3), + behavior, + Sets.newHashSet( + SchemaChangeEventType.CREATE_TABLE, + SchemaChangeEventType.ADD_COLUMN, + SchemaChangeEventType.RENAME_COLUMN)); + + harness.open(); + + // Test CreateTableEvent + { + List createAndInsertDataEvents = + Arrays.asList( + new CreateTableEvent(tableId, schemaV1), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 1, STRING, "Alice", SMALLINT, (short) 17)), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 2, STRING, "Bob", SMALLINT, (short) 18)), + DataChangeEvent.insertEvent( + tableId, + buildRecord(INT, 3, STRING, "Carol", SMALLINT, (short) 19))); + + processEvent(schemaOperator, createAndInsertDataEvents); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + createAndInsertDataEvents)); + + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV1); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV1); + + harness.clearOutputRecords(); + } + + // Test AddColumnEvent + { + List addColumnEvents = + Arrays.asList( + new AddColumnEvent( + tableId, + Arrays.asList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "score", INT, "Score data")), + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "height", DOUBLE, "Height data")))), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 4, + STRING, + "Derrida", + SMALLINT, + (short) 20, + INT, + 100, + DOUBLE, + 173.25)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 5, + STRING, + "Eve", + SMALLINT, + (short) 21, + INT, + 97, + DOUBLE, + 160.))); + processEvent(schemaOperator, addColumnEvents); + + List expectedEvents = new ArrayList<>(); + expectedEvents.add(new FlushEvent(tableId)); + expectedEvents.addAll(addColumnEvents); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo(expectedEvents); + + Schema schemaV2 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("name", STRING) + .physicalColumn("age", SMALLINT) + .physicalColumn("score", INT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + + // Downstream schema should not evolve in IGNORE mode + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV2); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV2); + + harness.clearOutputRecords(); + } + + // Test RenameColumnEvent + { + List renameColumnEvents = + Arrays.asList( + new RenameColumnEvent( + tableId, ImmutableMap.of("name", "namae", "score", "sukoa")), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 6, + STRING, + "Fiona", + SMALLINT, + (short) 22, + INT, + 100, + DOUBLE, + 173.25)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 7, + STRING, + "Gloria", + SMALLINT, + (short) 23, + INT, + 97, + DOUBLE, + 160.))); + + processEvent(schemaOperator, renameColumnEvents); + + List expectedEvents = new ArrayList<>(); + expectedEvents.add(new FlushEvent(tableId)); + expectedEvents.addAll(renameColumnEvents); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo(expectedEvents); + + Schema schemaV3 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING) + .physicalColumn("age", SMALLINT) + .physicalColumn("sukoa", INT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV3); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV3); + + harness.clearOutputRecords(); + } + + // Test AlterColumnTypeEvent (should be ignored) + { + List alterColumnTypeEvents = + Arrays.asList( + new AlterColumnTypeEvent( + tableId, ImmutableMap.of("sukoa", BIGINT, "age", FLOAT)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 8, STRING, "Helen", FLOAT, 22f, BIGINT, 100L, + DOUBLE, 173.25)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 9, STRING, "Iva", FLOAT, 23f, BIGINT, 97L, DOUBLE, + 160.))); + + processEvent(schemaOperator, alterColumnTypeEvents); + + List expectedEvents = + Arrays.asList( + new FlushEvent(tableId), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 8, STRING, "Helen", SMALLINT, null, INT, null, + DOUBLE, 173.25)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 9, STRING, "Iva", SMALLINT, null, INT, null, + DOUBLE, 160.))); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo(expectedEvents); + + Schema schemaV4 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING) + .physicalColumn("age", FLOAT) + .physicalColumn("sukoa", BIGINT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + + Schema schemaV4E = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING) + .physicalColumn("age", SMALLINT) + .physicalColumn("sukoa", INT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV4); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV4E); + + harness.clearOutputRecords(); + } + + // Test DropColumnEvent (should be ignored) + { + List dropColumnEvents = + Arrays.asList( + new DropColumnEvent(tableId, Arrays.asList("sukoa", "height")), + DataChangeEvent.insertEvent( + tableId, buildRecord(INT, 12, STRING, "Jane", FLOAT, 11f)), + DataChangeEvent.insertEvent( + tableId, buildRecord(INT, 13, STRING, "Kryo", FLOAT, 23f))); + + processEvent(schemaOperator, dropColumnEvents); + + List expectedEvents = + Arrays.asList( + new FlushEvent(tableId), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 12, STRING, "Jane", SMALLINT, null, INT, null, + DOUBLE, null)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 13, STRING, "Kryo", SMALLINT, null, INT, null, + DOUBLE, null))); + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo(expectedEvents); + + Schema schemaV5 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING) + .physicalColumn("age", FLOAT) + .primaryKey("id") + .build(); + Schema schemaV5E = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING) + .physicalColumn("age", SMALLINT) + .physicalColumn("sukoa", INT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + Assertions.assertThat(harness.getLatestUpstreamSchema(tableId)).isEqualTo(schemaV5); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV5E); + + harness.clearOutputRecords(); + } + harness.close(); + } + + private RecordData buildRecord(final Object... args) { + List dataTypes = new ArrayList<>(); + List objects = new ArrayList<>(); + for (int i = 0; i < args.length; i += 2) { + DataType dataType = (DataType) args[i]; + Object object = args[i + 1]; + dataTypes.add(dataType); + if (dataType.equals(STRING)) { + objects.add(BinaryStringData.fromString((String) object)); + } else { + objects.add(object); + } + } + return new BinaryRecordDataGenerator(RowType.of(dataTypes.toArray(new DataType[0]))) + .generate(objects.toArray()); + } + + private void processEvent(SchemaOperator operator, List events) throws Exception { + for (Event event : events) { + operator.processElement(new StreamRecord<>(event)); + } + } +} diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/SchemaOperatorTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/SchemaOperatorTest.java index 3802d14ecd7..eb45a253113 100644 --- a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/SchemaOperatorTest.java +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/SchemaOperatorTest.java @@ -123,6 +123,7 @@ void testProcessSchemaChangeEventWithTimeOut() throws Exception { schemaOperator.processElement( new StreamRecord<>( new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)))); + harness.close(); } @Test @@ -137,6 +138,7 @@ void testProcessSchemaChangeEventWithOutTimeOut() throws Exception { schemaOperator.processElement( new StreamRecord<>( new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)))); + harness.close(); } private OneInputStreamOperatorTestHarness createTestHarness( 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..1387758d475 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 @@ -333,7 +333,7 @@ void testMergingTableWithDifferentSchemas() { new AddColumnEvent.ColumnWithPosition( new PhysicalColumn("first_name", DataTypes.STRING(), null))); - assertThat(schemaManager.getLatestSchema(MERGED_TABLE)) + assertThat(schemaManager.getLatestUpstreamSchema(MERGED_TABLE)) .contains( Schema.newBuilder() .column(Column.physicalColumn("id", DataTypes.BIGINT())) @@ -365,8 +365,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/operators/schema/coordinator/SchemaManagerTest.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaManagerTest.java index 941714dcc5d..2766ece388b 100644 --- a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaManagerTest.java +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaManagerTest.java @@ -60,13 +60,15 @@ class SchemaManagerTest { @Test void testHandlingCreateTableEvent() { SchemaManager schemaManager = new SchemaManager(); - schemaManager.applySchemaChange(new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); - assertThat(schemaManager.getLatestSchema(CUSTOMERS)).isPresent().contains(CUSTOMERS_SCHEMA); + schemaManager.applyEvolvedSchemaChange(new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + assertThat(schemaManager.getLatestEvolvedSchema(CUSTOMERS)) + .isPresent() + .contains(CUSTOMERS_SCHEMA); // Cannot apply CreateTableEvent multiple times assertThatThrownBy( () -> - schemaManager.applySchemaChange( + schemaManager.applyEvolvedSchemaChange( new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA))) .isInstanceOf(IllegalArgumentException.class) .hasMessage( @@ -94,9 +96,9 @@ void testHandlingAddColumnEvent() { AddColumnEvent.ColumnPosition.BEFORE, "phone")); - schemaManager.applySchemaChange(new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); - schemaManager.applySchemaChange(new AddColumnEvent(CUSTOMERS, newColumns)); - assertThat(schemaManager.getLatestSchema(CUSTOMERS)) + schemaManager.applyEvolvedSchemaChange(new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + schemaManager.applyEvolvedSchemaChange(new AddColumnEvent(CUSTOMERS, newColumns)); + assertThat(schemaManager.getLatestEvolvedSchema(CUSTOMERS)) .contains( Schema.newBuilder() .physicalColumn("append_first", DataTypes.BIGINT()) @@ -112,92 +114,199 @@ void testHandlingAddColumnEvent() { @Test void testHandlingAlterColumnTypeEvent() { - SchemaManager schemaManager = new SchemaManager(); - schemaManager.applySchemaChange(new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); - schemaManager.applySchemaChange( - new AlterColumnTypeEvent(CUSTOMERS, ImmutableMap.of("phone", DataTypes.STRING()))); - assertThat(schemaManager.getLatestSchema(CUSTOMERS)) - .contains( - Schema.newBuilder() - .physicalColumn("id", DataTypes.INT()) - .physicalColumn("name", DataTypes.STRING()) - .physicalColumn("phone", DataTypes.STRING()) - .primaryKey("id") - .build()); + { + SchemaManager schemaManager = new SchemaManager(); + schemaManager.applyUpstreamSchemaChange( + new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + schemaManager.applyUpstreamSchemaChange( + new AlterColumnTypeEvent( + CUSTOMERS, ImmutableMap.of("phone", DataTypes.STRING()))); + assertThat(schemaManager.getLatestUpstreamSchema(CUSTOMERS)) + .contains( + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("phone", DataTypes.STRING()) + .primaryKey("id") + .build()); + } + + { + SchemaManager schemaManager = new SchemaManager(); + schemaManager.applyEvolvedSchemaChange( + new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + schemaManager.applyEvolvedSchemaChange( + new AlterColumnTypeEvent( + CUSTOMERS, ImmutableMap.of("phone", DataTypes.STRING()))); + assertThat(schemaManager.getLatestEvolvedSchema(CUSTOMERS)) + .contains( + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("name", DataTypes.STRING()) + .physicalColumn("phone", DataTypes.STRING()) + .primaryKey("id") + .build()); + } } @Test void testHandlingDropColumnEvent() { - SchemaManager schemaManager = new SchemaManager(); - schemaManager.applySchemaChange(new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); - schemaManager.applySchemaChange( - new DropColumnEvent(CUSTOMERS, Arrays.asList("name", "phone"))); - assertThat(schemaManager.getLatestSchema(CUSTOMERS)) - .contains( - Schema.newBuilder() - .physicalColumn("id", DataTypes.INT()) - .primaryKey("id") - .build()); + { + SchemaManager schemaManager = new SchemaManager(); + schemaManager.applyUpstreamSchemaChange( + new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + schemaManager.applyUpstreamSchemaChange( + new DropColumnEvent(CUSTOMERS, Arrays.asList("name", "phone"))); + assertThat(schemaManager.getLatestUpstreamSchema(CUSTOMERS)) + .contains( + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .primaryKey("id") + .build()); + } + { + SchemaManager schemaManager = new SchemaManager(); + schemaManager.applyEvolvedSchemaChange( + new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + schemaManager.applyEvolvedSchemaChange( + new DropColumnEvent(CUSTOMERS, Arrays.asList("name", "phone"))); + assertThat(schemaManager.getLatestEvolvedSchema(CUSTOMERS)) + .contains( + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .primaryKey("id") + .build()); + } } @Test void testHandlingRenameColumnEvent() { - SchemaManager schemaManager = new SchemaManager(); - schemaManager.applySchemaChange(new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); - schemaManager.applySchemaChange( - new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("name", "new_name"))); - assertThat(schemaManager.getLatestSchema(CUSTOMERS)) - .contains( - Schema.newBuilder() - .physicalColumn("id", DataTypes.INT()) - .physicalColumn("new_name", DataTypes.STRING()) - .physicalColumn("phone", DataTypes.BIGINT()) - .primaryKey("id") - .build()); + { + SchemaManager schemaManager = new SchemaManager(); + schemaManager.applyUpstreamSchemaChange( + new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + schemaManager.applyUpstreamSchemaChange( + new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("name", "new_name"))); + assertThat(schemaManager.getLatestUpstreamSchema(CUSTOMERS)) + .contains( + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("new_name", DataTypes.STRING()) + .physicalColumn("phone", DataTypes.BIGINT()) + .primaryKey("id") + .build()); + } + { + SchemaManager schemaManager = new SchemaManager(); + schemaManager.applyEvolvedSchemaChange( + new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + schemaManager.applyEvolvedSchemaChange( + new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("name", "new_name"))); + assertThat(schemaManager.getLatestEvolvedSchema(CUSTOMERS)) + .contains( + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("new_name", DataTypes.STRING()) + .physicalColumn("phone", DataTypes.BIGINT()) + .primaryKey("id") + .build()); + } } @Test void testGettingHistoricalSchema() { - SchemaManager schemaManager = new SchemaManager(); - schemaManager.applySchemaChange(new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); - schemaManager.applySchemaChange( - new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("name", "new_name"))); - schemaManager.applySchemaChange( - new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("phone", "new_phone"))); - assertThat(schemaManager.getSchema(CUSTOMERS, 1)) - .isEqualTo( - Schema.newBuilder() - .physicalColumn("id", DataTypes.INT()) - .physicalColumn("new_name", DataTypes.STRING()) - .physicalColumn("phone", DataTypes.BIGINT()) - .primaryKey("id") - .build()); + { + SchemaManager schemaManager = new SchemaManager(); + schemaManager.applyUpstreamSchemaChange( + new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + schemaManager.applyUpstreamSchemaChange( + new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("name", "new_name"))); + schemaManager.applyUpstreamSchemaChange( + new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("phone", "new_phone"))); + assertThat(schemaManager.getUpstreamSchema(CUSTOMERS, 1)) + .isEqualTo( + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("new_name", DataTypes.STRING()) + .physicalColumn("phone", DataTypes.BIGINT()) + .primaryKey("id") + .build()); + } + { + SchemaManager schemaManager = new SchemaManager(); + schemaManager.applyEvolvedSchemaChange( + new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + schemaManager.applyEvolvedSchemaChange( + new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("name", "new_name"))); + schemaManager.applyEvolvedSchemaChange( + new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("phone", "new_phone"))); + assertThat(schemaManager.getEvolvedSchema(CUSTOMERS, 1)) + .isEqualTo( + Schema.newBuilder() + .physicalColumn("id", DataTypes.INT()) + .physicalColumn("new_name", DataTypes.STRING()) + .physicalColumn("phone", DataTypes.BIGINT()) + .primaryKey("id") + .build()); + } } @Test void testVersionCleanup() { - SchemaManager schemaManager = new SchemaManager(); - schemaManager.applySchemaChange(new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); - schemaManager.applySchemaChange( - new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("name", "new_name"))); - schemaManager.applySchemaChange( - new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("phone", "new_phone"))); - schemaManager.applySchemaChange( - new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("new_phone", "new_phone_2"))); - assertThatThrownBy(() -> schemaManager.getSchema(CUSTOMERS, 0)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Schema version %s does not exist for table \"%s\"", 0, CUSTOMERS); + { + SchemaManager schemaManager = new SchemaManager(); + schemaManager.applyUpstreamSchemaChange( + new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + schemaManager.applyUpstreamSchemaChange( + new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("name", "new_name"))); + schemaManager.applyUpstreamSchemaChange( + new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("phone", "new_phone"))); + schemaManager.applyUpstreamSchemaChange( + new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("new_phone", "new_phone_2"))); + assertThatThrownBy(() -> schemaManager.getUpstreamSchema(CUSTOMERS, 0)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Schema version %s does not exist for table \"%s\"", 0, CUSTOMERS); + } + { + SchemaManager schemaManager = new SchemaManager(); + schemaManager.applyEvolvedSchemaChange( + new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + schemaManager.applyEvolvedSchemaChange( + new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("name", "new_name"))); + schemaManager.applyEvolvedSchemaChange( + new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("phone", "new_phone"))); + schemaManager.applyEvolvedSchemaChange( + new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("new_phone", "new_phone_2"))); + assertThatThrownBy(() -> schemaManager.getEvolvedSchema(CUSTOMERS, 0)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Schema version %s does not exist for table \"%s\"", 0, CUSTOMERS); + } } @Test void testSerde() throws Exception { - SchemaManager schemaManager = new SchemaManager(); - schemaManager.applySchemaChange(new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); - schemaManager.applySchemaChange(new CreateTableEvent(PRODUCTS, PRODUCTS_SCHEMA)); - byte[] serialized = SchemaManager.SERIALIZER.serialize(schemaManager); - SchemaManager deserialized = - SchemaManager.SERIALIZER.deserialize( - SchemaManager.Serializer.CURRENT_VERSION, serialized); - assertThat(deserialized).isEqualTo(schemaManager); + { + SchemaManager schemaManager = new SchemaManager(); + schemaManager.applyUpstreamSchemaChange( + new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + schemaManager.applyUpstreamSchemaChange( + new CreateTableEvent(PRODUCTS, PRODUCTS_SCHEMA)); + byte[] serialized = SchemaManager.SERIALIZER.serialize(schemaManager); + SchemaManager deserialized = + SchemaManager.SERIALIZER.deserialize( + SchemaManager.Serializer.CURRENT_VERSION, serialized); + assertThat(deserialized).isEqualTo(schemaManager); + } + { + SchemaManager schemaManager = new SchemaManager(); + schemaManager.applyEvolvedSchemaChange( + new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + schemaManager.applyEvolvedSchemaChange(new CreateTableEvent(PRODUCTS, PRODUCTS_SCHEMA)); + byte[] serialized = SchemaManager.SERIALIZER.serialize(schemaManager); + SchemaManager deserialized = + SchemaManager.SERIALIZER.deserialize( + SchemaManager.Serializer.CURRENT_VERSION, serialized); + assertThat(deserialized).isEqualTo(schemaManager); + } } } 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")) + }; + } +} diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/testutils/operators/EventOperatorTestHarness.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/testutils/operators/EventOperatorTestHarness.java index b96a17b56dc..4489be4ad95 100644 --- a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/testutils/operators/EventOperatorTestHarness.java +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/testutils/operators/EventOperatorTestHarness.java @@ -20,10 +20,18 @@ import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.FlushEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaRegistry; +import org.apache.flink.cdc.runtime.operators.schema.event.ApplyEvolvedSchemaChangeRequest; +import org.apache.flink.cdc.runtime.operators.schema.event.ApplyUpstreamSchemaChangeRequest; import org.apache.flink.cdc.runtime.operators.schema.event.FlushSuccessEvent; +import org.apache.flink.cdc.runtime.operators.schema.event.GetEvolvedSchemaRequest; +import org.apache.flink.cdc.runtime.operators.schema.event.GetEvolvedSchemaResponse; +import org.apache.flink.cdc.runtime.operators.schema.event.GetUpstreamSchemaRequest; +import org.apache.flink.cdc.runtime.operators.schema.event.GetUpstreamSchemaResponse; import org.apache.flink.cdc.runtime.operators.schema.event.SchemaChangeRequest; import org.apache.flink.cdc.runtime.operators.schema.event.SinkWriterRegisterEvent; import org.apache.flink.cdc.runtime.operators.sink.SchemaEvolutionClient; @@ -48,7 +56,11 @@ import java.io.IOException; import java.time.Duration; import java.util.ArrayList; +import java.util.Collections; import java.util.LinkedList; +import java.util.Set; + +import static org.apache.flink.cdc.runtime.operators.schema.event.CoordinationResponseUtils.unwrap; /** * Harness for testing customized operators handling {@link Event}s in CDC pipeline. @@ -74,6 +86,15 @@ public class EventOperatorTestHarness, E ex private final LinkedList> outputRecords = new LinkedList<>(); public EventOperatorTestHarness(OP operator, int numOutputs) { + this(operator, numOutputs, null, SchemaChangeBehavior.EVOLVE); + } + + public EventOperatorTestHarness(OP operator, int numOutputs, Duration duration) { + this(operator, numOutputs, duration, SchemaChangeBehavior.EVOLVE); + } + + public EventOperatorTestHarness( + OP operator, int numOutputs, Duration duration, SchemaChangeBehavior behavior) { this.operator = operator; this.numOutputs = numOutputs; schemaRegistry = @@ -81,12 +102,18 @@ public EventOperatorTestHarness(OP operator, int numOutputs) { "SchemaOperator", new MockOperatorCoordinatorContext( SCHEMA_OPERATOR_ID, Thread.currentThread().getContextClassLoader()), - new CollectingMetadataApplier(null), - new ArrayList<>()); + new CollectingMetadataApplier(duration), + new ArrayList<>(), + behavior); schemaRegistryGateway = new TestingSchemaRegistryGateway(schemaRegistry); } - public EventOperatorTestHarness(OP operator, int numOutputs, Duration duration) { + public EventOperatorTestHarness( + OP operator, + int numOutputs, + Duration duration, + SchemaChangeBehavior behavior, + Set enabledEventTypes) { this.operator = operator; this.numOutputs = numOutputs; schemaRegistry = @@ -94,8 +121,30 @@ public EventOperatorTestHarness(OP operator, int numOutputs, Duration duration) "SchemaOperator", new MockOperatorCoordinatorContext( SCHEMA_OPERATOR_ID, Thread.currentThread().getContextClassLoader()), - new CollectingMetadataApplier(duration), - new ArrayList<>()); + new CollectingMetadataApplier(duration, enabledEventTypes), + new ArrayList<>(), + behavior); + schemaRegistryGateway = new TestingSchemaRegistryGateway(schemaRegistry); + } + + public EventOperatorTestHarness( + OP operator, + int numOutputs, + Duration duration, + SchemaChangeBehavior behavior, + Set enabledEventTypes, + Set errorsOnEventTypes) { + this.operator = operator; + this.numOutputs = numOutputs; + schemaRegistry = + new SchemaRegistry( + "SchemaOperator", + new MockOperatorCoordinatorContext( + SCHEMA_OPERATOR_ID, Thread.currentThread().getContextClassLoader()), + new CollectingMetadataApplier( + duration, enabledEventTypes, errorsOnEventTypes), + new ArrayList<>(), + behavior); schemaRegistryGateway = new TestingSchemaRegistryGateway(schemaRegistry); } @@ -108,13 +157,51 @@ public LinkedList> getOutputRecords() { return outputRecords; } + public void clearOutputRecords() { + outputRecords.clear(); + } + public OP getOperator() { return operator; } public void registerTableSchema(TableId tableId, Schema schema) { + schemaRegistry.handleCoordinationRequest( + new ApplyUpstreamSchemaChangeRequest( + tableId, new CreateTableEvent(tableId, schema))); schemaRegistry.handleCoordinationRequest( new SchemaChangeRequest(tableId, new CreateTableEvent(tableId, schema))); + schemaRegistry.handleCoordinationRequest( + new ApplyEvolvedSchemaChangeRequest( + tableId, Collections.singletonList(new CreateTableEvent(tableId, schema)))); + } + + public Schema getLatestUpstreamSchema(TableId tableId) throws Exception { + return ((GetUpstreamSchemaResponse) + unwrap( + schemaRegistry + .handleCoordinationRequest( + new GetUpstreamSchemaRequest( + tableId, + GetUpstreamSchemaRequest + .LATEST_SCHEMA_VERSION)) + .get())) + .getSchema() + .orElse(null); + } + + public Schema getLatestEvolvedSchema(TableId tableId) throws Exception { + return ((GetEvolvedSchemaResponse) + unwrap( + schemaRegistry + .handleCoordinationRequest( + new GetEvolvedSchemaRequest( + tableId, + GetEvolvedSchemaRequest + .LATEST_SCHEMA_VERSION)) + .get())) + .getSchema() + .orElse(null); } @Override diff --git a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/testutils/schema/CollectingMetadataApplier.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/testutils/schema/CollectingMetadataApplier.java index 1ebb5d449c5..70e54138823 100644 --- a/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/testutils/schema/CollectingMetadataApplier.java +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/testutils/schema/CollectingMetadataApplier.java @@ -18,11 +18,17 @@ package org.apache.flink.cdc.runtime.testutils.schema; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; +import org.apache.flink.cdc.common.event.SchemaChangeEventTypeFamily; import org.apache.flink.cdc.common.sink.MetadataApplier; import java.time.Duration; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; /** * A {@link MetadataApplier} for testing that holds all schema change events in a list for further @@ -32,9 +38,40 @@ public class CollectingMetadataApplier implements MetadataApplier { private final List schemaChangeEvents = new ArrayList<>(); private final Duration duration; + private final Set enabledEventTypes; + private final Set errorsOnEventTypes; public CollectingMetadataApplier(Duration duration) { this.duration = duration; + this.enabledEventTypes = + Arrays.stream(SchemaChangeEventTypeFamily.ALL).collect(Collectors.toSet()); + this.errorsOnEventTypes = Collections.emptySet(); + } + + public CollectingMetadataApplier( + Duration duration, Set enabledEventTypes) { + this.duration = duration; + this.enabledEventTypes = enabledEventTypes; + this.errorsOnEventTypes = Collections.emptySet(); + } + + public CollectingMetadataApplier( + Duration duration, + Set enabledEventTypes, + Set errorsOnEventTypes) { + this.duration = duration; + this.enabledEventTypes = enabledEventTypes; + this.errorsOnEventTypes = errorsOnEventTypes; + } + + @Override + public boolean acceptsSchemaEvolutionType(SchemaChangeEventType schemaChangeEventType) { + return enabledEventTypes.contains(schemaChangeEventType); + } + + @Override + public Set getSupportedSchemaEvolutionTypes() { + return Arrays.stream(SchemaChangeEventTypeFamily.ALL).collect(Collectors.toSet()); } @Override @@ -43,8 +80,11 @@ public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) { if (duration != null) { try { Thread.sleep(duration.toMillis()); - } catch (Exception ignore) { - + if (errorsOnEventTypes.contains(schemaChangeEvent.getType())) { + throw new RuntimeException("Dummy metadata apply exception for test."); + } + } catch (InterruptedException ignore) { + // Ignores sleep interruption } } } diff --git a/flink-cdc-runtime/src/test/resources/log4j2-test.properties b/flink-cdc-runtime/src/test/resources/log4j2-test.properties index cad8aa6dd68..31de99a333e 100644 --- a/flink-cdc-runtime/src/test/resources/log4j2-test.properties +++ b/flink-cdc-runtime/src/test/resources/log4j2-test.properties @@ -15,7 +15,7 @@ # Set root logger level to OFF to not flood build logs # set manually to INFO for debugging purposes -rootLogger.level = OFF +rootLogger.level = DEBUG rootLogger.appenderRef.test.ref = TestLogger appender.testlogger.name = TestLogger