Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -956,6 +956,69 @@ void testTransformWithTemporalFunction() throws Exception {
Arrays.stream(outputEvents).forEach(this::extractDataLines);
}

@ParameterizedTest
@EnumSource
public void testTransformWithColumnNameMap(ValuesDataSink.SinkApi sinkApi) throws Exception {
FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster();

// Setup value source
Configuration sourceConfig = new Configuration();
sourceConfig.set(
ValuesDataSourceOptions.EVENT_SET_ID,
ValuesDataSourceHelper.EventSetId.COMPLEX_COLUMN_NAME_TABLE);
SourceDef sourceDef =
new SourceDef(ValuesDataFactory.IDENTIFIER, "Value Source", sourceConfig);

// Setup value sink
Configuration sinkConfig = new Configuration();
sinkConfig.set(ValuesDataSinkOptions.MATERIALIZED_IN_MEMORY, true);
sinkConfig.set(ValuesDataSinkOptions.SINK_API, sinkApi);
SinkDef sinkDef = new SinkDef(ValuesDataFactory.IDENTIFIER, "Value Sink", sinkConfig);

// Setup transform
TransformDef transformDef =
new TransformDef(
"default_namespace.default_schema.table1",
"*, `timestamp-type`",
"`foo-bar` > 0",
null,
null,
null,
null,
null);

// Setup pipeline
Configuration pipelineConfig = new Configuration();
pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1);
pipelineConfig.set(
PipelineOptions.PIPELINE_SCHEMA_CHANGE_BEHAVIOR, SchemaChangeBehavior.EVOLVE);
PipelineDef pipelineDef =
new PipelineDef(
sourceDef,
sinkDef,
Collections.emptyList(),
new ArrayList<>(Arrays.asList(transformDef)),
Collections.emptyList(),
pipelineConfig);

// Execute the pipeline
PipelineExecution execution = composer.compose(pipelineDef);
execution.execute();

// 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={`class` STRING NOT NULL,`foo-bar` INT,`bar-foo` INT,`timestamp-type` STRING NOT NULL}, primaryKeys=class, options=()}",
"DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[class1, 1, 10, type1], op=INSERT, meta=({timestamp-type=type1})}",
"DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[class2, 2, 100, type2], op=INSERT, meta=({timestamp-type=type2})}",
"AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`import-package` STRING, position=AFTER, existedColumnName=bar-foo}]}",
"RenameColumnEvent{tableId=default_namespace.default_schema.table1, nameMapping={bar-foo=bar-baz}}",
"DropColumnEvent{tableId=default_namespace.default_schema.table1, droppedColumnNames=[bar-baz]}",
"DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[class1, 1, , type1], after=[], op=DELETE, meta=({timestamp-type=type1})}",
"DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[class2, 2, , type2], after=[new-class2, 20, new-package2, type2], op=UPDATE, meta=({timestamp-type=type2})}");
}

void runGenericTransformTest(
ValuesDataSink.SinkApi sinkApi,
List<TransformDef> transformDefs,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* 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.connectors.values.source;

import org.apache.flink.cdc.common.source.SupportedMetadataColumn;
import org.apache.flink.cdc.common.types.DataType;
import org.apache.flink.cdc.common.types.DataTypes;

import java.util.Map;

/** A {@link SupportedMetadataColumn} for timestamp-type. */
public class TimestampTypeMetadataColumn implements SupportedMetadataColumn {

@Override
public String getName() {
return "timestamp-type";
}

@Override
public DataType getType() {
return DataTypes.STRING();
}

@Override
public Class<?> getJavaClass() {
return String.class;
}

@Override
public Object read(Map<String, String> metadata) {
return metadata.getOrDefault(getName(), null);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,9 @@ public MetadataAccessor getMetadataAccessor() {

@Override
public SupportedMetadataColumn[] supportedMetadataColumns() {
return new SupportedMetadataColumn[] {new OpTsMetadataColumn()};
return new SupportedMetadataColumn[] {
new OpTsMetadataColumn(), new TimestampTypeMetadataColumn()
};
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.apache.flink.cdc.common.schema.Schema;
import org.apache.flink.cdc.common.types.DataTypes;
import org.apache.flink.cdc.common.types.RowType;
import org.apache.flink.cdc.common.utils.SchemaUtils;
import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;

import java.util.ArrayList;
Expand All @@ -54,7 +55,8 @@ public enum EventSetId {
SINGLE_SPLIT_MULTI_TABLES,
MULTI_SPLITS_SINGLE_TABLE,
CUSTOM_SOURCE_EVENTS,
TRANSFORM_TABLE
TRANSFORM_TABLE,
COMPLEX_COLUMN_NAME_TABLE
}

public static final TableId TABLE_1 =
Expand Down Expand Up @@ -120,6 +122,11 @@ public static void setSourceEvents(EventSetId eventType) {
sourceEvents = transformTable();
break;
}
case COMPLEX_COLUMN_NAME_TABLE:
{
sourceEvents = complexColumnNameTable();
break;
}
default:
throw new IllegalArgumentException(eventType + " is not supported");
}
Expand Down Expand Up @@ -644,4 +651,126 @@ public static List<List<Event>> transformTable() {
eventOfSplits.add(split1);
return eventOfSplits;
}

public static List<List<Event>> complexColumnNameTable() {
List<List<Event>> eventOfSplits = new ArrayList<>();
List<Event> split1 = new ArrayList<>();

// create table
Schema schema =
Schema.newBuilder()
.physicalColumn("class", DataTypes.STRING())
.physicalColumn("foo-bar", DataTypes.INT())
.physicalColumn("bar-foo", DataTypes.INT())
.primaryKey("class")
.build();
CreateTableEvent createTableEvent = new CreateTableEvent(TABLE_1, schema);
split1.add(createTableEvent);

BinaryRecordDataGenerator generator =
new BinaryRecordDataGenerator((RowType) schema.toRowDataType());
// insert
DataChangeEvent insertEvent1 =
DataChangeEvent.insertEvent(
TABLE_1,
generator.generate(
new Object[] {
BinaryStringData.fromString("class0"), 0, 0,
}),
new HashMap<String, String>() {
{
put("timestamp-type", "type0");
}
});
split1.add(insertEvent1);
DataChangeEvent insertEvent2 =
DataChangeEvent.insertEvent(
TABLE_1,
generator.generate(
new Object[] {
BinaryStringData.fromString("class1"), 1, 10,
}),
new HashMap<String, String>() {
{
put("timestamp-type", "type1");
}
});
split1.add(insertEvent2);
DataChangeEvent insertEvent3 =
DataChangeEvent.insertEvent(
TABLE_1,
generator.generate(
new Object[] {BinaryStringData.fromString("class2"), 2, 100}),
new HashMap<String, String>() {
{
put("timestamp-type", "type2");
}
});
split1.add(insertEvent3);

// add column
AddColumnEvent.ColumnWithPosition columnWithPosition =
new AddColumnEvent.ColumnWithPosition(
Column.physicalColumn("import-package", DataTypes.STRING()));
AddColumnEvent addColumnEvent =
new AddColumnEvent(TABLE_1, Collections.singletonList(columnWithPosition));
split1.add(addColumnEvent);
schema = SchemaUtils.applySchemaChangeEvent(schema, addColumnEvent);

// rename column
Map<String, String> nameMapping = new HashMap<>();
nameMapping.put("bar-foo", "bar-baz");
RenameColumnEvent renameColumnEvent = new RenameColumnEvent(TABLE_1, nameMapping);
split1.add(renameColumnEvent);
schema = SchemaUtils.applySchemaChangeEvent(schema, renameColumnEvent);

// drop column
DropColumnEvent dropColumnEvent =
new DropColumnEvent(TABLE_1, Collections.singletonList("bar-baz"));
split1.add(dropColumnEvent);
schema = SchemaUtils.applySchemaChangeEvent(schema, dropColumnEvent);

generator = new BinaryRecordDataGenerator((RowType) schema.toRowDataType());

// delete
split1.add(
DataChangeEvent.deleteEvent(
TABLE_1,
generator.generate(
new Object[] {
BinaryStringData.fromString("class1"),
1,
BinaryStringData.fromString(""),
}),
new HashMap<String, String>() {
{
put("timestamp-type", "type1");
}
}));

// update
split1.add(
DataChangeEvent.updateEvent(
TABLE_1,
generator.generate(
new Object[] {
BinaryStringData.fromString("class2"),
2,
BinaryStringData.fromString("")
}),
generator.generate(
new Object[] {
BinaryStringData.fromString("new-class2"),
20,
BinaryStringData.fromString("new-package2"),
}),
new HashMap<String, String>() {
{
put("timestamp-type", "type2");
}
}));

eventOfSplits.add(split1);
return eventOfSplits;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -409,7 +409,7 @@ private Optional<DataChangeEvent> processDataChangeEvent(DataChangeEvent dataCha
Optional<TransformFilter> transformFilterOptional = transform.getFilter();

if (transformFilterOptional.isPresent()
&& transformFilterOptional.get().isVaild()) {
&& transformFilterOptional.get().isValid()) {
TransformFilter transformFilter = transformFilterOptional.get();
if (!transformFilterProcessorMap.containsKey(
Tuple2.of(tableId, transformFilter))) {
Expand Down
Loading
Loading