parent
9d150c07a3
commit
6017b16528
@ -1,43 +0,0 @@
|
||||
/*
|
||||
* 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.composer.flink.translator;
|
||||
|
||||
import org.apache.flink.cdc.common.event.Event;
|
||||
import org.apache.flink.cdc.common.event.TableId;
|
||||
import org.apache.flink.cdc.composer.definition.RouteDef;
|
||||
import org.apache.flink.cdc.runtime.operators.route.RouteFunction;
|
||||
import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo;
|
||||
import org.apache.flink.streaming.api.datastream.DataStream;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/** Translator used to build {@link RouteFunction}. */
|
||||
public class RouteTranslator {
|
||||
|
||||
public DataStream<Event> translate(DataStream<Event> input, List<RouteDef> routes) {
|
||||
if (routes.isEmpty()) {
|
||||
return input;
|
||||
}
|
||||
RouteFunction.Builder routeFunctionBuilder = RouteFunction.newBuilder();
|
||||
for (RouteDef route : routes) {
|
||||
routeFunctionBuilder.addRoute(
|
||||
route.getSourceTable(), TableId.parse(route.getSinkTable()));
|
||||
}
|
||||
return input.map(routeFunctionBuilder.build(), new EventTypeInfo()).name("Route");
|
||||
}
|
||||
}
|
@ -0,0 +1,315 @@
|
||||
/*
|
||||
* 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.coordinator;
|
||||
|
||||
import org.apache.flink.api.java.tuple.Tuple2;
|
||||
import org.apache.flink.cdc.common.event.AddColumnEvent;
|
||||
import org.apache.flink.cdc.common.event.AlterColumnTypeEvent;
|
||||
import org.apache.flink.cdc.common.event.CreateTableEvent;
|
||||
import org.apache.flink.cdc.common.event.DropColumnEvent;
|
||||
import org.apache.flink.cdc.common.event.RenameColumnEvent;
|
||||
import org.apache.flink.cdc.common.event.SchemaChangeEvent;
|
||||
import org.apache.flink.cdc.common.event.TableId;
|
||||
import org.apache.flink.cdc.common.schema.Column;
|
||||
import org.apache.flink.cdc.common.schema.PhysicalColumn;
|
||||
import org.apache.flink.cdc.common.schema.Schema;
|
||||
import org.apache.flink.cdc.common.schema.Selectors;
|
||||
import org.apache.flink.cdc.common.types.DataType;
|
||||
import org.apache.flink.cdc.common.types.DataTypeFamily;
|
||||
import org.apache.flink.cdc.common.types.DataTypes;
|
||||
import org.apache.flink.cdc.common.utils.ChangeEventUtils;
|
||||
import org.apache.flink.cdc.runtime.serializer.TableIdSerializer;
|
||||
import org.apache.flink.core.memory.DataInputViewStreamWrapper;
|
||||
import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
|
||||
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
|
||||
/** Derive schema changes based on the routing rules. */
|
||||
public class SchemaDerivation {
|
||||
private final SchemaManager schemaManager;
|
||||
private final List<Tuple2<Selectors, TableId>> routes;
|
||||
private final Map<TableId, Set<TableId>> derivationMapping;
|
||||
|
||||
public SchemaDerivation(
|
||||
SchemaManager schemaManager,
|
||||
List<Tuple2<Selectors, TableId>> routes,
|
||||
Map<TableId, Set<TableId>> derivationMapping) {
|
||||
this.schemaManager = schemaManager;
|
||||
this.routes = routes;
|
||||
this.derivationMapping = derivationMapping;
|
||||
}
|
||||
|
||||
public List<SchemaChangeEvent> applySchemaChange(SchemaChangeEvent schemaChangeEvent) {
|
||||
for (Tuple2<Selectors, TableId> route : routes) {
|
||||
TableId originalTable = schemaChangeEvent.tableId();
|
||||
|
||||
// Check routing table
|
||||
if (!route.f0.isMatch(originalTable)) {
|
||||
continue;
|
||||
}
|
||||
|
||||
// Matched a routing rule
|
||||
TableId derivedTable = route.f1;
|
||||
Set<TableId> originalTables =
|
||||
derivationMapping.computeIfAbsent(derivedTable, t -> new HashSet<>());
|
||||
originalTables.add(originalTable);
|
||||
|
||||
if (originalTables.size() == 1) {
|
||||
// 1-to-1 mapping. Replace the table ID directly
|
||||
SchemaChangeEvent derivedSchemaChangeEvent =
|
||||
ChangeEventUtils.recreateSchemaChangeEvent(schemaChangeEvent, derivedTable);
|
||||
schemaManager.applySchemaChange(derivedSchemaChangeEvent);
|
||||
return Collections.singletonList(derivedSchemaChangeEvent);
|
||||
}
|
||||
|
||||
// Many-to-1 mapping (merging tables)
|
||||
Schema derivedTableSchema = schemaManager.getLatestSchema(derivedTable).get();
|
||||
if (schemaChangeEvent instanceof CreateTableEvent) {
|
||||
return handleCreateTableEvent(
|
||||
(CreateTableEvent) schemaChangeEvent, derivedTableSchema, derivedTable);
|
||||
} else if (schemaChangeEvent instanceof AddColumnEvent) {
|
||||
return handleAddColumnEvent(
|
||||
(AddColumnEvent) schemaChangeEvent, derivedTableSchema, derivedTable);
|
||||
} else if (schemaChangeEvent instanceof AlterColumnTypeEvent) {
|
||||
return handleAlterColumnTypeEvent(
|
||||
(AlterColumnTypeEvent) schemaChangeEvent, derivedTableSchema, derivedTable);
|
||||
} else if (schemaChangeEvent instanceof DropColumnEvent) {
|
||||
return Collections.emptyList();
|
||||
} else if (schemaChangeEvent instanceof RenameColumnEvent) {
|
||||
return handleRenameColumnEvent(
|
||||
(RenameColumnEvent) schemaChangeEvent, derivedTableSchema, derivedTable);
|
||||
} else {
|
||||
throw new IllegalStateException(
|
||||
String.format(
|
||||
"Unrecognized SchemaChangeEvent type: %s", schemaChangeEvent));
|
||||
}
|
||||
}
|
||||
|
||||
// No routes are matched
|
||||
return Collections.singletonList(schemaChangeEvent);
|
||||
}
|
||||
|
||||
public Map<TableId, Set<TableId>> getDerivationMapping() {
|
||||
return derivationMapping;
|
||||
}
|
||||
|
||||
public static void serializeDerivationMapping(
|
||||
SchemaDerivation schemaDerivation, DataOutputStream out) throws IOException {
|
||||
TableIdSerializer tableIdSerializer = TableIdSerializer.INSTANCE;
|
||||
// Serialize derivation mapping in SchemaDerivation
|
||||
Map<TableId, Set<TableId>> derivationMapping = schemaDerivation.getDerivationMapping();
|
||||
out.write(derivationMapping.size());
|
||||
for (Map.Entry<TableId, Set<TableId>> entry : derivationMapping.entrySet()) {
|
||||
// Routed table ID
|
||||
TableId routedTableId = entry.getKey();
|
||||
tableIdSerializer.serialize(routedTableId, new DataOutputViewStreamWrapper(out));
|
||||
// Original table IDs
|
||||
Set<TableId> originalTableIds = entry.getValue();
|
||||
out.writeInt(originalTableIds.size());
|
||||
for (TableId originalTableId : originalTableIds) {
|
||||
tableIdSerializer.serialize(originalTableId, new DataOutputViewStreamWrapper(out));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static Map<TableId, Set<TableId>> deserializerDerivationMapping(DataInputStream in)
|
||||
throws IOException {
|
||||
TableIdSerializer tableIdSerializer = TableIdSerializer.INSTANCE;
|
||||
int derivationMappingSize = in.readInt();
|
||||
Map<TableId, Set<TableId>> derivationMapping = new HashMap<>(derivationMappingSize);
|
||||
for (int i = 0; i < derivationMappingSize; i++) {
|
||||
// Routed table ID
|
||||
TableId routedTableId =
|
||||
tableIdSerializer.deserialize(new DataInputViewStreamWrapper(in));
|
||||
// Original table IDs
|
||||
int numOriginalTables = in.readInt();
|
||||
Set<TableId> originalTableIds = new HashSet<>(numOriginalTables);
|
||||
for (int j = 0; j < numOriginalTables; j++) {
|
||||
TableId originalTableId =
|
||||
tableIdSerializer.deserialize(new DataInputViewStreamWrapper(in));
|
||||
originalTableIds.add(originalTableId);
|
||||
}
|
||||
derivationMapping.put(routedTableId, originalTableIds);
|
||||
}
|
||||
return derivationMapping;
|
||||
}
|
||||
|
||||
private List<SchemaChangeEvent> handleRenameColumnEvent(
|
||||
RenameColumnEvent renameColumnEvent, Schema derivedTableSchema, TableId derivedTable) {
|
||||
List<AddColumnEvent.ColumnWithPosition> newColumns = new ArrayList<>();
|
||||
renameColumnEvent
|
||||
.getNameMapping()
|
||||
.forEach(
|
||||
(before, after) -> {
|
||||
if (derivedTableSchema.getColumn(after).isPresent()) {
|
||||
return;
|
||||
}
|
||||
Column existedColumn = derivedTableSchema.getColumn(before).get();
|
||||
newColumns.add(
|
||||
new AddColumnEvent.ColumnWithPosition(
|
||||
new PhysicalColumn(
|
||||
after,
|
||||
existedColumn.getType(),
|
||||
existedColumn.getComment())));
|
||||
});
|
||||
List<SchemaChangeEvent> schemaChangeEvents = new ArrayList<>();
|
||||
if (!newColumns.isEmpty()) {
|
||||
AddColumnEvent derivedSchemaChangeEvent = new AddColumnEvent(derivedTable, newColumns);
|
||||
schemaChangeEvents.add(derivedSchemaChangeEvent);
|
||||
}
|
||||
schemaChangeEvents.forEach(schemaManager::applySchemaChange);
|
||||
return schemaChangeEvents;
|
||||
}
|
||||
|
||||
private List<SchemaChangeEvent> handleAlterColumnTypeEvent(
|
||||
AlterColumnTypeEvent alterColumnTypeEvent,
|
||||
Schema derivedTableSchema,
|
||||
TableId derivedTable) {
|
||||
Map<String, DataType> typeDifference = new HashMap<>();
|
||||
alterColumnTypeEvent
|
||||
.getTypeMapping()
|
||||
.forEach(
|
||||
(columnName, dataType) -> {
|
||||
Column existedColumnInDerivedTable =
|
||||
derivedTableSchema.getColumn(columnName).get();
|
||||
if (!existedColumnInDerivedTable.getType().equals(dataType)) {
|
||||
// Check type compatibility
|
||||
DataType widerType =
|
||||
getWiderType(
|
||||
existedColumnInDerivedTable.getType(), dataType);
|
||||
if (!widerType.equals(existedColumnInDerivedTable.getType())) {
|
||||
typeDifference.put(
|
||||
existedColumnInDerivedTable.getName(), widerType);
|
||||
}
|
||||
}
|
||||
});
|
||||
List<SchemaChangeEvent> schemaChangeEvents = new ArrayList<>();
|
||||
if (!typeDifference.isEmpty()) {
|
||||
AlterColumnTypeEvent derivedSchemaChangeEvent =
|
||||
new AlterColumnTypeEvent(derivedTable, typeDifference);
|
||||
schemaChangeEvents.add(derivedSchemaChangeEvent);
|
||||
}
|
||||
schemaChangeEvents.forEach(schemaManager::applySchemaChange);
|
||||
return schemaChangeEvents;
|
||||
}
|
||||
|
||||
private List<SchemaChangeEvent> handleAddColumnEvent(
|
||||
AddColumnEvent addColumnEvent, Schema derivedTableSchema, TableId derivedTable) {
|
||||
List<AddColumnEvent.ColumnWithPosition> newColumns = new ArrayList<>();
|
||||
Map<String, DataType> newTypeMapping = new HashMap<>();
|
||||
// Check if new column already existed in the derived table
|
||||
for (AddColumnEvent.ColumnWithPosition addedColumn : addColumnEvent.getAddedColumns()) {
|
||||
Optional<Column> optionalColumnInDerivedTable =
|
||||
derivedTableSchema.getColumn(addedColumn.getAddColumn().getName());
|
||||
if (!optionalColumnInDerivedTable.isPresent()) {
|
||||
// Non-existed column. Use AddColumn
|
||||
newColumns.add(new AddColumnEvent.ColumnWithPosition(addedColumn.getAddColumn()));
|
||||
} else {
|
||||
// Existed column. Check type compatibility
|
||||
Column existedColumnInDerivedTable = optionalColumnInDerivedTable.get();
|
||||
if (!existedColumnInDerivedTable
|
||||
.getType()
|
||||
.equals(addedColumn.getAddColumn().getType())) {
|
||||
DataType widerType =
|
||||
getWiderType(
|
||||
existedColumnInDerivedTable.getType(),
|
||||
addedColumn.getAddColumn().getType());
|
||||
if (!widerType.equals(existedColumnInDerivedTable.getType())) {
|
||||
newTypeMapping.put(existedColumnInDerivedTable.getName(), widerType);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
List<SchemaChangeEvent> schemaChangeEvents = new ArrayList<>();
|
||||
if (!newColumns.isEmpty()) {
|
||||
schemaChangeEvents.add(new AddColumnEvent(derivedTable, newColumns));
|
||||
}
|
||||
if (!newTypeMapping.isEmpty()) {
|
||||
schemaChangeEvents.add(new AlterColumnTypeEvent(derivedTable, newTypeMapping));
|
||||
}
|
||||
schemaChangeEvents.forEach(schemaManager::applySchemaChange);
|
||||
return schemaChangeEvents;
|
||||
}
|
||||
|
||||
private List<SchemaChangeEvent> handleCreateTableEvent(
|
||||
CreateTableEvent createTableEvent, Schema derivedTableSchema, TableId derivedTable) {
|
||||
List<AddColumnEvent.ColumnWithPosition> newColumns = new ArrayList<>();
|
||||
Map<String, DataType> newTypeMapping = new HashMap<>();
|
||||
// Check if there is any columns that doesn't exist in the derived table
|
||||
// and perform add-column for non-existed columns.
|
||||
for (Column column : createTableEvent.getSchema().getColumns()) {
|
||||
Optional<Column> optionalColumnInDerivedTable =
|
||||
derivedTableSchema.getColumn(column.getName());
|
||||
if (!optionalColumnInDerivedTable.isPresent()) {
|
||||
// Non-existed column. Use AddColumn
|
||||
newColumns.add(new AddColumnEvent.ColumnWithPosition(column));
|
||||
} else {
|
||||
// Existed column. Check type compatibility
|
||||
Column existedColumnInDerivedTable = optionalColumnInDerivedTable.get();
|
||||
if (!existedColumnInDerivedTable.getType().equals(column.getType())) {
|
||||
DataType widerType =
|
||||
getWiderType(existedColumnInDerivedTable.getType(), column.getType());
|
||||
if (!widerType.equals(existedColumnInDerivedTable.getType())) {
|
||||
newTypeMapping.put(existedColumnInDerivedTable.getName(), widerType);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
List<SchemaChangeEvent> schemaChangeEvents = new ArrayList<>();
|
||||
if (!newColumns.isEmpty()) {
|
||||
schemaChangeEvents.add(new AddColumnEvent(derivedTable, newColumns));
|
||||
}
|
||||
if (!newTypeMapping.isEmpty()) {
|
||||
schemaChangeEvents.add(new AlterColumnTypeEvent(derivedTable, newTypeMapping));
|
||||
}
|
||||
schemaChangeEvents.forEach(schemaManager::applySchemaChange);
|
||||
return schemaChangeEvents;
|
||||
}
|
||||
|
||||
private DataType getWiderType(DataType thisType, DataType thatType) {
|
||||
if (thisType.equals(thatType)) {
|
||||
return thisType;
|
||||
}
|
||||
if (thisType.is(DataTypeFamily.INTEGER_NUMERIC)
|
||||
&& thatType.is(DataTypeFamily.INTEGER_NUMERIC)) {
|
||||
return DataTypes.BIGINT();
|
||||
}
|
||||
if (thisType.is(DataTypeFamily.CHARACTER_STRING)
|
||||
&& thatType.is(DataTypeFamily.CHARACTER_STRING)) {
|
||||
return DataTypes.STRING();
|
||||
}
|
||||
if (thisType.is(DataTypeFamily.APPROXIMATE_NUMERIC)
|
||||
&& thatType.is(DataTypeFamily.APPROXIMATE_NUMERIC)) {
|
||||
return DataTypes.DOUBLE();
|
||||
}
|
||||
throw new IllegalStateException(
|
||||
String.format("Incompatible types: \"%s\" and \"%s\"", thisType, thatType));
|
||||
}
|
||||
}
|
@ -1,195 +0,0 @@
|
||||
/*
|
||||
* 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.route;
|
||||
|
||||
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.OperationType;
|
||||
import org.apache.flink.cdc.common.event.RenameColumnEvent;
|
||||
import org.apache.flink.cdc.common.event.TableId;
|
||||
import org.apache.flink.cdc.common.schema.Column;
|
||||
import org.apache.flink.cdc.common.schema.PhysicalColumn;
|
||||
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.typeutils.BinaryRecordDataGenerator;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
|
||||
import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.apache.flink.cdc.common.testutils.assertions.EventAssertions.assertThat;
|
||||
|
||||
class RouteFunctionTest {
|
||||
private static final TableId CUSTOMERS =
|
||||
TableId.tableId("my_company", "my_branch", "customers");
|
||||
private static final TableId NEW_CUSTOMERS =
|
||||
TableId.tableId("my_new_company", "my_new_branch", "customers");
|
||||
private static final Schema CUSTOMERS_SCHEMA =
|
||||
Schema.newBuilder()
|
||||
.physicalColumn("id", DataTypes.INT())
|
||||
.physicalColumn("name", DataTypes.STRING())
|
||||
.physicalColumn("phone", DataTypes.BIGINT())
|
||||
.primaryKey("id")
|
||||
.build();
|
||||
|
||||
@Test
|
||||
void testDataChangeEventRouting() throws Exception {
|
||||
RouteFunction router =
|
||||
RouteFunction.newBuilder()
|
||||
.addRoute("my_company.\\.+.customers", NEW_CUSTOMERS)
|
||||
.build();
|
||||
router.open(new Configuration());
|
||||
BinaryRecordDataGenerator recordDataGenerator =
|
||||
new BinaryRecordDataGenerator(((RowType) CUSTOMERS_SCHEMA.toRowDataType()));
|
||||
|
||||
// Insert
|
||||
DataChangeEvent insertEvent =
|
||||
DataChangeEvent.insertEvent(
|
||||
CUSTOMERS,
|
||||
recordDataGenerator.generate(
|
||||
new Object[] {1, new BinaryStringData("Alice"), 12345678L}));
|
||||
assertThat(router.map(insertEvent))
|
||||
.asDataChangeEvent()
|
||||
.hasTableId(NEW_CUSTOMERS)
|
||||
.hasOperationType(OperationType.INSERT)
|
||||
.withAfterRecordData()
|
||||
.hasArity(3)
|
||||
.withSchema(CUSTOMERS_SCHEMA)
|
||||
.hasFields(1, new BinaryStringData("Alice"), 12345678L);
|
||||
|
||||
// Update
|
||||
DataChangeEvent updateEvent =
|
||||
DataChangeEvent.updateEvent(
|
||||
CUSTOMERS,
|
||||
recordDataGenerator.generate(
|
||||
new Object[] {1, new BinaryStringData("Alice"), 12345678L}),
|
||||
recordDataGenerator.generate(
|
||||
new Object[] {1, new BinaryStringData("Alice"), 87654321L}));
|
||||
DataChangeEvent mappedUpdateEvent = (DataChangeEvent) router.map(updateEvent);
|
||||
assertThat(mappedUpdateEvent)
|
||||
.hasTableId(NEW_CUSTOMERS)
|
||||
.hasOperationType(OperationType.UPDATE);
|
||||
assertThat(mappedUpdateEvent.before())
|
||||
.withSchema(CUSTOMERS_SCHEMA)
|
||||
.hasFields(1, new BinaryStringData("Alice"), 12345678L);
|
||||
assertThat(mappedUpdateEvent.after())
|
||||
.withSchema(CUSTOMERS_SCHEMA)
|
||||
.hasFields(1, new BinaryStringData("Alice"), 87654321L);
|
||||
|
||||
// Replace
|
||||
DataChangeEvent replaceEvent =
|
||||
DataChangeEvent.replaceEvent(
|
||||
CUSTOMERS,
|
||||
recordDataGenerator.generate(
|
||||
new Object[] {1, new BinaryStringData("Bob"), 87654321L}));
|
||||
assertThat(router.map(replaceEvent))
|
||||
.asDataChangeEvent()
|
||||
.hasTableId(NEW_CUSTOMERS)
|
||||
.hasOperationType(OperationType.REPLACE)
|
||||
.withAfterRecordData()
|
||||
.hasArity(3)
|
||||
.withSchema(CUSTOMERS_SCHEMA)
|
||||
.hasFields(1, new BinaryStringData("Bob"), 87654321L);
|
||||
|
||||
// Delete
|
||||
DataChangeEvent deleteEvent =
|
||||
DataChangeEvent.deleteEvent(
|
||||
CUSTOMERS,
|
||||
recordDataGenerator.generate(
|
||||
new Object[] {1, new BinaryStringData("Bob"), 87654321L}));
|
||||
assertThat(router.map(deleteEvent))
|
||||
.asDataChangeEvent()
|
||||
.hasTableId(NEW_CUSTOMERS)
|
||||
.hasOperationType(OperationType.DELETE)
|
||||
.withBeforeRecordData()
|
||||
.hasArity(3)
|
||||
.withSchema(CUSTOMERS_SCHEMA)
|
||||
.hasFields(1, new BinaryStringData("Bob"), 87654321L);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSchemaChangeEventRouting() throws Exception {
|
||||
RouteFunction router =
|
||||
RouteFunction.newBuilder()
|
||||
.addRoute("\\.+_company.\\.+_branch.customers", NEW_CUSTOMERS)
|
||||
.build();
|
||||
router.open(new Configuration());
|
||||
|
||||
// CreateTableEvent
|
||||
CreateTableEvent createTableEvent = new CreateTableEvent(CUSTOMERS, CUSTOMERS_SCHEMA);
|
||||
assertThat(router.map(createTableEvent))
|
||||
.asSchemaChangeEvent()
|
||||
.hasTableId(NEW_CUSTOMERS)
|
||||
.asCreateTableEvent()
|
||||
.hasSchema(CUSTOMERS_SCHEMA);
|
||||
|
||||
// AddColumnEvent
|
||||
AddColumnEvent.ColumnWithPosition newColumn =
|
||||
new AddColumnEvent.ColumnWithPosition(
|
||||
Column.physicalColumn("address", DataTypes.STRING()),
|
||||
AddColumnEvent.ColumnPosition.LAST,
|
||||
null);
|
||||
AddColumnEvent addColumnEvent =
|
||||
new AddColumnEvent(CUSTOMERS, Collections.singletonList(newColumn));
|
||||
assertThat(router.map(addColumnEvent))
|
||||
.asSchemaChangeEvent()
|
||||
.asAddColumnEvent()
|
||||
.hasTableId(NEW_CUSTOMERS)
|
||||
.containsAddedColumns(newColumn);
|
||||
|
||||
// DropColumnEvent
|
||||
PhysicalColumn droppedColumn = Column.physicalColumn("address", DataTypes.STRING());
|
||||
List<String> droppedColumns = Collections.singletonList(droppedColumn.getName());
|
||||
DropColumnEvent dropColumnEvent = new DropColumnEvent(CUSTOMERS, droppedColumns);
|
||||
assertThat(router.map(dropColumnEvent))
|
||||
.asSchemaChangeEvent()
|
||||
.asDropColumnEvent()
|
||||
.containsDroppedColumns(droppedColumn.getName())
|
||||
.hasTableId(NEW_CUSTOMERS);
|
||||
|
||||
// RenameColumnEvent
|
||||
Map<String, String> columnRenaming = ImmutableMap.of("phone", "mobile");
|
||||
RenameColumnEvent renameColumnEvent = new RenameColumnEvent(CUSTOMERS, columnRenaming);
|
||||
assertThat(router.map(renameColumnEvent))
|
||||
.asSchemaChangeEvent()
|
||||
.asRenameColumnEvent()
|
||||
.containsNameMapping(columnRenaming)
|
||||
.hasTableId(NEW_CUSTOMERS);
|
||||
|
||||
// AlterColumnTypeEvent
|
||||
Map<String, DataType> typeMapping = ImmutableMap.of("mobile", DataTypes.STRING());
|
||||
AlterColumnTypeEvent alterColumnTypeEvent =
|
||||
new AlterColumnTypeEvent(CUSTOMERS, typeMapping);
|
||||
assertThat(router.map(alterColumnTypeEvent))
|
||||
.asSchemaChangeEvent()
|
||||
.asAlterColumnTypeEvent()
|
||||
.containsTypeMapping(typeMapping)
|
||||
.hasTableId(NEW_CUSTOMERS);
|
||||
}
|
||||
}
|
@ -0,0 +1,365 @@
|
||||
/*
|
||||
* 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.coordinator;
|
||||
|
||||
import org.apache.flink.api.java.tuple.Tuple2;
|
||||
import org.apache.flink.cdc.common.event.AddColumnEvent;
|
||||
import org.apache.flink.cdc.common.event.AlterColumnTypeEvent;
|
||||
import org.apache.flink.cdc.common.event.CreateTableEvent;
|
||||
import org.apache.flink.cdc.common.event.DropColumnEvent;
|
||||
import org.apache.flink.cdc.common.event.RenameColumnEvent;
|
||||
import org.apache.flink.cdc.common.event.SchemaChangeEvent;
|
||||
import org.apache.flink.cdc.common.event.TableId;
|
||||
import org.apache.flink.cdc.common.schema.Column;
|
||||
import org.apache.flink.cdc.common.schema.PhysicalColumn;
|
||||
import org.apache.flink.cdc.common.schema.Schema;
|
||||
import org.apache.flink.cdc.common.schema.Selectors;
|
||||
import org.apache.flink.cdc.common.types.DataType;
|
||||
import org.apache.flink.cdc.common.types.DataTypes;
|
||||
|
||||
import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.apache.flink.cdc.common.testutils.assertions.EventAssertions.assertThat;
|
||||
import static org.assertj.core.api.Assertions.assertThatThrownBy;
|
||||
|
||||
/** Unit test for {@link SchemaDerivation}. */
|
||||
class SchemaDerivationTest {
|
||||
|
||||
private static final TableId TABLE_1 = TableId.tableId("mydb", "myschema", "mytable1");
|
||||
private static final TableId TABLE_2 = TableId.tableId("mydb", "myschema", "mytable2");
|
||||
private static final TableId MERGED_TABLE = TableId.tableId("mydb", "myschema", "mytables");
|
||||
|
||||
private static final Schema SCHEMA =
|
||||
Schema.newBuilder()
|
||||
.column(Column.physicalColumn("id", DataTypes.BIGINT()))
|
||||
.column(Column.physicalColumn("name", DataTypes.STRING()))
|
||||
.column(Column.physicalColumn("age", DataTypes.INT()))
|
||||
.build();
|
||||
|
||||
private static final Schema COMPATIBLE_SCHEMA =
|
||||
Schema.newBuilder()
|
||||
.column(Column.physicalColumn("id", DataTypes.BIGINT()))
|
||||
.column(Column.physicalColumn("name", DataTypes.STRING()))
|
||||
.column(Column.physicalColumn("age", DataTypes.BIGINT()))
|
||||
.column(Column.physicalColumn("gender", DataTypes.STRING()))
|
||||
.build();
|
||||
|
||||
private static final Schema INCOMPATIBLE_SCHEMA =
|
||||
Schema.newBuilder()
|
||||
.column(Column.physicalColumn("id", DataTypes.BIGINT()))
|
||||
.column(Column.physicalColumn("name", DataTypes.STRING()))
|
||||
.column(Column.physicalColumn("age", DataTypes.STRING()))
|
||||
.column(Column.physicalColumn("gender", DataTypes.STRING()))
|
||||
.build();
|
||||
|
||||
private static final List<Tuple2<Selectors, TableId>> ROUTES =
|
||||
Collections.singletonList(
|
||||
Tuple2.of(
|
||||
new Selectors.SelectorsBuilder()
|
||||
.includeTables("mydb.myschema.mytable[0-9]")
|
||||
.build(),
|
||||
MERGED_TABLE));
|
||||
|
||||
@Test
|
||||
void testOneToOneMapping() {
|
||||
SchemaDerivation schemaDerivation =
|
||||
new SchemaDerivation(new SchemaManager(), ROUTES, new HashMap<>());
|
||||
|
||||
// Create table
|
||||
List<SchemaChangeEvent> derivedChangesAfterCreateTable =
|
||||
schemaDerivation.applySchemaChange(new CreateTableEvent(TABLE_1, SCHEMA));
|
||||
assertThat(derivedChangesAfterCreateTable).hasSize(1);
|
||||
assertThat(derivedChangesAfterCreateTable.get(0))
|
||||
.asCreateTableEvent()
|
||||
.hasTableId(MERGED_TABLE)
|
||||
.hasSchema(SCHEMA);
|
||||
|
||||
// Add column
|
||||
AddColumnEvent.ColumnWithPosition newCol1 =
|
||||
new AddColumnEvent.ColumnWithPosition(
|
||||
new PhysicalColumn("new_col1", DataTypes.STRING(), null));
|
||||
AddColumnEvent.ColumnWithPosition newCol2 =
|
||||
new AddColumnEvent.ColumnWithPosition(
|
||||
new PhysicalColumn("new_col2", DataTypes.STRING(), null));
|
||||
List<AddColumnEvent.ColumnWithPosition> newColumns = Arrays.asList(newCol1, newCol2);
|
||||
List<SchemaChangeEvent> derivedChangesAfterAddColumn =
|
||||
schemaDerivation.applySchemaChange(new AddColumnEvent(TABLE_1, newColumns));
|
||||
assertThat(derivedChangesAfterAddColumn).hasSize(1);
|
||||
assertThat(derivedChangesAfterAddColumn.get(0))
|
||||
.asAddColumnEvent()
|
||||
.hasTableId(MERGED_TABLE)
|
||||
.containsAddedColumns(newCol1, newCol2);
|
||||
|
||||
// Alter column type
|
||||
ImmutableMap<String, DataType> typeMapping = ImmutableMap.of("age", DataTypes.BIGINT());
|
||||
List<SchemaChangeEvent> derivedChangesAfterAlterTableType =
|
||||
schemaDerivation.applySchemaChange(new AlterColumnTypeEvent(TABLE_1, typeMapping));
|
||||
assertThat(derivedChangesAfterAlterTableType).hasSize(1);
|
||||
assertThat(derivedChangesAfterAlterTableType.get(0))
|
||||
.asAlterColumnTypeEvent()
|
||||
.hasTableId(MERGED_TABLE)
|
||||
.containsTypeMapping(typeMapping);
|
||||
|
||||
// Drop column
|
||||
List<String> droppedColumns = Arrays.asList("new_col1", "new_col2");
|
||||
List<SchemaChangeEvent> derivedChangesAfterDropColumn =
|
||||
schemaDerivation.applySchemaChange(new DropColumnEvent(TABLE_1, droppedColumns));
|
||||
assertThat(derivedChangesAfterDropColumn).hasSize(1);
|
||||
assertThat(derivedChangesAfterDropColumn.get(0))
|
||||
.asDropColumnEvent()
|
||||
.hasTableId(MERGED_TABLE)
|
||||
.containsDroppedColumns("new_col1", "new_col2");
|
||||
|
||||
// Rename column
|
||||
Map<String, String> renamedColumns = ImmutableMap.of("name", "last_name");
|
||||
List<SchemaChangeEvent> derivedChangesAfterRenameColumn =
|
||||
schemaDerivation.applySchemaChange(new RenameColumnEvent(TABLE_1, renamedColumns));
|
||||
assertThat(derivedChangesAfterRenameColumn).hasSize(1);
|
||||
assertThat(derivedChangesAfterRenameColumn.get(0))
|
||||
.asRenameColumnEvent()
|
||||
.hasTableId(MERGED_TABLE)
|
||||
.containsNameMapping(renamedColumns);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testMergingTablesWithExactSameSchema() {
|
||||
SchemaDerivation schemaDerivation =
|
||||
new SchemaDerivation(new SchemaManager(), ROUTES, new HashMap<>());
|
||||
|
||||
// Create table 1
|
||||
List<SchemaChangeEvent> derivedChangesAfterCreateTable =
|
||||
schemaDerivation.applySchemaChange(new CreateTableEvent(TABLE_1, SCHEMA));
|
||||
assertThat(derivedChangesAfterCreateTable).hasSize(1);
|
||||
assertThat(derivedChangesAfterCreateTable.get(0))
|
||||
.asCreateTableEvent()
|
||||
.hasTableId(MERGED_TABLE)
|
||||
.hasSchema(SCHEMA);
|
||||
// Create table 2
|
||||
assertThat(schemaDerivation.applySchemaChange(new CreateTableEvent(TABLE_2, SCHEMA)))
|
||||
.isEmpty();
|
||||
|
||||
// Add column for table 1
|
||||
AddColumnEvent.ColumnWithPosition newCol1 =
|
||||
new AddColumnEvent.ColumnWithPosition(
|
||||
new PhysicalColumn("new_col1", DataTypes.STRING(), null));
|
||||
AddColumnEvent.ColumnWithPosition newCol2 =
|
||||
new AddColumnEvent.ColumnWithPosition(
|
||||
new PhysicalColumn("new_col2", DataTypes.STRING(), null));
|
||||
List<AddColumnEvent.ColumnWithPosition> newColumns = Arrays.asList(newCol1, newCol2);
|
||||
List<SchemaChangeEvent> derivedChangesAfterAddColumn =
|
||||
schemaDerivation.applySchemaChange(new AddColumnEvent(TABLE_1, newColumns));
|
||||
assertThat(derivedChangesAfterAddColumn).hasSize(1);
|
||||
assertThat(derivedChangesAfterAddColumn.get(0))
|
||||
.asAddColumnEvent()
|
||||
.hasTableId(MERGED_TABLE)
|
||||
.containsAddedColumns(newCol1, newCol2);
|
||||
// Add column for table 2
|
||||
assertThat(schemaDerivation.applySchemaChange(new AddColumnEvent(TABLE_2, newColumns)))
|
||||
.isEmpty();
|
||||
|
||||
// Alter column type for table 1
|
||||
ImmutableMap<String, DataType> typeMapping = ImmutableMap.of("age", DataTypes.BIGINT());
|
||||
List<SchemaChangeEvent> derivedChangesAfterAlterColumnType =
|
||||
schemaDerivation.applySchemaChange(new AlterColumnTypeEvent(TABLE_1, typeMapping));
|
||||
assertThat(derivedChangesAfterAlterColumnType).hasSize(1);
|
||||
assertThat(derivedChangesAfterAlterColumnType.get(0))
|
||||
.asAlterColumnTypeEvent()
|
||||
.hasTableId(MERGED_TABLE)
|
||||
.containsTypeMapping(typeMapping);
|
||||
// Alter column type for table 2
|
||||
assertThat(
|
||||
schemaDerivation.applySchemaChange(
|
||||
new AlterColumnTypeEvent(TABLE_2, typeMapping)))
|
||||
.isEmpty();
|
||||
|
||||
// Drop column for table 1
|
||||
List<String> droppedColumns = Arrays.asList("new_col1", "new_col2");
|
||||
assertThat(schemaDerivation.applySchemaChange(new DropColumnEvent(TABLE_1, droppedColumns)))
|
||||
.isEmpty();
|
||||
// Drop column for table 2
|
||||
assertThat(schemaDerivation.applySchemaChange(new DropColumnEvent(TABLE_2, droppedColumns)))
|
||||
.isEmpty();
|
||||
|
||||
// Rename column for table 1
|
||||
Map<String, String> renamedColumns = ImmutableMap.of("name", "last_name");
|
||||
List<SchemaChangeEvent> derivedChangesAfterRenameColumn =
|
||||
schemaDerivation.applySchemaChange(new RenameColumnEvent(TABLE_1, renamedColumns));
|
||||
assertThat(derivedChangesAfterRenameColumn).hasSize(1);
|
||||
assertThat(derivedChangesAfterRenameColumn.get(0))
|
||||
.asAddColumnEvent()
|
||||
.hasTableId(MERGED_TABLE)
|
||||
.containsAddedColumns(
|
||||
new AddColumnEvent.ColumnWithPosition(
|
||||
new PhysicalColumn("last_name", DataTypes.STRING(), null)));
|
||||
// Rename column for table 2
|
||||
assertThat(
|
||||
schemaDerivation.applySchemaChange(
|
||||
new RenameColumnEvent(TABLE_2, renamedColumns)))
|
||||
.isEmpty();
|
||||
}
|
||||
|
||||
@Test
|
||||
void testMergingTableWithDifferentSchemas() {
|
||||
SchemaManager schemaManager = new SchemaManager();
|
||||
SchemaDerivation schemaDerivation =
|
||||
new SchemaDerivation(schemaManager, ROUTES, new HashMap<>());
|
||||
// Create table 1
|
||||
List<SchemaChangeEvent> derivedChangesAfterCreateTable =
|
||||
schemaDerivation.applySchemaChange(new CreateTableEvent(TABLE_1, SCHEMA));
|
||||
assertThat(derivedChangesAfterCreateTable).hasSize(1);
|
||||
assertThat(derivedChangesAfterCreateTable.get(0))
|
||||
.asCreateTableEvent()
|
||||
.hasTableId(MERGED_TABLE)
|
||||
.hasSchema(SCHEMA);
|
||||
// Create table 2
|
||||
List<SchemaChangeEvent> derivedChangesAfterCreateTable2 =
|
||||
schemaDerivation.applySchemaChange(
|
||||
new CreateTableEvent(TABLE_2, COMPATIBLE_SCHEMA));
|
||||
assertThat(derivedChangesAfterCreateTable2).hasSize(2);
|
||||
assertThat(derivedChangesAfterCreateTable2)
|
||||
.containsExactlyInAnyOrder(
|
||||
new AddColumnEvent(
|
||||
MERGED_TABLE,
|
||||
Collections.singletonList(
|
||||
new AddColumnEvent.ColumnWithPosition(
|
||||
new PhysicalColumn(
|
||||
"gender", DataTypes.STRING(), null)))),
|
||||
new AlterColumnTypeEvent(
|
||||
MERGED_TABLE, ImmutableMap.of("age", DataTypes.BIGINT())));
|
||||
|
||||
// Add column for table 1
|
||||
AddColumnEvent.ColumnWithPosition newCol1 =
|
||||
new AddColumnEvent.ColumnWithPosition(
|
||||
new PhysicalColumn("new_col1", DataTypes.VARCHAR(255), null));
|
||||
AddColumnEvent.ColumnWithPosition newCol2 =
|
||||
new AddColumnEvent.ColumnWithPosition(
|
||||
new PhysicalColumn("new_col2", DataTypes.VARCHAR(255), null));
|
||||
List<AddColumnEvent.ColumnWithPosition> newColumns = Arrays.asList(newCol1, newCol2);
|
||||
List<SchemaChangeEvent> derivedChangesAfterAddColumn =
|
||||
schemaDerivation.applySchemaChange(new AddColumnEvent(TABLE_1, newColumns));
|
||||
assertThat(derivedChangesAfterAddColumn).hasSize(1);
|
||||
assertThat(derivedChangesAfterAddColumn.get(0))
|
||||
.asAddColumnEvent()
|
||||
.hasTableId(MERGED_TABLE)
|
||||
.containsAddedColumns(newCol1, newCol2);
|
||||
// Add column for table 2
|
||||
List<SchemaChangeEvent> derivedChangesAfterAddColumnForTable2 =
|
||||
schemaDerivation.applySchemaChange(
|
||||
new AddColumnEvent(
|
||||
TABLE_2,
|
||||
Arrays.asList(
|
||||
new AddColumnEvent.ColumnWithPosition(
|
||||
new PhysicalColumn(
|
||||
"new_col1", DataTypes.STRING(), null)),
|
||||
new AddColumnEvent.ColumnWithPosition(
|
||||
new PhysicalColumn(
|
||||
"new_col2", DataTypes.STRING(), null)))));
|
||||
assertThat(derivedChangesAfterAddColumnForTable2).hasSize(1);
|
||||
assertThat(derivedChangesAfterAddColumnForTable2.get(0))
|
||||
.asAlterColumnTypeEvent()
|
||||
.containsTypeMapping(
|
||||
ImmutableMap.of(
|
||||
"new_col1", DataTypes.STRING(), "new_col2", DataTypes.STRING()));
|
||||
|
||||
// Alter column type for table 1
|
||||
ImmutableMap<String, DataType> typeMapping = ImmutableMap.of("age", DataTypes.BIGINT());
|
||||
List<SchemaChangeEvent> derivedChangesAfterAlterColumnType =
|
||||
schemaDerivation.applySchemaChange(new AlterColumnTypeEvent(TABLE_1, typeMapping));
|
||||
assertThat(derivedChangesAfterAlterColumnType).isEmpty();
|
||||
// Alter column type for table 2
|
||||
List<SchemaChangeEvent> derivedChangesAfterAlterColumnTypeForTable2 =
|
||||
schemaDerivation.applySchemaChange(
|
||||
new AlterColumnTypeEvent(
|
||||
TABLE_2, ImmutableMap.of("age", DataTypes.TINYINT())));
|
||||
assertThat(derivedChangesAfterAlterColumnTypeForTable2).isEmpty();
|
||||
|
||||
// Drop column for table 1
|
||||
List<String> droppedColumns = Arrays.asList("new_col1", "new_col2");
|
||||
assertThat(schemaDerivation.applySchemaChange(new DropColumnEvent(TABLE_1, droppedColumns)))
|
||||
.isEmpty();
|
||||
// Drop column for table 2
|
||||
assertThat(schemaDerivation.applySchemaChange(new DropColumnEvent(TABLE_2, droppedColumns)))
|
||||
.isEmpty();
|
||||
|
||||
// Rename column for table 1
|
||||
Map<String, String> renamedColumns = ImmutableMap.of("name", "last_name");
|
||||
List<SchemaChangeEvent> derivedChangesAfterRenameColumn =
|
||||
schemaDerivation.applySchemaChange(new RenameColumnEvent(TABLE_1, renamedColumns));
|
||||
assertThat(derivedChangesAfterRenameColumn).hasSize(1);
|
||||
assertThat(derivedChangesAfterRenameColumn.get(0))
|
||||
.asAddColumnEvent()
|
||||
.hasTableId(MERGED_TABLE)
|
||||
.containsAddedColumns(
|
||||
new AddColumnEvent.ColumnWithPosition(
|
||||
new PhysicalColumn("last_name", DataTypes.STRING(), null)));
|
||||
// Rename column for table 2
|
||||
List<SchemaChangeEvent> derivedChangesAfterRenameColumnForTable2 =
|
||||
schemaDerivation.applySchemaChange(
|
||||
new RenameColumnEvent(TABLE_2, ImmutableMap.of("name", "first_name")));
|
||||
assertThat(derivedChangesAfterRenameColumnForTable2).hasSize(1);
|
||||
assertThat(derivedChangesAfterRenameColumnForTable2.get(0))
|
||||
.asAddColumnEvent()
|
||||
.hasTableId(MERGED_TABLE)
|
||||
.containsAddedColumns(
|
||||
new AddColumnEvent.ColumnWithPosition(
|
||||
new PhysicalColumn("first_name", DataTypes.STRING(), null)));
|
||||
|
||||
assertThat(schemaManager.getLatestSchema(MERGED_TABLE))
|
||||
.contains(
|
||||
Schema.newBuilder()
|
||||
.column(Column.physicalColumn("id", DataTypes.BIGINT()))
|
||||
.column(Column.physicalColumn("name", DataTypes.STRING()))
|
||||
.column(Column.physicalColumn("age", DataTypes.BIGINT()))
|
||||
.column(Column.physicalColumn("gender", DataTypes.STRING()))
|
||||
.column(Column.physicalColumn("new_col1", DataTypes.STRING()))
|
||||
.column(Column.physicalColumn("new_col2", DataTypes.STRING()))
|
||||
.column(Column.physicalColumn("last_name", DataTypes.STRING()))
|
||||
.column(Column.physicalColumn("first_name", DataTypes.STRING()))
|
||||
.build());
|
||||
}
|
||||
|
||||
@Test
|
||||
void testIncompatibleTypes() {
|
||||
SchemaDerivation schemaDerivation =
|
||||
new SchemaDerivation(new SchemaManager(), ROUTES, new HashMap<>());
|
||||
// Create table 1
|
||||
List<SchemaChangeEvent> derivedChangesAfterCreateTable =
|
||||
schemaDerivation.applySchemaChange(new CreateTableEvent(TABLE_1, SCHEMA));
|
||||
assertThat(derivedChangesAfterCreateTable).hasSize(1);
|
||||
assertThat(derivedChangesAfterCreateTable.get(0))
|
||||
.asCreateTableEvent()
|
||||
.hasTableId(MERGED_TABLE)
|
||||
.hasSchema(SCHEMA);
|
||||
|
||||
// Create table 2
|
||||
assertThatThrownBy(
|
||||
() ->
|
||||
schemaDerivation.applySchemaChange(
|
||||
new CreateTableEvent(TABLE_2, INCOMPATIBLE_SCHEMA)))
|
||||
.isInstanceOf(IllegalStateException.class)
|
||||
.hasMessage("Incompatible types: \"INT\" and \"STRING\"");
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue