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 7ad07af1a44..cfee1f8109a 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"; @@ -136,6 +142,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>() {}); @@ -149,7 +172,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..3966ff46ffc 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 @@ -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/AlterColumnTypeEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/AlterColumnTypeEvent.java index 5eb5da0d121..a5a104140f0 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 @@ -77,4 +77,9 @@ public String toString() { public TableId tableId() { return tableId; } + + @Override + public SchemaChangeEventType getType() { + return SchemaChangeEventType.ALTER_COLUMN_TYPE; + } } 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..6d3e547172d 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 @@ -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..76cabbde7b7 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 @@ -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/RenameColumnEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/RenameColumnEvent.java index 63dd723812b..4558c1508c8 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 @@ -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/SchemaChangeEvent.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEvent.java index 1d37860fb5d..d5596e3b3de 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,7 @@ * 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(); +} 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..668ea76a4d4 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventType.java @@ -0,0 +1,63 @@ +/* + * 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.annotation.PublicEvolving; + +/** An enumeration of schema change event types for {@link SchemaChangeEvent}. */ +@PublicEvolving +public enum SchemaChangeEventType { + ADD_COLUMN, + ALTER_COLUMN_TYPE, + CREATE_TABLE, + DROP_COLUMN, + RENAME_COLUMN; + + public static SchemaChangeEventType ofEvent(SchemaChangeEvent event) { + if (event instanceof AddColumnEvent) { + return ADD_COLUMN; + } else if (event instanceof AlterColumnTypeEvent) { + return ALTER_COLUMN_TYPE; + } else if (event instanceof CreateTableEvent) { + return CREATE_TABLE; + } else if (event instanceof DropColumnEvent) { + return DROP_COLUMN; + } else if (event instanceof RenameColumnEvent) { + return RENAME_COLUMN; + } else { + throw new RuntimeException("Unknown schema change event type: " + event.getClass()); + } + } + + public static SchemaChangeEventType ofTag(String tag) { + switch (tag) { + case "add.column": + return ADD_COLUMN; + case "alter.column.type": + return ALTER_COLUMN_TYPE; + case "create.table": + return CREATE_TABLE; + case "drop.column": + return DROP_COLUMN; + case "rename.column": + return RENAME_COLUMN; + default: + throw new RuntimeException("Unknown schema change event type: " + tag); + } + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventTypeFamily.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventTypeFamily.java new file mode 100644 index 00000000000..5ff7187f616 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/event/SchemaChangeEventTypeFamily.java @@ -0,0 +1,80 @@ +/* + * 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.annotation.PublicEvolving; + +/** + * An enumeration of schema change event families for clustering {@link SchemaChangeEvent}s into + * categories. + */ +@PublicEvolving +public class SchemaChangeEventTypeFamily { + + public static final SchemaChangeEventType[] ADD = {SchemaChangeEventType.ADD_COLUMN}; + + public static final SchemaChangeEventType[] ALTER = {SchemaChangeEventType.ALTER_COLUMN_TYPE}; + + public static final SchemaChangeEventType[] CREATE = {SchemaChangeEventType.CREATE_TABLE}; + + public static final SchemaChangeEventType[] DROP = {SchemaChangeEventType.DROP_COLUMN}; + + public static final SchemaChangeEventType[] RENAME = {SchemaChangeEventType.RENAME_COLUMN}; + + public static final SchemaChangeEventType[] TABLE = {SchemaChangeEventType.CREATE_TABLE}; + + public static final SchemaChangeEventType[] COLUMN = { + SchemaChangeEventType.ADD_COLUMN, + SchemaChangeEventType.ALTER_COLUMN_TYPE, + SchemaChangeEventType.DROP_COLUMN, + SchemaChangeEventType.RENAME_COLUMN + }; + + public static final SchemaChangeEventType[] ALL = { + SchemaChangeEventType.ADD_COLUMN, + SchemaChangeEventType.CREATE_TABLE, + SchemaChangeEventType.ALTER_COLUMN_TYPE, + SchemaChangeEventType.DROP_COLUMN, + SchemaChangeEventType.RENAME_COLUMN + }; + + 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/exceptions/SchemaEvolveException.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/exceptions/SchemaEvolveException.java new file mode 100644 index 00000000000..fc5b482f149 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/exceptions/SchemaEvolveException.java @@ -0,0 +1,69 @@ +/* + * 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.exceptions; + +import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.util.FlinkRuntimeException; + +import javax.annotation.Nullable; + +/** An exception occurred during schema evolution. */ +public class SchemaEvolveException extends FlinkRuntimeException { + private final SchemaChangeEvent applyingEvent; + private final String exceptionMessage; + private final @Nullable Throwable cause; + + public SchemaEvolveException(SchemaChangeEvent applyingEvent, String exceptionMessage) { + this(applyingEvent, exceptionMessage, null); + } + + public SchemaEvolveException( + SchemaChangeEvent applyingEvent, String exceptionMessage, @Nullable Throwable cause) { + super(cause); + this.applyingEvent = applyingEvent; + this.exceptionMessage = exceptionMessage; + this.cause = cause; + } + + public SchemaChangeEvent getApplyingEvent() { + return applyingEvent; + } + + public String getExceptionMessage() { + return exceptionMessage; + } + + @Nullable + public Throwable getCause() { + return cause; + } + + @Override + public String toString() { + return "SchemaEvolveException{" + + "applyingEvent=" + + applyingEvent + + ", exceptionMessage='" + + exceptionMessage + + '\'' + + ", cause='" + + cause + + '\'' + + '}'; + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/exceptions/UnsupportedSchemaChangeEventException.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/exceptions/UnsupportedSchemaChangeEventException.java new file mode 100644 index 00000000000..4be20525abc --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/exceptions/UnsupportedSchemaChangeEventException.java @@ -0,0 +1,28 @@ +/* + * 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.exceptions; + +import org.apache.flink.cdc.common.event.SchemaChangeEvent; + +/** A special kind of {@link SchemaEvolveException} that sink doesn't support such event type. */ +public class UnsupportedSchemaChangeEventException extends SchemaEvolveException { + + public UnsupportedSchemaChangeEventException(SchemaChangeEvent applyingEvent) { + super(applyingEvent, "Sink doesn't support such schema change event.", null); + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/pipeline/PipelineOptions.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/pipeline/PipelineOptions.java index 5c32a0290f1..48a4fbb13a3 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/pipeline/PipelineOptions.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/pipeline/PipelineOptions.java @@ -55,9 +55,13 @@ public class PipelineOptions { .linebreak() .add( ListElement.list( + text("IGNORE: Drop all schema change events."), + text( + "LENIENT: Apply schema changes to downstream tolerantly, and keeps executing if applying fails."), + text( + "TRY_EVOLVE: Apply schema changes to downstream, but keeps executing if applying fails."), text( "EVOLVE: Apply schema changes to downstream. This requires sink to support handling schema changes."), - text("IGNORE: Drop all schema change events."), text( "EXCEPTION: Throw an exception to terminate the sync pipeline."))) .build()); 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..4822b72a783 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 @@ -19,10 +19,15 @@ import org.apache.flink.cdc.common.annotation.PublicEvolving; -/** Behavior for handling schema changes. */ +/** + * Behavior for handling schema changes. Enums are sorted from the most tolerant strategy (IGNORE) + * to the most aggressive one (EXCEPTION). + */ @PublicEvolving public enum SchemaChangeBehavior { - EVOLVE, IGNORE, + LENIENT, + TRY_EVOLVE, + EVOLVE, EXCEPTION } 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..d112fd1926e 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,35 @@ 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 org.apache.flink.cdc.common.event.SchemaChangeEventTypeFamily; +import org.apache.flink.cdc.common.exceptions.SchemaEvolveException; import java.io.Serializable; +import java.util.Arrays; +import java.util.Set; +import java.util.stream.Collectors; /** {@code MetadataApplier} is used to apply metadata changes to external systems. */ @PublicEvolving public interface MetadataApplier extends Serializable { /** Apply the given {@link SchemaChangeEvent} to external systems. */ - void applySchemaChange(SchemaChangeEvent schemaChangeEvent); + void applySchemaChange(SchemaChangeEvent schemaChangeEvent) throws SchemaEvolveException; + + /** Sets enabled schema evolution event types of current metadata applier. */ + default MetadataApplier setAcceptedSchemaEvolutionTypes( + Set schemaEvolutionTypes) { + return this; + } + + /** Checks if this metadata applier should this event type. */ + default boolean acceptsSchemaEvolutionType(SchemaChangeEventType schemaChangeEventType) { + return true; + } + + /** Checks what kind of schema change events downstream can handle. */ + default Set getSupportedSchemaEvolutionTypes() { + return Arrays.stream(SchemaChangeEventTypeFamily.ALL).collect(Collectors.toSet()); + } } 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..ee72ef6e23e 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,6 +17,7 @@ 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.AlterColumnTypeEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; @@ -24,8 +25,16 @@ import org.apache.flink.cdc.common.event.DropColumnEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; +import org.apache.flink.cdc.common.event.SchemaChangeEventTypeFamily; import org.apache.flink.cdc.common.event.TableId; +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 { public static DataChangeEvent recreateDataChangeEvent( @@ -81,4 +90,37 @@ public static SchemaChangeEvent recreateSchemaChangeEvent( "Unsupported schema change event with type \"%s\"", schemaChangeEvent.getClass().getCanonicalName())); } + + 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)); + } + } + + for (String excludeTag : excludedSchemaEvolutionTypes) { + resultTypes.removeAll(resolveSchemaEvolutionTag(excludeTag)); + } + + 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); + } + } + return types; + } } 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..0a34a750185 --- /dev/null +++ b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/utils/ChangeEventUtilsTest.java @@ -0,0 +1,118 @@ +/* + * 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_TYPE; +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.RENAME_COLUMN; + +/** A test for the {@link org.apache.flink.cdc.common.utils.ChangeEventUtils}. */ +public class ChangeEventUtilsTest { + @Test + public void testResolveSchemaEvolutionOptions() { + Assertions.assertThat( + ChangeEventUtils.resolveSchemaEvolutionOptions( + Collections.emptyList(), Collections.emptyList())) + .isEqualTo( + Sets.set( + CREATE_TABLE, + ADD_COLUMN, + ALTER_COLUMN_TYPE, + DROP_COLUMN, + RENAME_COLUMN)); + + Assertions.assertThat( + ChangeEventUtils.resolveSchemaEvolutionOptions( + Collections.emptyList(), Collections.singletonList("drop"))) + .isEqualTo(Sets.set(CREATE_TABLE, ADD_COLUMN, ALTER_COLUMN_TYPE, RENAME_COLUMN)); + + Assertions.assertThat( + ChangeEventUtils.resolveSchemaEvolutionOptions( + Arrays.asList("create", "add"), Collections.emptyList())) + .isEqualTo(Sets.set(CREATE_TABLE, ADD_COLUMN)); + + Assertions.assertThat( + ChangeEventUtils.resolveSchemaEvolutionOptions( + Collections.singletonList("column"), + Collections.singletonList("drop.column"))) + .isEqualTo(Sets.set(ADD_COLUMN, ALTER_COLUMN_TYPE, RENAME_COLUMN)); + + Assertions.assertThat( + ChangeEventUtils.resolveSchemaEvolutionOptions( + Collections.emptyList(), Collections.singletonList("drop.column"))) + .isEqualTo(Sets.set(CREATE_TABLE, ADD_COLUMN, ALTER_COLUMN_TYPE, RENAME_COLUMN)); + } + + @Test + public void testResolveSchemaEvolutionTag() { + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("all")) + .isEqualTo( + Arrays.asList( + ADD_COLUMN, + CREATE_TABLE, + ALTER_COLUMN_TYPE, + DROP_COLUMN, + RENAME_COLUMN)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("column")) + .isEqualTo( + Arrays.asList(ADD_COLUMN, ALTER_COLUMN_TYPE, DROP_COLUMN, RENAME_COLUMN)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("table")) + .isEqualTo(Collections.singletonList(CREATE_TABLE)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("rename")) + .isEqualTo(Collections.singletonList(RENAME_COLUMN)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("rename.column")) + .isEqualTo(Collections.singletonList(RENAME_COLUMN)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("drop")) + .isEqualTo(Collections.singletonList(DROP_COLUMN)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("drop.column")) + .isEqualTo(Collections.singletonList(DROP_COLUMN)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("create")) + .isEqualTo(Collections.singletonList(CREATE_TABLE)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("create.table")) + .isEqualTo(Collections.singletonList(CREATE_TABLE)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("alter")) + .isEqualTo(Collections.singletonList(ALTER_COLUMN_TYPE)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("alter.column.type")) + .isEqualTo(Collections.singletonList(ALTER_COLUMN_TYPE)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("add")) + .isEqualTo(Collections.singletonList(ADD_COLUMN)); + + Assertions.assertThat(ChangeEventUtils.resolveSchemaEvolutionTag("add.column")) + .isEqualTo(Collections.singletonList(ADD_COLUMN)); + } +} 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 0735b90cf32..a6f5e65a846 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,12 @@ public PipelineExecution compose(PipelineDef pipelineDef) { stream = schemaOperatorTranslator.translate( - stream, parallelism, dataSink.getMetadataApplier(), pipelineDef.getRoute()); + stream, + parallelism, + dataSink.getMetadataApplier() + .setAcceptedSchemaEvolutionTypes( + 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 1f5dc44bdc5..c965c88a623 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 @@ -56,18 +56,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() { @@ -78,7 +67,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( @@ -91,7 +81,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 a8acf798996..bf3e9ab46a8 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 @@ -471,6 +471,74 @@ void testOneToOneRouting() throws Exception { "DataChangeEvent{tableId=default_namespace.default_schema.routed1, before=[2, 2], after=[2, x], op=UPDATE, meta=()}"); } + @Test + void testIdenticalOneToOneRouting() throws Exception { + FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster(); + + // Setup value source + Configuration sourceConfig = new Configuration(); + sourceConfig.set( + ValuesDataSourceOptions.EVENT_SET_ID, + ValuesDataSourceHelper.EventSetId.SINGLE_SPLIT_MULTI_TABLES); + SourceDef sourceDef = + new SourceDef(ValuesDataFactory.IDENTIFIER, "Value Source", sourceConfig); + + // Setup value sink + Configuration sinkConfig = new Configuration(); + sinkConfig.set(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY, true); + SinkDef sinkDef = new SinkDef(ValuesDataFactory.IDENTIFIER, "Value Sink", sinkConfig); + + // Setup route + TableId routedTable1 = TABLE_1; + TableId routedTable2 = TABLE_2; + List routeDef = + Arrays.asList( + new RouteDef(TABLE_1.toString(), routedTable1.toString(), null, null), + new RouteDef(TABLE_2.toString(), routedTable2.toString(), null, null)); + + // Setup pipeline + Configuration pipelineConfig = new Configuration(); + pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1); + PipelineDef pipelineDef = + new PipelineDef( + sourceDef, sinkDef, routeDef, Collections.emptyList(), pipelineConfig); + + // Execute the pipeline + PipelineExecution execution = composer.compose(pipelineDef); + execution.execute(); + + // Check result in ValuesDatabase + List routed1Results = ValuesDatabase.getResults(routedTable1); + assertThat(routed1Results) + .contains( + "default_namespace.default_schema.table1:col1=2;newCol3=x", + "default_namespace.default_schema.table1:col1=3;newCol3="); + List routed2Results = ValuesDatabase.getResults(routedTable2); + assertThat(routed2Results) + .contains( + "default_namespace.default_schema.table2:col1=1;col2=1", + "default_namespace.default_schema.table2:col1=2;col2=2", + "default_namespace.default_schema.table2:col1=3;col2=3"); + + // Check the order and content of all received events + String[] outputEvents = outCaptor.toString().trim().split("\n"); + assertThat(outputEvents) + .containsExactly( + "CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING}, primaryKeys=col1, options=()}", + "CreateTableEvent{tableId=default_namespace.default_schema.table2, schema=columns={`col1` STRING,`col2` STRING}, primaryKeys=col1, options=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[3, 3], op=INSERT, meta=()}", + "AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=LAST, existedColumnName=null}]}", + "DataChangeEvent{tableId=default_namespace.default_schema.table2, before=[], after=[1, 1], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table2, before=[], after=[2, 2], op=INSERT, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table2, before=[], after=[3, 3], op=INSERT, meta=()}", + "RenameColumnEvent{tableId=default_namespace.default_schema.table1, nameMapping={col2=newCol2, col3=newCol3}}", + "DropColumnEvent{tableId=default_namespace.default_schema.table1, droppedColumnNames=[newCol2]}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[1, 1], after=[], op=DELETE, meta=()}", + "DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[2, 2], after=[2, x], op=UPDATE, meta=()}"); + } + @Test void testMergingWithRoute() throws Exception { FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster(); 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 8ea6f603222..811d03298de 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 @@ -24,7 +24,10 @@ import org.apache.flink.cdc.common.event.DropColumnEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.exceptions.SchemaEvolveException; +import org.apache.flink.cdc.common.exceptions.UnsupportedSchemaChangeEventException; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.sink.MetadataApplier; @@ -36,6 +39,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; @@ -52,7 +57,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. */ @@ -61,15 +70,34 @@ public class DorisMetadataApplier implements MetadataApplier { private DorisOptions dorisOptions; private SchemaChangeManager schemaChangeManager; private Configuration config; + private Set enabledSchemaEvolutionTypes; public DorisMetadataApplier(DorisOptions dorisOptions, Configuration config) { this.dorisOptions = dorisOptions; this.schemaChangeManager = new SchemaChangeManager(dorisOptions); this.config = config; + this.enabledSchemaEvolutionTypes = getSupportedSchemaEvolutionTypes(); + } + + @Override + public MetadataApplier setAcceptedSchemaEvolutionTypes( + Set schemaEvolutionTypes) { + this.enabledSchemaEvolutionTypes = schemaEvolutionTypes; + return this; + } + + @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) { + public void applySchemaChange(SchemaChangeEvent event) throws SchemaEvolveException { try { // send schema change op to doris if (event instanceof CreateTableEvent) { @@ -82,12 +110,11 @@ public void applySchemaChange(SchemaChangeEvent event) { applyRenameColumnEvent((RenameColumnEvent) event); } else if (event instanceof AlterColumnTypeEvent) { applyAlterColumnTypeEvent((AlterColumnTypeEvent) event); - } else if (event instanceof AlterColumnTypeEvent) { - throw new RuntimeException("Unsupported schema change event, " + event); + } else { + throw new UnsupportedSchemaChangeEventException(event); } } catch (Exception ex) { - throw new RuntimeException( - "Failed to schema change, " + event + ", reason: " + ex.getMessage()); + throw new SchemaEvolveException(event, ex.getMessage(), null); } } 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 fbaf26744bc..438cc37815d 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; @@ -54,6 +55,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; @@ -424,7 +426,11 @@ private void runJobWithEvents(List events) throws Exception { schemaOperatorTranslator.translate( stream, DEFAULT_PARALLELISM, - dorisSink.getMetadataApplier(), + dorisSink + .getMetadataApplier() + .setAcceptedSchemaEvolutionTypes( + 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/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..2ab264c5a9e 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 { @@ -94,7 +100,33 @@ public EventSinkProvider getEventSinkProvider() { @Override public MetadataApplier getMetadataApplier() { - // simply do nothing here because Kafka do not maintain the schemas. - return schemaChangeEvent -> {}; + return new MetadataApplier() { + + private Set enabledEventTypes = + Arrays.stream(SchemaChangeEventTypeFamily.ALL).collect(Collectors.toSet()); + + @Override + public MetadataApplier setAcceptedSchemaEvolutionTypes( + Set schemaEvolutionTypes) { + enabledEventTypes = schemaEvolutionTypes; + return this; + } + + @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-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 b75950f5b5a..3ae3c9c4c73 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 @@ -23,11 +23,16 @@ import org.apache.flink.cdc.common.event.DropColumnEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.exceptions.SchemaEvolveException; +import org.apache.flink.cdc.common.exceptions.UnsupportedSchemaChangeEventException; import org.apache.flink.cdc.common.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 +40,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 +58,8 @@ */ public class PaimonMetadataApplier implements MetadataApplier { + private static final Logger LOG = LoggerFactory.getLogger(PaimonMetadataApplier.class); + // Catalog is unSerializable. private transient Catalog catalog; @@ -60,10 +70,13 @@ public class PaimonMetadataApplier implements MetadataApplier { private final Map> partitionMaps; + private Set enabledSchemaEvolutionTypes; + public PaimonMetadataApplier(Options catalogOptions) { this.catalogOptions = catalogOptions; this.tableOptions = new HashMap<>(); this.partitionMaps = new HashMap<>(); + this.enabledSchemaEvolutionTypes = getSupportedSchemaEvolutionTypes(); } public PaimonMetadataApplier( @@ -73,10 +86,34 @@ public PaimonMetadataApplier( this.catalogOptions = catalogOptions; this.tableOptions = tableOptions; this.partitionMaps = partitionMaps; + this.enabledSchemaEvolutionTypes = getSupportedSchemaEvolutionTypes(); + } + + @Override + public MetadataApplier setAcceptedSchemaEvolutionTypes( + Set schemaEvolutionTypes) { + this.enabledSchemaEvolutionTypes = schemaEvolutionTypes; + return this; + } + + @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 - public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) { + public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) + throws SchemaEvolveException { if (catalog == null) { catalog = FlinkCatalogFactory.createPaimonCatalog(catalogOptions); } @@ -92,11 +129,10 @@ public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) { } else if (schemaChangeEvent instanceof AlterColumnTypeEvent) { applyAlterColumn((AlterColumnTypeEvent) schemaChangeEvent); } else { - throw new UnsupportedOperationException( - "PaimonDataSink doesn't support schema change event " + schemaChangeEvent); + throw new UnsupportedSchemaChangeEventException(schemaChangeEvent); } } catch (Exception e) { - throw new RuntimeException(e); + throw new SchemaEvolveException(schemaChangeEvent, "schema change applying failure", e); } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/test/java/org/apache/flink/cdc/connectors/paimon/sink/PaimonMetadataApplierTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/test/java/org/apache/flink/cdc/connectors/paimon/sink/PaimonMetadataApplierTest.java index fe29b467471..214f2051e11 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/test/java/org/apache/flink/cdc/connectors/paimon/sink/PaimonMetadataApplierTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/test/java/org/apache/flink/cdc/connectors/paimon/sink/PaimonMetadataApplierTest.java @@ -23,6 +23,7 @@ import org.apache.flink.cdc.common.event.DropColumnEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.exceptions.SchemaEvolveException; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.sink.MetadataApplier; import org.apache.flink.cdc.common.types.DataType; @@ -94,7 +95,7 @@ private void initialize(String metastore) @ValueSource(strings = {"filesystem", "hive"}) public void testApplySchemaChange(String metastore) throws Catalog.TableNotExistException, Catalog.DatabaseNotEmptyException, - Catalog.DatabaseNotExistException { + Catalog.DatabaseNotExistException, SchemaEvolveException { initialize(metastore); MetadataApplier metadataApplier = new PaimonMetadataApplier(catalogOptions); CreateTableEvent createTableEvent = @@ -182,7 +183,7 @@ public void testApplySchemaChange(String metastore) @ValueSource(strings = {"filesystem", "hive"}) public void testCreateTableWithOptions(String metastore) throws Catalog.TableNotExistException, Catalog.DatabaseNotEmptyException, - Catalog.DatabaseNotExistException { + Catalog.DatabaseNotExistException, SchemaEvolveException { initialize(metastore); Map tableOptions = new HashMap<>(); tableOptions.put("bucket", "-1"); @@ -228,7 +229,7 @@ public void testCreateTableWithOptions(String metastore) @ValueSource(strings = {"filesystem", "hive"}) public void testCreateTableWithAllDataTypes(String metastore) throws Catalog.TableNotExistException, Catalog.DatabaseNotEmptyException, - Catalog.DatabaseNotExistException { + Catalog.DatabaseNotExistException, SchemaEvolveException { initialize(metastore); MetadataApplier metadataApplier = new PaimonMetadataApplier(catalogOptions); CreateTableEvent createTableEvent = @@ -338,7 +339,7 @@ public void testCreateTableWithAllDataTypes(String metastore) @ValueSource(strings = {"filesystem", "hive"}) public void testAddColumnWithPosition(String metastore) throws Catalog.DatabaseNotEmptyException, Catalog.DatabaseNotExistException, - Catalog.TableNotExistException { + Catalog.TableNotExistException, SchemaEvolveException { initialize(metastore); MetadataApplier metadataApplier = new PaimonMetadataApplier(catalogOptions); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/test/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonSinkITCase.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/test/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonSinkITCase.java index 300fdc91ad9..f44fd47a333 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/test/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonSinkITCase.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/test/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonSinkITCase.java @@ -31,6 +31,7 @@ import org.apache.flink.cdc.common.event.DropColumnEvent; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.exceptions.SchemaEvolveException; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.types.DataTypes; @@ -129,7 +130,7 @@ private void initialize(String metastore) .dropDatabase(TEST_DATABASE, true, true); } - private List createTestEvents() { + private List createTestEvents() throws SchemaEvolveException { List testEvents = new ArrayList<>(); // create table Schema schema = @@ -172,7 +173,7 @@ private List createTestEvents() { @ValueSource(strings = {"filesystem", "hive"}) public void testSinkWithDataChange(String metastore) throws IOException, InterruptedException, Catalog.DatabaseNotEmptyException, - Catalog.DatabaseNotExistException { + Catalog.DatabaseNotExistException, SchemaEvolveException { initialize(metastore); PaimonSink paimonSink = new PaimonSink<>( @@ -258,7 +259,7 @@ public void testSinkWithDataChange(String metastore) @ValueSource(strings = {"filesystem", "hive"}) public void testSinkWithSchemaChange(String metastore) throws IOException, InterruptedException, Catalog.DatabaseNotEmptyException, - Catalog.DatabaseNotExistException { + Catalog.DatabaseNotExistException, SchemaEvolveException { initialize(metastore); PaimonSink paimonSink = new PaimonSink( @@ -393,7 +394,7 @@ public void testSinkWithSchemaChange(String metastore) @ValueSource(strings = {"filesystem", "hive"}) public void testSinkWithMultiTables(String metastore) throws IOException, InterruptedException, Catalog.DatabaseNotEmptyException, - Catalog.DatabaseNotExistException { + Catalog.DatabaseNotExistException, SchemaEvolveException { initialize(metastore); PaimonSink paimonSink = new PaimonSink<>( 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 34b483d00a7..865613eda58 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 @@ -23,10 +23,15 @@ import org.apache.flink.cdc.common.event.DropColumnEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.exceptions.SchemaEvolveException; +import org.apache.flink.cdc.common.exceptions.UnsupportedSchemaChangeEventException; import org.apache.flink.cdc.common.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 +41,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Set; import static org.apache.flink.cdc.connectors.starrocks.sink.StarRocksUtils.toStarRocksDataType; @@ -50,6 +56,7 @@ public class StarRocksMetadataApplier implements MetadataApplier { private final TableCreateConfig tableCreateConfig; private final SchemaChangeConfig schemaChangeConfig; private boolean isOpened; + private Set enabledSchemaEvolutionTypes; public StarRocksMetadataApplier( StarRocksCatalog catalog, @@ -59,10 +66,32 @@ public StarRocksMetadataApplier( this.tableCreateConfig = tableCreateConfig; this.schemaChangeConfig = schemaChangeConfig; this.isOpened = false; + this.enabledSchemaEvolutionTypes = getSupportedSchemaEvolutionTypes(); + } + + @Override + public MetadataApplier setAcceptedSchemaEvolutionTypes( + Set schemaEvolutionTypes) { + this.enabledSchemaEvolutionTypes = schemaEvolutionTypes; + return this; + } + + @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 - public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) { + public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) + throws SchemaEvolveException { if (!isOpened) { isOpened = true; catalog.open(); @@ -79,12 +108,11 @@ public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) { } else if (schemaChangeEvent instanceof AlterColumnTypeEvent) { applyAlterColumn((AlterColumnTypeEvent) schemaChangeEvent); } else { - throw new UnsupportedOperationException( - "StarRocksDataSink doesn't support schema change event " + schemaChangeEvent); + throw new UnsupportedSchemaChangeEventException(schemaChangeEvent); } } - private void applyCreateTable(CreateTableEvent createTableEvent) { + private void applyCreateTable(CreateTableEvent createTableEvent) throws SchemaEvolveException { StarRocksTable starRocksTable = StarRocksUtils.toStarRocksTable( createTableEvent.tableId(), @@ -99,11 +127,11 @@ private void applyCreateTable(CreateTableEvent createTableEvent) { LOG.info("Successful to create table, event: {}", createTableEvent); } catch (StarRocksCatalogException e) { LOG.error("Failed to create table, event: {}", createTableEvent.tableId(), e); - throw new RuntimeException("Failed to create table, event: " + createTableEvent, e); + throw new SchemaEvolveException(createTableEvent, "Failed to create table", e); } } - private void applyAddColumn(AddColumnEvent addColumnEvent) { + private void applyAddColumn(AddColumnEvent addColumnEvent) throws SchemaEvolveException { List addColumns = new ArrayList<>(); for (AddColumnEvent.ColumnWithPosition columnWithPosition : addColumnEvent.getAddedColumns()) { @@ -174,9 +202,9 @@ private void applyAddColumn(AddColumnEvent addColumnEvent) { "Failed to apply add column because of alter exception, event: {}", addColumnEvent, alterException); - throw new RuntimeException( - "Failed to apply add column because of alter exception, event: " - + addColumnEvent, + throw new SchemaEvolveException( + addColumnEvent, + "Failed to apply add column because of alter exception, event: ", alterException); } else { String errorMsg = @@ -184,11 +212,11 @@ private void applyAddColumn(AddColumnEvent addColumnEvent) { "Failed to apply add column because of validation failure, event: %s, table: %s", addColumnEvent, table); LOG.error(errorMsg); - throw new RuntimeException(errorMsg); + throw new SchemaEvolveException(addColumnEvent, errorMsg, null); } } - private void applyDropColumn(DropColumnEvent dropColumnEvent) { + private void applyDropColumn(DropColumnEvent dropColumnEvent) throws SchemaEvolveException { List dropColumns = dropColumnEvent.getDroppedColumnNames(); TableId tableId = dropColumnEvent.tableId(); StarRocksCatalogException alterException = null; @@ -240,9 +268,9 @@ private void applyDropColumn(DropColumnEvent dropColumnEvent) { "Failed to apply drop column because of alter exception, event: {}", dropColumnEvent, alterException); - throw new RuntimeException( - "Failed to apply drop column because of alter exception, event: " - + dropColumnEvent, + throw new SchemaEvolveException( + dropColumnEvent, + "Failed to apply drop column because of alter exception", alterException); } else { String errorMsg = @@ -250,23 +278,25 @@ private void applyDropColumn(DropColumnEvent dropColumnEvent) { "Failed to apply drop column because of validation failure, event: %s, table: %s", dropColumnEvent, table); LOG.error(errorMsg); - throw new RuntimeException(errorMsg); + throw new SchemaEvolveException(dropColumnEvent, errorMsg, null); } } - private void applyRenameColumn(RenameColumnEvent renameColumnEvent) { + private void applyRenameColumn(RenameColumnEvent renameColumnEvent) + throws SchemaEvolveException { // TODO StarRocks plans to support column rename since 3.3 which has not been released. // Support it later. - throw new UnsupportedOperationException("Rename column is not supported currently"); + throw new UnsupportedSchemaChangeEventException(renameColumnEvent); } - private void applyAlterColumn(AlterColumnTypeEvent alterColumnTypeEvent) { + private void applyAlterColumn(AlterColumnTypeEvent alterColumnTypeEvent) + throws SchemaEvolveException { // 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 // types after changing. One way is that the framework delivers the old schema. We can // support // the alter after a discussion. - throw new UnsupportedOperationException("Alter column is not supported currently"); + throw new UnsupportedSchemaChangeEventException(alterColumnTypeEvent); } } 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..c5e833e545b 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,11 @@ private void runJobWithEvents(List events) throws Exception { schemaOperatorTranslator.translate( stream, DEFAULT_PARALLELISM, - starRocksSink.getMetadataApplier(), + starRocksSink + .getMetadataApplier() + .setAcceptedSchemaEvolutionTypes( + 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 4d80deecec2..e19c4a844b2 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 @@ -26,7 +26,9 @@ import org.apache.flink.cdc.common.event.DropColumnEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.exceptions.SchemaEvolveException; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.sink.MetadataApplier; @@ -36,12 +38,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; @@ -71,12 +76,88 @@ public class ValuesDatabase { */ public static class ValuesMetadataApplier implements MetadataApplier { + private Set enabledSchemaEvolutionTypes; + + public ValuesMetadataApplier() { + this.enabledSchemaEvolutionTypes = getSupportedSchemaEvolutionTypes(); + } + + @Override + public MetadataApplier setAcceptedSchemaEvolutionTypes( + Set schemaEvolutionTypes) { + this.enabledSchemaEvolutionTypes = schemaEvolutionTypes; + return this; + } + + @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 Set enabledSchemaEvolutionTypes; + + public ErrorOnChangeMetadataApplier() { + enabledSchemaEvolutionTypes = getSupportedSchemaEvolutionTypes(); + } + + @Override + public MetadataApplier setAcceptedSchemaEvolutionTypes( + Set schemaEvolutionTypes) { + enabledSchemaEvolutionTypes = schemaEvolutionTypes; + return this; + } + + @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) + throws SchemaEvolveException { + if (schemaChangeEvent instanceof CreateTableEvent) { + TableId tableId = schemaChangeEvent.tableId(); + if (!globalTables.containsKey(tableId)) { + globalTables.put( + tableId, + new ValuesTable( + tableId, ((CreateTableEvent) schemaChangeEvent).getSchema())); + } + } else { + throw new SchemaEvolveException( + schemaChangeEvent, + "Rejected schema change event since error.on.schema.change is enabled.", + null); + } + } + } + /** provide namespace/schema/table lists for {@link ValuesDataSource}. */ public static class ValuesMetadataAccessor implements MetadataAccessor { 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 1b5983f1da5..a1c0ae24b1f 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 @@ -57,7 +57,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 @@ -78,6 +80,7 @@ public Set> optionalOptions() { options.add(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY); options.add(ValuesDataSinkOptions.PRINT_ENABLED); options.add(ValuesDataSinkOptions.SINK_API); + 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..7d4ee3dad8e 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 @@ -52,10 +52,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 @@ -70,7 +77,11 @@ public EventSinkProvider getEventSinkProvider() { @Override public MetadataApplier getMetadataApplier() { - return new ValuesDatabase.ValuesMetadataApplier(); + if (errorOnSchemaChange) { + return new ValuesDatabase.ErrorOnChangeMetadataApplier(); + } else { + return new ValuesDatabase.ValuesMetadataApplier(); + } } /** an e2e {@link Sink} implementation that print all {@link DataChangeEvent} out. */ 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..8862c7eca54 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 @@ -25,6 +25,7 @@ import org.apache.flink.cdc.common.event.DropColumnEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.exceptions.SchemaEvolveException; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.sink.MetadataApplier; @@ -91,7 +92,7 @@ public class ValuesDatabaseTest { *

|----|----|. */ @Before - public void before() { + public void before() throws SchemaEvolveException { ValuesDatabase.clear(); metadataApplier = new ValuesDatabase.ValuesMetadataApplier(); metadataAccessor = new ValuesDatabase.ValuesMetadataAccessor(); @@ -159,7 +160,7 @@ public void testValuesMetadataAccessor() { } @Test - public void testApplySchemaChangeEvent() { + public void testApplySchemaChangeEvent() throws SchemaEvolveException { AddColumnEvent.ColumnWithPosition columnWithPosition = new AddColumnEvent.ColumnWithPosition( Column.physicalColumn("col3", new CharType())); @@ -258,7 +259,7 @@ public void testApplyDataChangeEvent() { } @Test - public void testSchemaChangeWithExistedData() { + public void testSchemaChangeWithExistedData() throws SchemaEvolveException { AddColumnEvent.ColumnWithPosition columnWithPosition = new AddColumnEvent.ColumnWithPosition( Column.physicalColumn("col3", new CharType())); 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 43319558c7c..ada384e0703 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,8 +190,7 @@ 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;"); @@ -212,8 +210,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( @@ -246,13 +243,13 @@ public void testSyncWholeDatabase() throws Exception { 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..d187853daf6 --- /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,409 @@ +/* + * 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.Collections; +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 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(); + } + + @Test + public void testSchemaEvolve() throws Exception { + testGenericSchemaEvolution( + "evolve", + false, + false, + false, + Arrays.asList( + "AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`gender` TINYINT, position=AFTER, existedColumnName=age}]}", + "DataChangeEvent{tableId=%s.members, before=[], after=[1012, Eve, 17, 0], op=INSERT, meta=()}", + "AlterColumnTypeEvent{tableId=%s.members, nameMapping={age=DOUBLE}}", + "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=()}")); + } + + @Test + public void testSchemaEvolveWithIncompatibleChanges() throws Exception { + testGenericSchemaEvolution( + "evolve", + true, + false, + false, + Collections.singletonList( + "java.lang.IllegalStateException: Incompatible types: \"INT\" and \"DOUBLE\""), + Collections.singletonList( + "org.apache.flink.runtime.JobException: Recovery is suppressed by NoRestartBackoffTimeStrategy")); + } + + @Test + public void testSchemaEvolveWithException() throws Exception { + testGenericSchemaEvolution( + "evolve", + false, + true, + false, + Collections.singletonList( + "AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`gender` TINYINT, position=AFTER, existedColumnName=age}]}"), + Arrays.asList( + "Failed to apply schema change event AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`gender` TINYINT, position=AFTER, existedColumnName=age}]}.", + "SchemaEvolveException{applyingEvent=AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`gender` TINYINT, position=AFTER, existedColumnName=age}]}, exceptionMessage='Rejected schema change event since error.on.schema.change is enabled.', cause='null'}", + "org.apache.flink.runtime.JobException: Recovery is suppressed by NoRestartBackoffTimeStrategy")); + } + + @Test + public void testSchemaTryEvolveWithException() throws Exception { + testGenericSchemaEvolution( + "try_evolve", + false, + true, + false, + Arrays.asList( + // 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=[], after=[1013, Fiona, null], op=INSERT, meta=()}"), + Arrays.asList( + "Failed to apply schema change AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`gender` TINYINT, position=AFTER, existedColumnName=age}]} to table %s.members.", + "SchemaEvolveException{applyingEvent=AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`gender` TINYINT, position=AFTER, existedColumnName=age}]}, exceptionMessage='Rejected schema change event since error.on.schema.change is enabled.', cause='null'}")); + } + + @Test + public void testSchemaIgnore() throws Exception { + + testGenericSchemaEvolution( + "ignore", + false, + false, + false, + Arrays.asList( + "DataChangeEvent{tableId=%s.members, before=[], after=[1012, Eve, 17], op=INSERT, meta=()}", + "DataChangeEvent{tableId=%s.members, before=[], after=[1013, Fiona, null], op=INSERT, meta=()}")); + } + + @Test + public void testSchemaException() throws Exception { + testGenericSchemaEvolution( + "exception", + false, + false, + false, + Collections.emptyList(), + Collections.singletonList( + "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.")); + } + + @Test + public void testLenientSchemaEvolution() throws Exception { + + testGenericSchemaEvolution( + "lenient", + false, + false, + false, + Arrays.asList( + "AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`gender` TINYINT, position=AFTER, existedColumnName=age}]}", + "DataChangeEvent{tableId=%s.members, before=[], after=[1012, Eve, 17, 0], op=INSERT, meta=()}", + "AlterColumnTypeEvent{tableId=%s.members, nameMapping={age=DOUBLE}}", + "AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`precise_age` DOUBLE, position=LAST, existedColumnName=null}]}", + "AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`biological_sex` TINYINT, position=LAST, existedColumnName=null}]}", + "DataChangeEvent{tableId=%s.members, before=[], after=[1013, Fiona, null, null, 16.0, null], op=INSERT, meta=()}")); + } + + @Test + public void testFineGrainedSchemaEvolution() throws Exception { + + testGenericSchemaEvolution( + "evolve", + false, + false, + true, + Arrays.asList( + "AddColumnEvent{tableId=%s.members, addedColumns=[ColumnWithPosition{column=`gender` TINYINT, position=AFTER, existedColumnName=age}]}", + "DataChangeEvent{tableId=%s.members, before=[], after=[1012, Eve, 17, 0], op=INSERT, meta=()}", + "AlterColumnTypeEvent{tableId=%s.members, nameMapping={age=DOUBLE}}", + "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=()}"), + Collections.singletonList( + "Ignored schema change DropColumnEvent{tableId=%s.members, droppedColumnNames=[biological_sex]} to table %s.members.")); + } + + @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)); + } + + private void testGenericSchemaEvolution( + String behavior, + boolean mergeTable, + boolean triggerError, + boolean fineGrained, + List expectedTaskManagerEvents) + throws Exception { + testGenericSchemaEvolution( + behavior, + mergeTable, + triggerError, + fineGrained, + expectedTaskManagerEvents, + Collections.emptyList()); + } + + private void testGenericSchemaEvolution( + String behavior, + boolean mergeTable, + boolean triggerError, + boolean fineGrained, + List expectedTaskManagerEvents, + List expectedJobManagerEvents) + throws Exception { + + String dbName = schemaEvolveDatabase.getDatabaseName(); + + String pipelineJob = + String.format( + "source:\n" + + " type: mysql\n" + + " hostname: %s\n" + + " port: 3306\n" + + " username: %s\n" + + " password: %s\n" + + " tables: %s.%s\n" + + " server-id: 5400-5404\n" + + " server-time-zone: UTC\n" + + "\n" + + "sink:\n" + + " type: values\n" + + (fineGrained + ? " exclude.schema.changes:\n" + " - drop\n" + : "") + + (triggerError ? " error.on.schema.change: true" : "") + + (mergeTable + ? String.format( + "route:\n" + + " - source-table: %s.(members|new_members)\n" + + " sink-table: %s.merged", + dbName, dbName) + : "") + + "\n" + + "pipeline:\n" + + " schema.change.behavior: %s\n" + + " parallelism: 1", + INTER_CONTAINER_MYSQL_ALIAS, + MYSQL_TEST_USER, + MYSQL_TEST_PASSWORD, + dbName, + mergeTable ? "(members|new_members)" : "members", + behavior); + 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(dbName, mergeTable ? "merged" : "members"); + + LOG.info("Starting schema evolution"); + String mysqlJdbcUrl = + String.format( + "jdbc:mysql://%s:%s/%s", MYSQL.getHost(), MYSQL.getDatabasePort(), dbName); + + try (Connection conn = + DriverManager.getConnection( + mysqlJdbcUrl, MYSQL_TEST_USER, MYSQL_TEST_PASSWORD); + Statement stmt = conn.createStatement()) { + + waitForIncrementalStage(dbName, mergeTable ? "merged" : "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 expectedTmEvents = + expectedTaskManagerEvents.stream() + .map(s -> String.format(s, dbName, dbName)) + .collect(Collectors.toList()); + + validateResult(expectedTmEvents, taskManagerConsumer); + + List expectedJmEvents = + expectedJobManagerEvents.stream() + .map(s -> String.format(s, dbName, dbName)) + .collect(Collectors.toList()); + + validateResult(expectedJmEvents, jobManagerConsumer); + } + + private void validateSnapshotData(String dbName, 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, dbName, tableName)) + .collect(Collectors.toList()); + + validateResult(expected, taskManagerConsumer); + } + + private void waitForIncrementalStage(String dbName, 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=()}", + dbName, tableName), + taskManagerConsumer); + } + + 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 3f0999ee143..0691e88067c 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 @@ -62,6 +62,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 = @@ -137,14 +138,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( @@ -193,20 +192,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); @@ -246,6 +242,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( @@ -302,23 +302,17 @@ 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())); } @Test @@ -361,13 +355,13 @@ public void testTemporalFunctions() throws Exception { 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 d1f468bfe7c..bcb4476882b 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 @@ -17,15 +17,20 @@ package org.apache.flink.cdc.runtime.operators.schema; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; 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.exceptions.SchemaEvolveException; +import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; import org.apache.flink.cdc.common.route.RouteRule; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Schema; @@ -35,13 +40,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.ApplyOriginalSchemaChangeRequest; 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; @@ -100,7 +109,9 @@ public class SchemaOperator extends AbstractStreamOperator private transient TaskOperatorEventGateway toCoordinator; private transient SchemaEvolutionClient schemaEvolutionClient; - private transient LoadingCache cachedSchemas; + private transient LoadingCache originalSchema; + private transient LoadingCache evolvedSchema; + private transient LoadingCache schemaDivergesMap; /** * Storing mapping relations between upstream tableId (source table) mapping to downstream @@ -109,17 +120,42 @@ public class SchemaOperator extends AbstractStreamOperator private transient LoadingCache> tableIdMappingCache; 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 @@ -143,14 +179,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); + } + }); + originalSchema = + CacheBuilder.newBuilder() + .expireAfterAccess(CACHE_EXPIRE_DURATION) + .build( + new CacheLoader() { + @Override + public Schema load(TableId tableId) throws Exception { + return getLatestOriginalSchema(tableId); + } + }); + schemaDivergesMap = + CacheBuilder.newBuilder() + .expireAfterAccess(CACHE_EXPIRE_DURATION) + .build( + new CacheLoader() { + @Override + public Boolean load(TableId tableId) throws Exception { + return checkSchemaDiverges(tableId); } }); tableIdMappingCache = @@ -183,77 +239,109 @@ public void initializeState(StateInitializationContext context) throws Exception public void processElement(StreamRecord streamRecord) throws InterruptedException, TimeoutException, ExecutionException { 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)); + 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, ExecutionException { + TableId tableId = event.tableId(); + LOG.info("Table {} received SchemaChangeEvent and start to be blocked.", tableId); + handleSchemaChangeEvent(tableId, event); + // Update caches + originalSchema.put(tableId, getLatestOriginalSchema(tableId)); + schemaDivergesMap.put(tableId, checkSchemaDiverges(tableId)); + + List optionalRoutedTable = getRoutedTables(tableId); + if (!optionalRoutedTable.isEmpty()) { tableIdMappingCache .get(tableId) - .forEach(routed -> cachedSchemas.put(routed, getLatestSchema(routed))); - return; + .forEach(routed -> evolvedSchema.put(routed, getLatestEvolvedSchema(routed))); + } else { + evolvedSchema.put(tableId, getLatestEvolvedSchema(tableId)); } + } - // Data changes - DataChangeEvent dataChangeEvent = (DataChangeEvent) event; - List optionalRoutedTable = tableIdMappingCache.get(dataChangeEvent.tableId()); - if (optionalRoutedTable.isEmpty()) { - output.collect(streamRecord); - } else { + private void processDataChangeEvents(StreamRecord streamRecord, DataChangeEvent event) { + TableId tableId = event.tableId(); + List optionalRoutedTable = getRoutedTables(tableId); + if (!optionalRoutedTable.isEmpty()) { optionalRoutedTable.forEach( - route -> - output.collect( - new StreamRecord<>( - maybeFillInNullForEmptyColumns( - dataChangeEvent, route)))); + evolvedTableId -> { + output.collect( + new StreamRecord<>( + normalizeSchemaChangeEvents(event, evolvedTableId, 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 = this.originalSchema.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); @@ -261,7 +349,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()) { @@ -273,11 +364,18 @@ private RecordData regenerateRecordData( RecordData.FieldGetter fieldGetter = RecordData.createFieldGetter( originalSchema.getColumn(columnName).get().getType(), columnIndex); - // Check type compatibility - if (originalSchema.getColumn(columnName).get().getType().equals(column.getType())) { + // Check type compatibility, ignoring nullability + if (originalSchema + .getColumn(columnName) + .get() + .getType() + .nullable() + .equals(column.getType().nullable())) { fieldGetters.add(fieldGetter); } else { - fieldGetters.add(new TypeCoercionFieldGetter(column.getType(), fieldGetter)); + fieldGetters.add( + new TypeCoercionFieldGetter( + column.getType(), fieldGetter, tolerantMode)); } } } @@ -307,6 +405,16 @@ private TableId resolveReplacement( 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()) { @@ -315,9 +423,63 @@ private void handleSchemaChangeEvent(TableId tableId, SchemaChangeEvent schemaCh 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.LENIENT + || schemaChangeBehavior == SchemaChangeBehavior.IGNORE) { + if (schemaEvolveResponse.hasException()) { + schemaEvolveResponse + .getFailedSchemaChangeEvents() + .forEach( + e -> + LOG.warn( + "Failed to apply event {}, but keeps running in tolerant mode. Caused by: {}", + e.f0, + e.f1)); + } + } else { + throw new SchemaEvolveException( + schemaChangeEvent, + "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()); } } @@ -326,7 +488,18 @@ private SchemaChangeResponse requestSchemaChange( return sendRequestToCoordinator(new SchemaChangeRequest(tableId, schemaChangeEvent)); } - private void requestReleaseUpstream() throws InterruptedException, TimeoutException { + private void requestApplyOriginalSchemaChanges( + TableId tableId, SchemaChangeEvent schemaChangeEvent) { + sendRequestToCoordinator(new ApplyOriginalSchemaChangeRequest(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; @@ -338,6 +511,7 @@ private void requestReleaseUpstream() throws InterruptedException, TimeoutExcept throw new TimeoutException("TimeOut when requesting release upstream"); } } + return ((ReleaseUpstreamResponse) coordinationResponse); } private @@ -353,9 +527,24 @@ RESPONSE sendRequestToCoordinator(REQUEST request) { } } - private Schema getLatestSchema(TableId tableId) { + private Schema getLatestEvolvedSchema(TableId tableId) { + try { + Optional optionalSchema = schemaEvolutionClient.getLatestEvolvedSchema(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 Schema getLatestOriginalSchema(TableId tableId) { try { - Optional optionalSchema = schemaEvolutionClient.getLatestSchema(tableId); + Optional optionalSchema = + schemaEvolutionClient.getLatestOriginalSchema(tableId); if (!optionalSchema.isPresent()) { throw new IllegalStateException( String.format("Schema doesn't exist for table \"%s\"", tableId)); @@ -367,6 +556,15 @@ private Schema getLatestSchema(TableId tableId) { } } + private Boolean checkSchemaDiverges(TableId tableId) { + try { + return getLatestEvolvedSchema(tableId).equals(getLatestOriginalSchema(tableId)); + } catch (IllegalStateException e) { + // schema fetch failed, regard it as diverged + return true; + } + } + private static class NullFieldGetter implements RecordData.FieldGetter { @Nullable @Override @@ -378,11 +576,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 @@ -403,38 +612,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 eba7c771221..7cd35a20d3e 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 @@ -19,6 +19,7 @@ import org.apache.flink.cdc.common.annotation.Internal; import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; import org.apache.flink.cdc.common.route.RouteRule; import org.apache.flink.cdc.common.sink.MetadataApplier; import org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaRegistryProvider; @@ -40,17 +41,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)); + MetadataApplier metadataApplier, + List routingRules, + 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 b936da6080c..e4b547b216b 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 @@ -106,11 +106,11 @@ public List applySchemaChange(SchemaChangeEvent schemaChangeE // single source mapping, replace the table ID directly SchemaChangeEvent derivedSchemaChangeEvent = ChangeEventUtils.recreateSchemaChangeEvent(schemaChangeEvent, derivedTable); - schemaManager.applySchemaChange(derivedSchemaChangeEvent); events.add(derivedSchemaChangeEvent); } else { // multiple source mapping (merging tables) - Schema derivedTableSchema = schemaManager.getLatestSchema(derivedTable).get(); + Schema derivedTableSchema = + schemaManager.getLatestEvolvedSchema(derivedTable).get(); if (schemaChangeEvent instanceof CreateTableEvent) { events.addAll( handleCreateTableEvent( @@ -229,7 +229,6 @@ private List handleRenameColumnEvent( AddColumnEvent derivedSchemaChangeEvent = new AddColumnEvent(derivedTable, newColumns); schemaChangeEvents.add(derivedSchemaChangeEvent); } - schemaChangeEvents.forEach(schemaManager::applySchemaChange); return schemaChangeEvents; } @@ -261,7 +260,6 @@ private List handleAlterColumnTypeEvent( new AlterColumnTypeEvent(derivedTable, typeDifference); schemaChangeEvents.add(derivedSchemaChangeEvent); } - schemaChangeEvents.forEach(schemaManager::applySchemaChange); return schemaChangeEvents; } @@ -300,7 +298,6 @@ private List handleAddColumnEvent( if (!newTypeMapping.isEmpty()) { schemaChangeEvents.add(new AlterColumnTypeEvent(derivedTable, newTypeMapping)); } - schemaChangeEvents.forEach(schemaManager::applySchemaChange); return schemaChangeEvents; } @@ -336,7 +333,6 @@ private List handleCreateTableEvent( if (!newTypeMapping.isEmpty()) { schemaChangeEvents.add(new AlterColumnTypeEvent(derivedTable, newTypeMapping)); } - schemaChangeEvents.forEach(schemaManager::applySchemaChange); return schemaChangeEvents; } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaManager.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/coordinator/SchemaManager.java index 30fc86a6085..b87ef5b128c 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 @@ -21,6 +21,7 @@ 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 +49,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> originalSchemas; + + // Schema management + private final Map> evolvedSchemas; public SchemaManager() { - tableSchemas = new HashMap<>(); + evolvedSchemas = new HashMap<>(); + originalSchemas = new HashMap<>(); + behavior = SchemaChangeBehavior.EVOLVE; + } + + public SchemaManager(SchemaChangeBehavior behavior) { + evolvedSchemas = new HashMap<>(); + originalSchemas = new HashMap<>(); + this.behavior = behavior; + } + + public SchemaManager( + Map> originalSchemas, + Map> evolvedSchemas, + SchemaChangeBehavior behavior) { + this.evolvedSchemas = evolvedSchemas; + this.originalSchemas = originalSchemas; + this.behavior = behavior; + } + + public SchemaChangeBehavior getBehavior() { + return behavior; + } + + public final boolean schemaExists( + Map> schemaMap, TableId tableId) { + return schemaMap.containsKey(tableId) && !schemaMap.get(tableId).isEmpty(); + } + + public final boolean originalSchemaExists(TableId tableId) { + return schemaExists(originalSchemas, tableId); + } + + public final boolean evolvedSchemaExists(TableId tableId) { + return schemaExists(evolvedSchemas, tableId); } - public SchemaManager(Map> tableSchemas) { - this.tableSchemas = tableSchemas; + /** 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)); } - /** Check if schema exists for the specified table ID. */ - public final boolean schemaExists(TableId tableId) { - return tableSchemas.containsKey(tableId) && !tableSchemas.get(tableId).isEmpty(); + /** Get the latest original schema of the specified table. */ + public Optional getLatestOriginalSchema(TableId tableId) { + return getLatestSchemaVersion(originalSchemas, tableId) + .map(version -> originalSchemas.get(tableId).get(version)); } - /** Get the latest schema of the specified table. */ - public Optional getLatestSchema(TableId tableId) { - return getLatestSchemaVersion(tableId) - .map(version -> tableSchemas.get(tableId).get(version)); + /** Get schema at the specified version of a table. */ + public Schema getEvolvedSchema(TableId tableId, int version) { + checkArgument( + evolvedSchemas.containsKey(tableId), + "Unable to find evolved schema for table \"%s\"", + 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 getSchema(TableId tableId, int version) { + public Schema getOriginalSchema(TableId tableId, int version) { checkArgument( - tableSchemas.containsKey(tableId), - "Unable to find schema for table \"%s\"", + originalSchemas.containsKey(tableId), + "Unable to find original schema for table \"%s\"", tableId); - SortedMap versionedSchemas = tableSchemas.get(tableId); + SortedMap versionedSchemas = originalSchemas.get(tableId); checkArgument( versionedSchemas.containsKey(version), "Schema version %s does not exist for table \"%s\"", @@ -98,18 +149,38 @@ public Schema getSchema(TableId tableId, int version) { } /** Apply schema change to a table. */ - public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) { + public void applyOriginalSchemaChange(SchemaChangeEvent schemaChangeEvent) { if (schemaChangeEvent instanceof CreateTableEvent) { - handleCreateTableEvent(((CreateTableEvent) schemaChangeEvent)); + handleCreateTableEvent(originalSchemas, ((CreateTableEvent) schemaChangeEvent)); } else { - Optional optionalSchema = getLatestSchema(schemaChangeEvent.tableId()); + Optional optionalSchema = getLatestOriginalSchema(schemaChangeEvent.tableId()); checkArgument( optionalSchema.isPresent(), "Unable to apply SchemaChangeEvent for table \"%s\" without existing schema", schemaChangeEvent.tableId()); - LOG.info("Handling schema change event: {}", schemaChangeEvent); + LOG.info("Handling original schema change event: {}", schemaChangeEvent); registerNewSchema( + originalSchemas, + schemaChangeEvent.tableId(), + SchemaUtils.applySchemaChangeEvent(optionalSchema.get(), schemaChangeEvent)); + } + } + + /** Apply schema change to a table. */ + public void applyEvolvedSchemaChange(SchemaChangeEvent schemaChangeEvent) { + if (schemaChangeEvent instanceof CreateTableEvent) { + handleCreateTableEvent(evolvedSchemas, ((CreateTableEvent) schemaChangeEvent)); + } else { + Optional optionalSchema = getLatestEvolvedSchema(schemaChangeEvent.tableId()); + checkArgument( + optionalSchema.isPresent(), + "Unable to apply SchemaChangeEvent for table \"%s\" without existing schema", + schemaChangeEvent.tableId()); + + LOG.info("Handling evolved schema change event: {}", schemaChangeEvent); + registerNewSchema( + evolvedSchemas, schemaChangeEvent.tableId(), SchemaUtils.applySchemaChangeEvent(optionalSchema.get(), schemaChangeEvent)); } @@ -124,39 +195,45 @@ public boolean equals(Object o) { return false; } SchemaManager that = (SchemaManager) o; - return Objects.equals(tableSchemas, that.tableSchemas); + return Objects.equals(originalSchemas, that.originalSchemas) + && Objects.equals(evolvedSchemas, that.evolvedSchemas); } @Override public int hashCode() { - return Objects.hash(tableSchemas); + return Objects.hash(originalSchemas, 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) { + private void handleCreateTableEvent( + final Map> schemaMap, CreateTableEvent event) { checkArgument( - !schemaExists(event.tableId()), + !schemaExists(schemaMap, event.tableId()), "Unable to apply CreateTableEvent to an existing schema for table \"%s\"", event.tableId()); LOG.info("Handling schema change event: {}", event); - registerNewSchema(event.tableId(), event.getSchema()); + registerNewSchema(schemaMap, 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,7 +242,7 @@ 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); } } @@ -185,72 +262,95 @@ 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.originalSchemas, out); + out.writeUTF(schemaManager.getBehavior().name()); return baos.toByteArray(); } } + 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)); + } + } + } + @Override public SchemaManager deserialize(int version, byte[] serialized) throws IOException { - switch (version) { - case 0: - case 1: - case 2: - 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, original schema and evolved schema never differs + return new SchemaManager(schemas, schemas, SchemaChangeBehavior.EVOLVE); + } + case 2: + { + Map> evolvedSchemas = + deserializeSchemaMap(version, in); + Map> originalSchemas = + deserializeSchemaMap(version, in); + SchemaChangeBehavior behavior = + SchemaChangeBehavior.valueOf(in.readUTF()); + return new SchemaManager(originalSchemas, evolvedSchemas, behavior); } - return new SchemaManager(tableSchemas); - } - default: - throw new IOException("Unrecognized serialization version " + version); + default: + throw new RuntimeException("Unknown serialize version: " + version); + } + } + } + + 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 8de38e1406d..cc7c682070f 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 @@ -18,12 +18,19 @@ package org.apache.flink.cdc.runtime.operators.schema.coordinator; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; import org.apache.flink.cdc.common.route.RouteRule; 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.ApplyOriginalSchemaChangeRequest; +import org.apache.flink.cdc.runtime.operators.schema.event.ApplyOriginalSchemaChangeResponse; 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.GetOriginalSchemaRequest; +import org.apache.flink.cdc.runtime.operators.schema.event.GetOriginalSchemaResponse; 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; @@ -95,24 +102,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 @@ -176,9 +196,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 GetOriginalSchemaRequest) { + return CompletableFuture.completedFuture( + wrap(handleGetOriginalSchemaRequest((GetOriginalSchemaRequest) request))); + } else if (request instanceof ApplyOriginalSchemaChangeRequest) { + return CompletableFuture.completedFuture( + wrap( + handleApplyOriginalSchemaChangeRequest( + (ApplyOriginalSchemaChangeRequest) 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) { @@ -197,6 +230,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: { @@ -210,7 +244,10 @@ 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: @@ -228,7 +265,10 @@ public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData new SchemaDerivation(schemaManager, routes, derivationMapping); requestHandler = new SchemaRegistryRequestHandler( - metadataApplier, schemaManager, schemaDerivation); + metadataApplier, + schemaManager, + schemaDerivation, + schemaChangeBehavior); break; } default: @@ -258,22 +298,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 GetOriginalSchemaResponse handleGetOriginalSchemaRequest( + GetOriginalSchemaRequest getOriginalSchemaRequest) { + LOG.info("Handling original schema request: {}", getOriginalSchemaRequest); + int schemaVersion = getOriginalSchemaRequest.getSchemaVersion(); + TableId tableId = getOriginalSchemaRequest.getTableId(); + if (schemaVersion == GetOriginalSchemaRequest.LATEST_SCHEMA_VERSION) { + return new GetOriginalSchemaResponse( + schemaManager.getLatestOriginalSchema(tableId).orElse(null)); + } else { + try { + return new GetOriginalSchemaResponse( + schemaManager.getOriginalSchema(tableId, schemaVersion)); + } catch (IllegalArgumentException iae) { + LOG.warn( + "Some client is requesting an non-existed original schema for table {} with version {}", + tableId, + schemaVersion); + return new GetOriginalSchemaResponse(null); + } + } + } + + private ApplyOriginalSchemaChangeResponse handleApplyOriginalSchemaChangeRequest( + ApplyOriginalSchemaChangeRequest applyOriginalSchemaChangeRequest) { + schemaManager.applyOriginalSchemaChange( + applyOriginalSchemaChangeRequest.getSchemaChangeEvent()); + return new ApplyOriginalSchemaChangeResponse(); + } + + 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 0db0cf3a7ef..dd7f2dc36ca 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 @@ -18,6 +18,7 @@ package org.apache.flink.cdc.runtime.operators.schema.coordinator; import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; import org.apache.flink.cdc.common.route.RouteRule; import org.apache.flink.cdc.common.sink.MetadataApplier; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -34,16 +35,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 @@ -53,6 +57,7 @@ public OperatorID getOperatorId() { @Override public OperatorCoordinator create(OperatorCoordinator.Context context) throws Exception { - return new SchemaRegistry(operatorName, context, metadataApplier, routingRules); + return new SchemaRegistry( + operatorName, context, metadataApplier, routingRules, 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..77360169e24 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,11 +17,22 @@ 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.AddColumnEvent; +import org.apache.flink.cdc.common.event.AlterColumnTypeEvent; import org.apache.flink.cdc.common.event.CreateTableEvent; +import org.apache.flink.cdc.common.event.DropColumnEvent; +import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.exceptions.SchemaEvolveException; +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.sink.MetadataApplier; +import org.apache.flink.cdc.common.types.DataType; import org.apache.flink.cdc.runtime.operators.schema.event.RefreshPendingListsResponse; import org.apache.flink.cdc.runtime.operators.schema.event.ReleaseUpstreamRequest; import org.apache.flink.cdc.runtime.operators.schema.event.ReleaseUpstreamResponse; @@ -37,14 +48,19 @@ import java.io.Closeable; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.flink.cdc.runtime.operators.schema.coordinator.SchemaRegistryRequestHandler.RequestStatus.RECEIVED_RELEASE_REQUEST; import static org.apache.flink.cdc.runtime.operators.schema.event.CoordinationResponseUtils.wrap; @@ -69,28 +85,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 +128,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 (SchemaEvolveException e) { + LOG.error( + "Failed to apply schema change {} to table {}. Caused by: {}", + changeEvent, + tableId, + e); + failedSchemaChanges.add(Tuple2.of(changeEvent, e)); + } } - } 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; } /** @@ -131,13 +176,13 @@ public CompletableFuture handleSchemaChangeRequest( "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())) { + && schemaManager.originalSchemaExists(request.getTableId())) { return CompletableFuture.completedFuture( wrap(new SchemaChangeResponse(Collections.emptyList()))); } - schemaManager.applySchemaChange(request.getSchemaChangeEvent()); + schemaManager.applyOriginalSchemaChange(request.getSchemaChangeEvent()); List derivedSchemaChangeEvents = - schemaDerivation.applySchemaChange(request.getSchemaChangeEvent()); + calculateDerivedSchemaChangeEvents(request.getSchemaChangeEvent()); CompletableFuture response = CompletableFuture.completedFuture( wrap(new SchemaChangeResponse(derivedSchemaChangeEvents))); @@ -195,15 +240,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,15 +265,14 @@ private void startNextSchemaChangeRequest() { PendingSchemaChange pendingSchemaChange = pendingSchemaChanges.get(0); SchemaChangeRequest request = pendingSchemaChange.changeRequest; if (request.getSchemaChangeEvent() instanceof CreateTableEvent - && schemaManager.schemaExists(request.getTableId())) { + && schemaManager.evolvedSchemaExists(request.getTableId())) { pendingSchemaChange .getResponseFuture() .complete(wrap(new SchemaChangeResponse(Collections.emptyList()))); pendingSchemaChanges.remove(0); } else { - schemaManager.applySchemaChange(request.getSchemaChangeEvent()); List derivedSchemaChangeEvents = - schemaDerivation.applySchemaChange(request.getSchemaChangeEvent()); + calculateDerivedSchemaChangeEvents(request.getSchemaChangeEvent()); pendingSchemaChange .getResponseFuture() .complete(wrap(new SchemaChangeResponse(derivedSchemaChangeEvents))); @@ -243,13 +292,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))); } } @@ -260,6 +312,114 @@ public void close() throws IOException { } } + private List calculateDerivedSchemaChangeEvents(SchemaChangeEvent event) { + if (SchemaChangeBehavior.LENIENT.equals(schemaChangeBehavior)) { + return lenientizeSchemaChangeEvent(event).stream() + .flatMap(evt -> schemaDerivation.applySchemaChange(evt).stream()) + .collect(Collectors.toList()); + } else { + return schemaDerivation.applySchemaChange(event); + } + } + + private List lenientizeSchemaChangeEvent(SchemaChangeEvent event) { + if (event instanceof CreateTableEvent) { + return Collections.singletonList(event); + } + TableId tableId = event.tableId(); + Schema evolvedSchema = + schemaManager + .getLatestEvolvedSchema(tableId) + .orElseThrow( + () -> + new IllegalStateException( + "Evolved schema does not exist, not ready for schema change event " + + event)); + switch (event.getType()) { + case ADD_COLUMN: + { + AddColumnEvent addColumnEvent = (AddColumnEvent) event; + return Collections.singletonList( + new AddColumnEvent( + tableId, + addColumnEvent.getAddedColumns().stream() + .map( + col -> + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + col.getAddColumn() + .getName(), + col.getAddColumn() + .getType() + .nullable(), + col.getAddColumn() + .getComment()))) + .collect(Collectors.toList()))); + } + case DROP_COLUMN: + { + DropColumnEvent dropColumnEvent = (DropColumnEvent) event; + Map convertNullableColumns = + dropColumnEvent.getDroppedColumnNames().stream() + .map(evolvedSchema::getColumn) + .flatMap(e -> e.map(Stream::of).orElse(Stream.empty())) + .filter(col -> !col.getType().isNullable()) + .collect( + Collectors.toMap( + Column::getName, + column -> column.getType().nullable())); + + if (convertNullableColumns.isEmpty()) { + return Collections.emptyList(); + } else { + return Collections.singletonList( + new AlterColumnTypeEvent(tableId, convertNullableColumns)); + } + } + case RENAME_COLUMN: + { + RenameColumnEvent renameColumnEvent = (RenameColumnEvent) event; + List appendColumns = new ArrayList<>(); + Map convertNullableColumns = new HashMap<>(); + renameColumnEvent + .getNameMapping() + .forEach( + (key, value) -> { + Column column = + evolvedSchema + .getColumn(key) + .orElseThrow( + () -> + new IllegalArgumentException( + "Non-existed column " + + key + + " in evolved schema.")); + if (!column.getType().isNullable()) { + // It's a not-nullable column, we need to cast it to + // nullable first + convertNullableColumns.put( + key, column.getType().nullable()); + } + appendColumns.add( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + value, + column.getType().nullable(), + column.getComment()))); + }); + + List events = new ArrayList<>(); + events.add(new AddColumnEvent(tableId, appendColumns)); + if (!convertNullableColumns.isEmpty()) { + events.add(new AlterColumnTypeEvent(tableId, convertNullableColumns)); + } + return events; + } + default: + return Collections.singletonList(event); + } + } + private static class PendingSchemaChange { private final SchemaChangeRequest changeRequest; private List derivedSchemaChangeEvents; 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..787adfc5e41 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ApplyEvolvedSchemaChangeResponse.java @@ -0,0 +1,32 @@ +/* + * 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 SchemaRegistry} to {@link SchemaOperator} to request apply original + * schema changes, the evolved schema changes come from original schema changes with different + * schema evolution strategy. + */ +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/ApplyOriginalSchemaChangeRequest.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ApplyOriginalSchemaChangeRequest.java new file mode 100644 index 00000000000..d4c5d7feefa --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ApplyOriginalSchemaChangeRequest.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 original + * schema changes. + */ +public class ApplyOriginalSchemaChangeRequest 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 ApplyOriginalSchemaChangeRequest(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 ApplyOriginalSchemaChangeRequest)) { + return false; + } + ApplyOriginalSchemaChangeRequest that = (ApplyOriginalSchemaChangeRequest) 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/ApplyOriginalSchemaChangeResponse.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ApplyOriginalSchemaChangeResponse.java new file mode 100644 index 00000000000..0a92e965663 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/ApplyOriginalSchemaChangeResponse.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 SchemaRegistry} to {@link SchemaOperator} to request apply original + * schema changes. + */ +public class ApplyOriginalSchemaChangeResponse 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/GetOriginalSchemaRequest.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetOriginalSchemaRequest.java new file mode 100644 index 00000000000..36544da7746 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetOriginalSchemaRequest.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 original schema of a table. */ +@Internal +public class GetOriginalSchemaRequest implements CoordinationRequest { + public static final int LATEST_SCHEMA_VERSION = -1; + + private final TableId tableId; + private final int schemaVersion; + + public static GetOriginalSchemaRequest ofLatestSchema(TableId tableId) { + return new GetOriginalSchemaRequest(tableId, LATEST_SCHEMA_VERSION); + } + + public GetOriginalSchemaRequest(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 "GetOriginalSchemaRequest{" + + "tableId=" + + tableId + + ", schemaVersion=" + + schemaVersion + + '}'; + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetOriginalSchemaResponse.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetOriginalSchemaResponse.java new file mode 100644 index 00000000000..2c5343349c0 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/event/GetOriginalSchemaResponse.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 GetOriginalSchemaRequest}. */ +@Internal +public class GetOriginalSchemaResponse implements CoordinationResponse { + @Nullable private final Schema schema; + + public GetOriginalSchemaResponse(@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..4ba8e30c719 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/metrics/SchemaOperatorMetrics.java @@ -0,0 +1,87 @@ +/* + * 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.LENIENT, 1); + put(SchemaChangeBehavior.TRY_EVOLVE, 2); + put(SchemaChangeBehavior.EVOLVE, 3); + put(SchemaChangeBehavior.EXCEPTION, 4); + } + }; + + /** 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..aa256787f16 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.GetOriginalSchemaRequest; +import org.apache.flink.cdc.runtime.operators.schema.event.GetOriginalSchemaResponse; 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 getLatestOriginalSchema(TableId tableId) throws Exception { + GetOriginalSchemaResponse getOriginalSchemaResponse = + unwrap( + toCoordinator + .sendRequestToCoordinator( + schemaOperatorID, + new SerializedValue<>( + GetOriginalSchemaRequest.ofLatestSchema(tableId))) + .get()); + return getOriginalSchemaResponse.getSchema(); } } 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..9f3da6a6620 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 @@ -212,7 +212,8 @@ private SchemaChangeEvent cacheSchema(SchemaChangeEvent event) throws Exception private TableInfo getTableInfoFromSchemaEvolutionClient(TableId tableId) throws Exception { TableInfo tableInfo = tableInfoMap.get(tableId); if (tableInfo == null) { - Optional schemaOptional = schemaEvolutionClient.getLatestSchema(tableId); + Optional schemaOptional = + schemaEvolutionClient.getLatestOriginalSchema(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/partitioning/PrePartitionOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/PrePartitionOperator.java index 0c11005653f..13ddbb6b47f 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 @@ -114,7 +114,7 @@ private void broadcastEvent(Event toBroadcast) { private Schema loadLatestSchemaFromRegistry(TableId tableId) { Optional schema; try { - schema = schemaEvolutionClient.getLatestSchema(tableId); + schema = schemaEvolutionClient.getLatestEvolvedSchema(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/SchemaChangeEventSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/event/SchemaChangeEventSerializer.java index b2a9f0643c1..85de2fd7e07 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 @@ -26,6 +26,7 @@ import org.apache.flink.cdc.common.event.DropColumnEvent; import org.apache.flink.cdc.common.event.RenameColumnEvent; import org.apache.flink.cdc.common.event.SchemaChangeEvent; +import org.apache.flink.cdc.common.event.SchemaChangeEventType; import org.apache.flink.cdc.common.event.TableId; import org.apache.flink.cdc.runtime.serializer.EnumSerializer; import org.apache.flink.cdc.runtime.serializer.TypeSerializerSingleton; @@ -42,8 +43,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,7 +53,17 @@ public boolean isImmutableType() { @Override public SchemaChangeEvent createInstance() { - return () -> TableId.tableId("unknown", "unknown", "unknown"); + return new SchemaChangeEvent() { + @Override + public TableId tableId() { + return TableId.tableId("unknown", "unknown", "unknown"); + } + + @Override + public SchemaChangeEventType getType() { + return null; + } + }; } @Override @@ -85,20 +96,20 @@ public int getLength() { @Override public void serialize(SchemaChangeEvent record, DataOutputView target) throws IOException { if (record instanceof AlterColumnTypeEvent) { - enumSerializer.serialize(SchemaChangeEventClass.ALTER_COLUMN_TYPE, target); + enumSerializer.serialize(SchemaChangeEventType.ALTER_COLUMN_TYPE, target); AlterColumnTypeEventSerializer.INSTANCE.serialize( (AlterColumnTypeEvent) record, target); } else if (record instanceof CreateTableEvent) { - enumSerializer.serialize(SchemaChangeEventClass.CREATE_TABLE, target); + enumSerializer.serialize(SchemaChangeEventType.CREATE_TABLE, target); CreateTableEventSerializer.INSTANCE.serialize((CreateTableEvent) record, target); } else if (record instanceof RenameColumnEvent) { - enumSerializer.serialize(SchemaChangeEventClass.RENAME_COLUMN, target); + enumSerializer.serialize(SchemaChangeEventType.RENAME_COLUMN, target); RenameColumnEventSerializer.INSTANCE.serialize((RenameColumnEvent) record, target); } else if (record instanceof AddColumnEvent) { - enumSerializer.serialize(SchemaChangeEventClass.ADD_COLUMN, target); + enumSerializer.serialize(SchemaChangeEventType.ADD_COLUMN, target); AddColumnEventSerializer.INSTANCE.serialize((AddColumnEvent) record, target); } else if (record instanceof DropColumnEvent) { - enumSerializer.serialize(SchemaChangeEventClass.DROP_COLUMN, target); + enumSerializer.serialize(SchemaChangeEventType.DROP_COLUMN, target); DropColumnEventSerializer.INSTANCE.serialize((DropColumnEvent) record, target); } else { throw new IllegalArgumentException("Unknown schema change event: " + record); @@ -107,8 +118,8 @@ public void serialize(SchemaChangeEvent record, DataOutputView target) throws IO @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: @@ -121,7 +132,7 @@ public SchemaChangeEvent deserialize(DataInputView source) throws IOException { return AlterColumnTypeEventSerializer.INSTANCE.deserialize(source); default: throw new IllegalArgumentException( - "Unknown schema change event class: " + schemaChangeEventClass); + "Unknown schema change event class: " + schemaChangeEventType); } } @@ -150,12 +161,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/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..26a6276c058 --- /dev/null +++ b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/operators/schema/SchemaEvolveTest.java @@ -0,0 +1,2410 @@ +/* + * 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.getLatestOriginalSchema(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.getLatestOriginalSchema(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.getLatestOriginalSchema(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)), + 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.getLatestOriginalSchema(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.getLatestOriginalSchema(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.getLatestOriginalSchema(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.getLatestOriginalSchema(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.getLatestOriginalSchema(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)), + 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.getLatestOriginalSchema(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.getLatestOriginalSchema(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.getLatestOriginalSchema(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.getLatestOriginalSchema(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.getLatestOriginalSchema(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.getLatestOriginalSchema(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.getLatestOriginalSchema(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.getLatestOriginalSchema(tableId)).isEqualTo(schemaV5); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV1); + + harness.clearOutputRecords(); + } + harness.close(); + } + + /** Tests common evolve schema changes with exceptions expected. */ + @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.getLatestOriginalSchema(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.getLatestOriginalSchema(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.getLatestOriginalSchema(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.getLatestOriginalSchema(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.getLatestOriginalSchema(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.getLatestOriginalSchema(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.getLatestOriginalSchema(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.getLatestOriginalSchema(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.getLatestOriginalSchema(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.getLatestOriginalSchema(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.getLatestOriginalSchema(tableId)).isEqualTo(schemaV5); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV5E); + + harness.clearOutputRecords(); + } + harness.close(); + } + + /** Tests lenient schema change behavior. */ + @Test + public void testLenientSchemaEvolves() throws Exception { + TableId tableId = CUSTOMERS_TABLE_ID; + Schema schemaV1 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("name", STRING.notNull()) + .physicalColumn("age", SMALLINT) + .primaryKey("id") + .build(); + + SchemaChangeBehavior behavior = SchemaChangeBehavior.LENIENT; + + 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.getLatestOriginalSchema(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.notNull()) + .physicalColumn("age", SMALLINT) + .physicalColumn("score", INT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + Assertions.assertThat(harness.getLatestOriginalSchema(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); + + List lenientRenameColumnEvents = + Arrays.asList( + new AddColumnEvent( + tableId, + Arrays.asList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn("namae", STRING, null)), + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "toshi", SMALLINT, null)))), + new AlterColumnTypeEvent( + tableId, Collections.singletonMap("name", STRING)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 6, + STRING, + null, + SMALLINT, + null, + INT, + 100, + DOUBLE, + 173.25, + STRING, + "Fiona", + SMALLINT, + (short) 22)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 7, + STRING, + null, + SMALLINT, + null, + INT, + 97, + DOUBLE, + 160., + STRING, + "Gloria", + SMALLINT, + (short) 23))); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + lenientRenameColumnEvents)); + + Schema schemaV3 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING.notNull()) + .physicalColumn("toshi", SMALLINT) + .physicalColumn("score", INT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + + Schema schemaV3E = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("name", STRING) + .physicalColumn("age", SMALLINT) + .physicalColumn("score", INT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .physicalColumn("namae", STRING) + .physicalColumn("toshi", SMALLINT) + .primaryKey("id") + .build(); + + Assertions.assertThat(harness.getLatestOriginalSchema(tableId)).isEqualTo(schemaV3); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV3E); + + harness.clearOutputRecords(); + } + + // Test AlterColumnTypeEvent + { + List alterColumnTypeEvents = + Arrays.asList( + new AlterColumnTypeEvent( + tableId, ImmutableMap.of("score", BIGINT, "toshi", 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 lenientAlterColumnTypeEvents = + Arrays.asList( + new AlterColumnTypeEvent( + tableId, ImmutableMap.of("score", BIGINT, "toshi", FLOAT)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 8, STRING, null, SMALLINT, null, BIGINT, 100L, + DOUBLE, 173.25, STRING, "Helen", FLOAT, 22f)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 9, STRING, null, SMALLINT, null, BIGINT, 97L, + DOUBLE, 160., STRING, "Iva", FLOAT, 23f))); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + lenientAlterColumnTypeEvents)); + + Schema schemaV4 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING.notNull()) + .physicalColumn("toshi", FLOAT) + .physicalColumn("score", BIGINT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .primaryKey("id") + .build(); + + Schema schemaV4E = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("name", STRING) + .physicalColumn("age", SMALLINT) + .physicalColumn("score", BIGINT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .physicalColumn("namae", STRING) + .physicalColumn("toshi", FLOAT) + .primaryKey("id") + .build(); + Assertions.assertThat(harness.getLatestOriginalSchema(tableId)).isEqualTo(schemaV4); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV4E); + + 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); + + List lenientDropColumnEvents = + Arrays.asList( + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 12, STRING, null, SMALLINT, null, BIGINT, null, + DOUBLE, null, STRING, "Jane", FLOAT, 11f)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 13, STRING, null, SMALLINT, null, BIGINT, null, + DOUBLE, null, STRING, "Kryo", FLOAT, 23f))); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo(lenientDropColumnEvents); + + Schema schemaV5 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("namae", STRING.notNull()) + .physicalColumn("toshi", FLOAT) + .primaryKey("id") + .build(); + + Schema schemaV5E = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("name", STRING) + .physicalColumn("age", SMALLINT) + .physicalColumn("score", BIGINT, "Score data") + .physicalColumn("height", DOUBLE, "Height data") + .physicalColumn("namae", STRING) + .physicalColumn("toshi", FLOAT) + .primaryKey("id") + .build(); + Assertions.assertThat(harness.getLatestOriginalSchema(tableId)).isEqualTo(schemaV5); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV5E); + + harness.clearOutputRecords(); + } + harness.close(); + } + + @Test + public void testLenientEvolveTweaks() throws Exception { + TableId tableId = CUSTOMERS_TABLE_ID; + Schema schemaV1 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("iina", INT.notNull()) + .physicalColumn("name", STRING.notNull()) + .physicalColumn("age", SMALLINT) + .primaryKey("id") + .build(); + + SchemaChangeBehavior behavior = SchemaChangeBehavior.LENIENT; + + 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, INT, 0, STRING, "Alice", SMALLINT, (short) 17)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 2, INT, 0, STRING, "Bob", SMALLINT, (short) 18)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 3, + INT, + 0, + 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.getLatestOriginalSchema(tableId)).isEqualTo(schemaV1); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV1); + + harness.clearOutputRecords(); + } + + // Test drop a non-null column + { + List dropColumnEvents = + Arrays.asList( + new DropColumnEvent(tableId, Collections.singletonList("name")), + DataChangeEvent.insertEvent( + tableId, buildRecord(INT, 12, INT, 0, SMALLINT, (short) 11)), + DataChangeEvent.insertEvent( + tableId, buildRecord(INT, 13, INT, 0, SMALLINT, (short) 23))); + + processEvent(schemaOperator, dropColumnEvents); + + List lenientDropColumnEvents = + Arrays.asList( + new AlterColumnTypeEvent( + tableId, Collections.singletonMap("name", STRING)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 12, INT, 0, STRING, null, SMALLINT, (short) 11)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, 13, INT, 0, STRING, null, SMALLINT, (short) 23))); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + lenientDropColumnEvents)); + + Schema schemaV2 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("iina", INT.notNull()) + .physicalColumn("age", SMALLINT) + .primaryKey("id") + .build(); + + Schema schemaV2E = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("iina", INT.notNull()) + .physicalColumn("name", STRING) + .physicalColumn("age", SMALLINT) + .primaryKey("id") + .build(); + + Assertions.assertThat(harness.getLatestOriginalSchema(tableId)).isEqualTo(schemaV2); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV2E); + + harness.clearOutputRecords(); + } + + // Test inserting non-null column and somewhere in the middle + { + List addColumnEvents = + Arrays.asList( + new AddColumnEvent( + tableId, + Arrays.asList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn("nickname", STRING), + AddColumnEvent.ColumnPosition.AFTER, + "id"), + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn( + "extra", STRING.notNull())))), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 12, + STRING, + "Alice", + INT, + 0, + SMALLINT, + (short) 11, + STRING, + "ailisi")), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 13, + STRING, + "Bob", + INT, + 0, + SMALLINT, + (short) 23, + STRING, + "baobo"))); + + processEvent(schemaOperator, addColumnEvents); + + List lenientAddColumnEvents = + Arrays.asList( + new AddColumnEvent( + tableId, + Arrays.asList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn("nickname", STRING)), + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn("extra", STRING)))), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 12, + INT, + 0, + STRING, + null, + SMALLINT, + (short) 11, + STRING, + "Alice", + STRING, + "ailisi")), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 13, + INT, + 0, + STRING, + null, + SMALLINT, + (short) 23, + STRING, + "Bob", + STRING, + "baobo"))); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + lenientAddColumnEvents)); + + Schema schemaV3 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("nickname", STRING) + .physicalColumn("iina", INT.notNull()) + .physicalColumn("age", SMALLINT) + .physicalColumn("extra", STRING.notNull()) + .primaryKey("id") + .build(); + + Schema schemaV3E = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("iina", INT.notNull()) + .physicalColumn("name", STRING) + .physicalColumn("age", SMALLINT) + .physicalColumn("nickname", STRING) + .physicalColumn("extra", STRING) + .primaryKey("id") + .build(); + + Assertions.assertThat(harness.getLatestOriginalSchema(tableId)).isEqualTo(schemaV3); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV3E); + + harness.clearOutputRecords(); + } + + // Test renaming a non-null column + { + List renameColumnEvents = + Arrays.asList( + new RenameColumnEvent( + tableId, Collections.singletonMap("iina", "yina")), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 41, + STRING, + "Carol", + INT, + 0, + SMALLINT, + (short) 11, + STRING, + "kaluo")), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 42, + STRING, + "Dorothy", + INT, + 0, + SMALLINT, + (short) 11, + STRING, + "duoluoxi"))); + + processEvent(schemaOperator, renameColumnEvents); + + harness.getLatestEvolvedSchema(tableId); + List lenientRenameColumnEvents = + Arrays.asList( + new AddColumnEvent( + tableId, + Collections.singletonList( + new AddColumnEvent.ColumnWithPosition( + Column.physicalColumn("yina", INT)))), + new AlterColumnTypeEvent( + tableId, Collections.singletonMap("iina", INT)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 41, + INT, + null, + STRING, + null, + SMALLINT, + (short) 11, + STRING, + "Carol", + STRING, + "kaluo", + INT, + 0)), + DataChangeEvent.insertEvent( + tableId, + buildRecord( + INT, + 42, + INT, + null, + STRING, + null, + SMALLINT, + (short) 11, + STRING, + "Dorothy", + STRING, + "duoluoxi", + INT, + 0))); + + Assertions.assertThat( + harness.getOutputRecords().stream() + .map(StreamRecord::getValue) + .collect(Collectors.toList())) + .isEqualTo( + ListUtils.union( + Collections.singletonList(new FlushEvent(tableId)), + lenientRenameColumnEvents)); + + Schema schemaV4 = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("nickname", STRING) + .physicalColumn("yina", INT.notNull()) + .physicalColumn("age", SMALLINT) + .physicalColumn("extra", STRING.notNull()) + .primaryKey("id") + .build(); + + Schema schemaV4E = + Schema.newBuilder() + .physicalColumn("id", INT) + .physicalColumn("iina", INT) + .physicalColumn("name", STRING) + .physicalColumn("age", SMALLINT) + .physicalColumn("nickname", STRING) + .physicalColumn("extra", STRING) + .physicalColumn("yina", INT) + .primaryKey("id") + .build(); + + Assertions.assertThat(harness.getLatestOriginalSchema(tableId)).isEqualTo(schemaV4); + Assertions.assertThat(harness.getLatestEvolvedSchema(tableId)).isEqualTo(schemaV4E); + + harness.clearOutputRecords(); + } + } + + 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 adaf3b14002..9a2d1cfb4f9 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 @@ -147,8 +147,9 @@ void testOneToOneMapping() { @Test void testMergingTablesWithExactSameSchema() { + SchemaManager schemaManager = new SchemaManager(); SchemaDerivation schemaDerivation = - new SchemaDerivation(new SchemaManager(), ROUTES, new HashMap<>()); + new SchemaDerivation(schemaManager, ROUTES, new HashMap<>()); // Create table 1 List derivedChangesAfterCreateTable = @@ -158,6 +159,8 @@ void testMergingTablesWithExactSameSchema() { .asCreateTableEvent() .hasTableId(MERGED_TABLE) .hasSchema(SCHEMA); + derivedChangesAfterCreateTable.forEach(schemaManager::applyEvolvedSchemaChange); + // Create table 2 assertThat(schemaDerivation.applySchemaChange(new CreateTableEvent(TABLE_2, SCHEMA))) .isEmpty(); @@ -177,6 +180,8 @@ void testMergingTablesWithExactSameSchema() { .asAddColumnEvent() .hasTableId(MERGED_TABLE) .containsAddedColumns(newCol1, newCol2); + derivedChangesAfterAddColumn.forEach(schemaManager::applyEvolvedSchemaChange); + // Add column for table 2 assertThat(schemaDerivation.applySchemaChange(new AddColumnEvent(TABLE_2, newColumns))) .isEmpty(); @@ -190,6 +195,8 @@ void testMergingTablesWithExactSameSchema() { .asAlterColumnTypeEvent() .hasTableId(MERGED_TABLE) .containsTypeMapping(typeMapping); + derivedChangesAfterAlterColumnType.forEach(schemaManager::applyEvolvedSchemaChange); + // Alter column type for table 2 assertThat( schemaDerivation.applySchemaChange( @@ -215,6 +222,8 @@ void testMergingTablesWithExactSameSchema() { .containsAddedColumns( new AddColumnEvent.ColumnWithPosition( new PhysicalColumn("last_name", DataTypes.STRING(), null))); + derivedChangesAfterRenameColumn.forEach(schemaManager::applyEvolvedSchemaChange); + // Rename column for table 2 assertThat( schemaDerivation.applySchemaChange( @@ -235,6 +244,8 @@ void testMergingTableWithDifferentSchemas() { .asCreateTableEvent() .hasTableId(MERGED_TABLE) .hasSchema(SCHEMA); + derivedChangesAfterCreateTable.forEach(schemaManager::applyEvolvedSchemaChange); + // Create table 2 List derivedChangesAfterCreateTable2 = schemaDerivation.applySchemaChange( @@ -250,6 +261,7 @@ void testMergingTableWithDifferentSchemas() { "gender", DataTypes.STRING(), null)))), new AlterColumnTypeEvent( MERGED_TABLE, ImmutableMap.of("age", DataTypes.BIGINT()))); + derivedChangesAfterCreateTable2.forEach(schemaManager::applyEvolvedSchemaChange); // Add column for table 1 AddColumnEvent.ColumnWithPosition newCol1 = @@ -266,6 +278,8 @@ void testMergingTableWithDifferentSchemas() { .asAddColumnEvent() .hasTableId(MERGED_TABLE) .containsAddedColumns(newCol1, newCol2); + derivedChangesAfterAddColumn.forEach(schemaManager::applyEvolvedSchemaChange); + // Add column for table 2 List derivedChangesAfterAddColumnForTable2 = schemaDerivation.applySchemaChange( @@ -284,6 +298,7 @@ void testMergingTableWithDifferentSchemas() { .containsTypeMapping( ImmutableMap.of( "new_col1", DataTypes.STRING(), "new_col2", DataTypes.STRING())); + derivedChangesAfterAddColumnForTable2.forEach(schemaManager::applyEvolvedSchemaChange); // Alter column type for table 1 ImmutableMap typeMapping = ImmutableMap.of("age", DataTypes.BIGINT()); @@ -316,6 +331,8 @@ void testMergingTableWithDifferentSchemas() { .containsAddedColumns( new AddColumnEvent.ColumnWithPosition( new PhysicalColumn("last_name", DataTypes.STRING(), null))); + derivedChangesAfterRenameColumn.forEach(schemaManager::applyEvolvedSchemaChange); + // Rename column for table 2 List derivedChangesAfterRenameColumnForTable2 = schemaDerivation.applySchemaChange( @@ -327,8 +344,9 @@ void testMergingTableWithDifferentSchemas() { .containsAddedColumns( new AddColumnEvent.ColumnWithPosition( new PhysicalColumn("first_name", DataTypes.STRING(), null))); + derivedChangesAfterRenameColumnForTable2.forEach(schemaManager::applyEvolvedSchemaChange); - assertThat(schemaManager.getLatestSchema(MERGED_TABLE)) + assertThat(schemaManager.getLatestEvolvedSchema(MERGED_TABLE)) .contains( Schema.newBuilder() .column(Column.physicalColumn("id", DataTypes.BIGINT())) @@ -344,8 +362,9 @@ void testMergingTableWithDifferentSchemas() { @Test void testIncompatibleTypes() { + SchemaManager schemaManager = new SchemaManager(); SchemaDerivation schemaDerivation = - new SchemaDerivation(new SchemaManager(), ROUTES, new HashMap<>()); + new SchemaDerivation(schemaManager, ROUTES, new HashMap<>()); // Create table 1 List derivedChangesAfterCreateTable = schemaDerivation.applySchemaChange(new CreateTableEvent(TABLE_1, SCHEMA)); @@ -354,6 +373,7 @@ void testIncompatibleTypes() { .asCreateTableEvent() .hasTableId(MERGED_TABLE) .hasSchema(SCHEMA); + derivedChangesAfterCreateTable.forEach(schemaManager::applyEvolvedSchemaChange); // Create table 2 assertThatThrownBy( 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..88e1264623a 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.applyOriginalSchemaChange( + new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + schemaManager.applyOriginalSchemaChange( + new AlterColumnTypeEvent( + CUSTOMERS, ImmutableMap.of("phone", DataTypes.STRING()))); + assertThat(schemaManager.getLatestOriginalSchema(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.applyOriginalSchemaChange( + new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + schemaManager.applyOriginalSchemaChange( + new DropColumnEvent(CUSTOMERS, Arrays.asList("name", "phone"))); + assertThat(schemaManager.getLatestOriginalSchema(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.applyOriginalSchemaChange( + new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + schemaManager.applyOriginalSchemaChange( + new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("name", "new_name"))); + assertThat(schemaManager.getLatestOriginalSchema(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.applyOriginalSchemaChange( + new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + schemaManager.applyOriginalSchemaChange( + new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("name", "new_name"))); + schemaManager.applyOriginalSchemaChange( + new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("phone", "new_phone"))); + assertThat(schemaManager.getOriginalSchema(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.applyOriginalSchemaChange( + new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + schemaManager.applyOriginalSchemaChange( + new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("name", "new_name"))); + schemaManager.applyOriginalSchemaChange( + new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("phone", "new_phone"))); + schemaManager.applyOriginalSchemaChange( + new RenameColumnEvent(CUSTOMERS, ImmutableMap.of("new_phone", "new_phone_2"))); + assertThatThrownBy(() -> schemaManager.getOriginalSchema(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.applyOriginalSchemaChange( + new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA)); + schemaManager.applyOriginalSchemaChange( + 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/testutils/operators/EventOperatorTestHarness.java b/flink-cdc-runtime/src/test/java/org/apache/flink/cdc/runtime/testutils/operators/EventOperatorTestHarness.java index b96a17b56dc..b5262d2d5cd 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.ApplyOriginalSchemaChangeRequest; 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.GetOriginalSchemaRequest; +import org.apache.flink.cdc.runtime.operators.schema.event.GetOriginalSchemaResponse; 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 ApplyOriginalSchemaChangeRequest( + 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 getLatestOriginalSchema(TableId tableId) throws Exception { + return ((GetOriginalSchemaResponse) + unwrap( + schemaRegistry + .handleCoordinationRequest( + new GetOriginalSchemaRequest( + tableId, + GetOriginalSchemaRequest + .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..9398e1f1e9f 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,18 @@ 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.exceptions.SchemaEvolveException; 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,19 +39,55 @@ 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 void applySchemaChange(SchemaChangeEvent schemaChangeEvent) { + public boolean acceptsSchemaEvolutionType(SchemaChangeEventType schemaChangeEventType) { + return enabledEventTypes.contains(schemaChangeEventType); + } + + @Override + public Set getSupportedSchemaEvolutionTypes() { + return Arrays.stream(SchemaChangeEventTypeFamily.ALL).collect(Collectors.toSet()); + } + + @Override + public void applySchemaChange(SchemaChangeEvent schemaChangeEvent) + throws SchemaEvolveException { schemaChangeEvents.add(schemaChangeEvent); if (duration != null) { try { Thread.sleep(duration.toMillis()); - } catch (Exception ignore) { - + if (errorsOnEventTypes.contains(schemaChangeEvent.getType())) { + throw new SchemaEvolveException( + schemaChangeEvent, "Dummy metadata apply exception for test.", null); + } + } 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..1eebcb739fe 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 = INFO rootLogger.appenderRef.test.ref = TestLogger appender.testlogger.name = TestLogger