Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-36854] Fix comments and default expressions missing after being transformed #3780

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
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 @@ -411,7 +411,7 @@ void testOpTypeMetadataColumn(ValuesDataSink.SinkApi sinkApi) throws Exception {
String[] outputEvents = outCaptor.toString().trim().split("\n");
assertThat(outputEvents)
.containsExactly(
"CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING,`col12` STRING,`rk` STRING}, primaryKeys=col1, partitionKeys=col12, options=({key1=value1})}",
"CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING,`col12` STRING,`rk` STRING NOT NULL}, primaryKeys=col1, partitionKeys=col12, options=({key1=value1})}",
"DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1, 10, +I], op=INSERT, meta=()}",
"DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2, 20, +I], op=INSERT, meta=()}",
"AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=AFTER, existedColumnName=col2}]}",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -453,7 +453,7 @@ void testOpTypeMetadataColumn(ValuesDataSink.SinkApi sinkApi) throws Exception {
String[] outputEvents = outCaptor.toString().trim().split(LINE_SEPARATOR);
assertThat(outputEvents)
.containsExactly(
"CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING,`col12` STRING,`rk` STRING}, primaryKeys=col1, partitionKeys=col12, options=({key1=value1})}",
"CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING,`col2` STRING,`col12` STRING,`rk` STRING NOT NULL}, primaryKeys=col1, partitionKeys=col12, options=({key1=value1})}",
"DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1, 10, +I], op=INSERT, meta=()}",
"DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2, 20, +I], op=INSERT, meta=()}",
"AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=LAST, existedColumnName=null}]}",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@

import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap;

import org.assertj.core.api.Assertions;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Disabled;
Expand Down Expand Up @@ -1515,6 +1516,93 @@ void testTransformUnmatchedSchemaEvolution() throws Exception {
"DataChangeEvent{tableId=default_namespace.default_schema.mytable1, before=[12th, 15, Oops], after=[], op=DELETE, meta=()}");
}

@Test
void testTransformWithCommentsAndDefaultExpr() throws Exception {
FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster();

// Setup value source
Configuration sourceConfig = new Configuration();
sourceConfig.set(
ValuesDataSourceOptions.EVENT_SET_ID,
ValuesDataSourceHelper.EventSetId.CUSTOM_SOURCE_EVENTS);

TableId myTable1 = TableId.tableId("default_namespace", "default_schema", "mytable1");
TableId myTable2 = TableId.tableId("default_namespace", "default_schema", "mytable2");
Schema table1Schema =
Schema.newBuilder()
.physicalColumn("id", DataTypes.INT(), "id column", "AUTO_INCREMENT()")
.physicalColumn("name", DataTypes.STRING(), "name column", "Jane Doe")
.physicalColumn("age", DataTypes.INT(), "age column", "17")
.partitionKey("id, age")
.primaryKey("id")
.build();
Schema table2Schema =
Schema.newBuilder()
.physicalColumn(
"id", DataTypes.BIGINT(), "column for id", "AUTO_DECREMENT()")
.physicalColumn(
"name", DataTypes.VARCHAR(255), "column for name", "John Smith")
.physicalColumn("age", DataTypes.TINYINT(), "column for age", "91")
.physicalColumn(
"description",
DataTypes.STRING(),
"column for descriptions",
"not important")
.primaryKey("id")
.partitionKey("id, name")
.build();

List<Event> events = getTestEvents(table1Schema, table2Schema, myTable1, myTable2);

ValuesDataSourceHelper.setSourceEvents(Collections.singletonList(events));

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 pipeline
Configuration pipelineConfig = new Configuration();
pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1);
PipelineDef pipelineDef =
new PipelineDef(
sourceDef,
sinkDef,
Collections.emptyList(),
Collections.singletonList(
new TransformDef(
"default_namespace.default_schema.\\.*",
"*, name AS new_name, age + 1 AS new_age, 'extras' AS extras",
null,
null,
null,
null,
null)),
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");

Assertions.assertThat(outputEvents)
.containsExactly(
"CreateTableEvent{tableId=default_namespace.default_schema.mytable1, schema=columns={`id` INT 'id column' 'AUTO_INCREMENT()',`name` STRING 'name column' 'Jane Doe',`age` INT 'age column' '17',`new_name` STRING 'name column' 'Jane Doe',`new_age` INT,`extras` STRING}, primaryKeys=id, partitionKeys=id, age, options=()}",
"DataChangeEvent{tableId=default_namespace.default_schema.mytable1, before=[], after=[1, Alice, 18, Alice, 19, extras], op=INSERT, meta=()}",
"DataChangeEvent{tableId=default_namespace.default_schema.mytable1, before=[], after=[2, Bob, 20, Bob, 21, extras], op=INSERT, meta=()}",
"DataChangeEvent{tableId=default_namespace.default_schema.mytable1, before=[2, Bob, 20, Bob, 21, extras], after=[2, Bob, 30, Bob, 31, extras], op=UPDATE, meta=()}",
"CreateTableEvent{tableId=default_namespace.default_schema.mytable2, schema=columns={`id` BIGINT 'column for id' 'AUTO_DECREMENT()',`name` VARCHAR(255) 'column for name' 'John Smith',`age` TINYINT 'column for age' '91',`description` STRING 'column for descriptions' 'not important',`new_name` VARCHAR(255) 'column for name' 'John Smith',`new_age` INT,`extras` STRING}, primaryKeys=id, partitionKeys=id, name, options=()}",
"DataChangeEvent{tableId=default_namespace.default_schema.mytable2, before=[], after=[3, Carol, 15, student, Carol, 16, extras], op=INSERT, meta=()}",
"DataChangeEvent{tableId=default_namespace.default_schema.mytable2, before=[], after=[4, Derrida, 25, student, Derrida, 26, extras], op=INSERT, meta=()}",
"DataChangeEvent{tableId=default_namespace.default_schema.mytable2, before=[4, Derrida, 25, student, Derrida, 26, extras], after=[], op=DELETE, meta=()}");
}

private List<Event> generateSchemaEvolutionEvents(TableId tableId) {
List<Event> events = new ArrayList<>();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -467,13 +467,13 @@ public void testWildcardWithMetadataColumnTransform() throws Exception {

waitUntilSpecificEvent(
String.format(
"CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`PRICEALPHA` INT,`AGEALPHA` INT,`NAMEALPHA` VARCHAR(128),`identifier_name` STRING,`type` STRING}, primaryKeys=ID, options=()}",
"CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`PRICEALPHA` INT,`AGEALPHA` INT,`NAMEALPHA` VARCHAR(128),`identifier_name` STRING,`type` STRING NOT NULL}, primaryKeys=ID, options=()}",
transformTestDatabase.getDatabaseName()),
60000L);

waitUntilSpecificEvent(
String.format(
"CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`CODENAMESBETA` VARCHAR(17),`AGEBETA` INT,`NAMEBETA` VARCHAR(128),`identifier_name` STRING,`type` STRING}, primaryKeys=ID, options=()}",
"CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`CODENAMESBETA` VARCHAR(17),`AGEBETA` INT,`NAMEBETA` VARCHAR(128),`identifier_name` STRING,`type` STRING NOT NULL}, primaryKeys=ID, options=()}",
transformTestDatabase.getDatabaseName()),
60000L);

Expand Down Expand Up @@ -551,13 +551,13 @@ public void testMultipleHittingTable() throws Exception {

waitUntilSpecificEvent(
String.format(
"CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`PRICEALPHA` INT,`AGEALPHA` INT,`NAMEALPHA` VARCHAR(128),`UID` INT,`NEWVERSION` STRING}, primaryKeys=ID, options=()}",
"CreateTableEvent{tableId=%s.TABLEALPHA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`PRICEALPHA` INT,`AGEALPHA` INT,`NAMEALPHA` VARCHAR(128),`UID` INT,`NEWVERSION` VARCHAR(17)}, primaryKeys=ID, options=()}",
transformTestDatabase.getDatabaseName()),
60000L);

waitUntilSpecificEvent(
String.format(
"CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`CODENAMESBETA` VARCHAR(17),`AGEBETA` INT,`NAMEBETA` VARCHAR(128),`UID` INT,`NEWVERSION` STRING}, primaryKeys=ID, options=()}",
"CreateTableEvent{tableId=%s.TABLEBETA, schema=columns={`ID` INT NOT NULL,`VERSION` VARCHAR(17),`CODENAMESBETA` VARCHAR(17),`AGEBETA` INT,`NAMEBETA` VARCHAR(128),`UID` INT,`NEWVERSION` VARCHAR(17)}, primaryKeys=ID, options=()}",
transformTestDatabase.getDatabaseName()),
60000L);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -329,7 +329,7 @@ private PostTransformChangeInfo getPostTransformChangeInfo(TableId tableId) {
return tableInfo;
}

private Schema transformSchema(TableId tableId, Schema schema) throws Exception {
private Schema transformSchema(TableId tableId, Schema schema) {
List<Schema> newSchemas = new ArrayList<>();
for (PostTransformer transform : transforms) {
Selectors selectors = transform.getSelectors();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@

import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;

/**
Expand Down Expand Up @@ -96,11 +97,36 @@ public boolean isValidTransformedProjectionColumn() {
return !StringUtils.isNullOrWhitespaceOnly(scriptExpression);
}

public static ProjectionColumn of(String columnName, DataType dataType) {
return new ProjectionColumn(Column.physicalColumn(columnName, dataType), null, null, null);
/**
* This projection is created with a plain column name. <br>
* Just like column {@code id} in {@code id, name AS new_name, age + 1 AS new_age}. <br>
* Comments and default expressions will be intact.
*/
public static ProjectionColumn ofForwarded(Column column) {
String name = column.getName();
return new ProjectionColumn(column, name, name, Collections.singletonList(name));
}

public static ProjectionColumn of(
/**
* This projection is created with a simple $id$ AS $new_id$ expression. <br>
* Just like column {@code new_name} in {@code id, name AS new_name, age + 1 AS new_age}. <br>
* Comments and default expressions will be intact.
*/
public static ProjectionColumn ofAliased(Column column, String newName) {
String originalName = column.getName();
return new ProjectionColumn(
column.copy(newName),
originalName,
originalName,
Collections.singletonList(originalName));
}

/**
* This projection is created with a complex calculation expression. <br>
* Just like column {@code new_age} in {@code id, name AS new_name, age + 1 AS new_age}. <br>
* No comments nor default expressions will be kept.
*/
public static ProjectionColumn ofCalculated(
String columnName,
DataType dataType,
String expression,
Expand Down
Loading
Loading