[cdc-runtime] Introduce TransformSchemaOperator and TransformDataOperator to support transformation

pull/3200/head
wenmo 11 months ago committed by Leonard Xu
parent c92903016d
commit bcad5d9d11

@ -19,6 +19,9 @@ package org.apache.flink.cdc.composer.flink.translator;
import org.apache.flink.cdc.common.event.Event;
import org.apache.flink.cdc.composer.definition.TransformDef;
import org.apache.flink.cdc.runtime.operators.transform.TransformDataOperator;
import org.apache.flink.cdc.runtime.operators.transform.TransformSchemaOperator;
import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo;
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.streaming.api.datastream.DataStream;
@ -32,7 +35,21 @@ public class TransformTranslator {
if (transforms.isEmpty()) {
return input;
}
return input;
TransformSchemaOperator.Builder transformSchemaFunctionBuilder =
TransformSchemaOperator.newBuilder();
for (TransformDef transform : transforms) {
if (transform.isValidProjection()) {
transformSchemaFunctionBuilder.addTransform(
transform.getSourceTable(),
transform.getProjection().get(),
transform.getPrimaryKeys(),
transform.getPartitionKeys(),
transform.getTableOptions());
}
}
return input.transform(
"Transform:Schema", new EventTypeInfo(), transformSchemaFunctionBuilder.build());
}
public DataStream<Event> translateData(
@ -43,6 +60,20 @@ public class TransformTranslator {
if (transforms.isEmpty()) {
return input;
}
return input;
TransformDataOperator.Builder transformDataFunctionBuilder =
TransformDataOperator.newBuilder();
for (TransformDef transform : transforms) {
if (transform.isValidProjection() || transform.isValidFilter()) {
transformDataFunctionBuilder.addTransform(
transform.getSourceTable(),
transform.isValidProjection() ? transform.getProjection().get() : null,
transform.isValidFilter() ? transform.getFilter().get() : null);
}
}
transformDataFunctionBuilder.addSchemaOperatorID(schemaOperatorID);
transformDataFunctionBuilder.addTimezone(timezone);
return input.transform(
"Transform:Data", new EventTypeInfo(), transformDataFunctionBuilder.build());
}
}

@ -23,6 +23,7 @@ import org.apache.flink.cdc.composer.PipelineExecution;
import org.apache.flink.cdc.composer.definition.PipelineDef;
import org.apache.flink.cdc.composer.definition.SinkDef;
import org.apache.flink.cdc.composer.definition.SourceDef;
import org.apache.flink.cdc.composer.definition.TransformDef;
import org.apache.flink.cdc.connectors.values.ValuesDatabase;
import org.apache.flink.cdc.connectors.values.factory.ValuesDataFactory;
import org.apache.flink.cdc.connectors.values.sink.ValuesDataSinkOptions;
@ -38,6 +39,8 @@ import org.junit.jupiter.api.extension.RegisterExtension;
import java.io.ByteArrayOutputStream;
import java.io.PrintStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
@ -249,4 +252,126 @@ class FlinkPipelineComposerITCase {
"default_namespace.default_schema.table1:col1=3;col2=3;col3=x",
"default_namespace.default_schema.table1:col1=5;col2=5;col3=");
}
@Test
void testTransform() throws Exception {
FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster();
// Setup value source
Configuration sourceConfig = new Configuration();
sourceConfig.set(
ValuesDataSourceOptions.EVENT_SET_ID,
ValuesDataSourceHelper.EventSetId.TRANSFORM_TABLE);
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 transform
TransformDef transformDef =
new TransformDef(
"default_namespace.default_schema.table1",
"*,concat(col1,'0') as col12",
"col1 <> '3'",
"col1",
"col12",
"key1=value1",
"");
// Setup pipeline
Configuration pipelineConfig = new Configuration();
pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1);
PipelineDef pipelineDef =
new PipelineDef(
sourceDef,
sinkDef,
Collections.emptyList(),
new ArrayList<>(Arrays.asList(transformDef)),
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={`col1` STRING,`col2` STRING,`col12` STRING}, primaryKeys=col1, partitionKeys=col12, options=({key1=value1})}",
"DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1, 10], op=INSERT, meta=()}",
"DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2, 20], op=INSERT, meta=()}",
"AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=LAST, existedColumnName=null}]}",
"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, 10], after=[], op=DELETE, meta=()}",
"DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[2, , 20], after=[2, x, 20], op=UPDATE, meta=()}");
}
@Test
void testTransformTwice() throws Exception {
FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster();
// Setup value source
Configuration sourceConfig = new Configuration();
sourceConfig.set(
ValuesDataSourceOptions.EVENT_SET_ID,
ValuesDataSourceHelper.EventSetId.TRANSFORM_TABLE);
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 transform
TransformDef transformDef1 =
new TransformDef(
"default_namespace.default_schema.table1",
"*,concat(col1,'1') as col12",
"col1 = '1'",
"col1",
"col12",
"key1=value1",
"");
TransformDef transformDef2 =
new TransformDef(
"default_namespace.default_schema.table1",
"*,concat(col1,'2') as col12",
"col1 = '2'",
null,
null,
null,
"");
// Setup pipeline
Configuration pipelineConfig = new Configuration();
pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1);
PipelineDef pipelineDef =
new PipelineDef(
sourceDef,
sinkDef,
Collections.emptyList(),
new ArrayList<>(Arrays.asList(transformDef1, transformDef2)),
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={`col1` STRING,`col2` STRING,`col12` STRING}, primaryKeys=col1, partitionKeys=col12, options=({key1=value1})}",
"DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1, 11], op=INSERT, meta=()}",
"DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2, 22], op=INSERT, meta=()}",
"AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=LAST, existedColumnName=null}]}",
"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, 11], after=[], op=DELETE, meta=()}",
"DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[2, , 22], after=[2, x, 22], op=UPDATE, meta=()}");
}
}

@ -52,7 +52,8 @@ public class ValuesDataSourceHelper {
SINGLE_SPLIT_SINGLE_TABLE,
SINGLE_SPLIT_MULTI_TABLES,
MULTI_SPLITS_SINGLE_TABLE,
CUSTOM_SOURCE_EVENTS
CUSTOM_SOURCE_EVENTS,
TRANSFORM_TABLE
}
public static final TableId TABLE_1 =
@ -102,6 +103,11 @@ public class ValuesDataSourceHelper {
{
break;
}
case TRANSFORM_TABLE:
{
sourceEvents = transformTable();
break;
}
default:
throw new IllegalArgumentException(eventType + " is not supported");
}
@ -483,4 +489,98 @@ public class ValuesDataSourceHelper {
return eventOfSplits;
}
public static List<List<Event>> transformTable() {
List<List<Event>> eventOfSplits = new ArrayList<>();
List<Event> split1 = new ArrayList<>();
// create table
Schema schema =
Schema.newBuilder()
.physicalColumn("col1", DataTypes.STRING())
.physicalColumn("col2", DataTypes.STRING())
.primaryKey("col1")
.build();
CreateTableEvent createTableEvent = new CreateTableEvent(TABLE_1, schema);
split1.add(createTableEvent);
BinaryRecordDataGenerator generator =
new BinaryRecordDataGenerator(RowType.of(DataTypes.STRING(), DataTypes.STRING()));
// insert
DataChangeEvent insertEvent1 =
DataChangeEvent.insertEvent(
TABLE_1,
generator.generate(
new Object[] {
BinaryStringData.fromString("1"),
BinaryStringData.fromString("1")
}));
split1.add(insertEvent1);
DataChangeEvent insertEvent2 =
DataChangeEvent.insertEvent(
TABLE_1,
generator.generate(
new Object[] {
BinaryStringData.fromString("2"),
BinaryStringData.fromString("2")
}));
split1.add(insertEvent2);
DataChangeEvent insertEvent3 =
DataChangeEvent.insertEvent(
TABLE_1,
generator.generate(
new Object[] {
BinaryStringData.fromString("3"),
BinaryStringData.fromString("3")
}));
split1.add(insertEvent3);
// add column
AddColumnEvent.ColumnWithPosition columnWithPosition =
new AddColumnEvent.ColumnWithPosition(
Column.physicalColumn("col3", DataTypes.STRING()));
AddColumnEvent addColumnEvent =
new AddColumnEvent(TABLE_1, Collections.singletonList(columnWithPosition));
split1.add(addColumnEvent);
// rename column
Map<String, String> nameMapping = new HashMap<>();
nameMapping.put("col2", "newCol2");
nameMapping.put("col3", "newCol3");
RenameColumnEvent renameColumnEvent = new RenameColumnEvent(TABLE_1, nameMapping);
split1.add(renameColumnEvent);
// drop column
DropColumnEvent dropColumnEvent =
new DropColumnEvent(TABLE_1, Collections.singletonList("newCol2"));
split1.add(dropColumnEvent);
// delete
split1.add(
DataChangeEvent.deleteEvent(
TABLE_1,
generator.generate(
new Object[] {
BinaryStringData.fromString("1"),
BinaryStringData.fromString("1")
})));
// update
split1.add(
DataChangeEvent.updateEvent(
TABLE_1,
generator.generate(
new Object[] {
BinaryStringData.fromString("2"),
BinaryStringData.fromString("")
}),
generator.generate(
new Object[] {
BinaryStringData.fromString("2"),
BinaryStringData.fromString("x")
})));
eventOfSplits.add(split1);
return eventOfSplits;
}
}

@ -0,0 +1,238 @@
/*
* 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.functions;
import org.apache.flink.annotation.Internal;
import org.apache.flink.table.functions.BuiltInFunctionDefinition;
import org.apache.calcite.sql.SqlFunction;
import org.apache.calcite.sql.SqlFunctionCategory;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.SqlOperatorBinding;
import org.apache.calcite.sql.type.SqlOperandTypeChecker;
import org.apache.calcite.sql.type.SqlOperandTypeInference;
import org.apache.calcite.sql.type.SqlReturnTypeInference;
import org.apache.calcite.sql.validate.SqlMonotonicity;
import javax.annotation.Nullable;
import java.util.Optional;
import java.util.function.Function;
import static org.apache.flink.table.functions.BuiltInFunctionDefinition.DEFAULT_VERSION;
import static org.apache.flink.table.functions.BuiltInFunctionDefinition.qualifyFunctionName;
import static org.apache.flink.table.functions.BuiltInFunctionDefinition.validateFunction;
import static org.apache.flink.util.Preconditions.checkNotNull;
/**
* This is the case when the operator has a special parsing syntax or uses other Calcite-specific
* features that are not exposed via {@link BuiltInFunctionDefinition} yet.
*
* <p>Note: Try to keep usages of this class to a minimum and use Flink's {@link
* BuiltInFunctionDefinition} stack instead.
*
* <p>For simple functions, use the provided builder. Otherwise, this class can also be extended.
*/
@Internal
public class BuiltInScalarFunction extends SqlFunction {
private final @Nullable Integer version;
private final boolean isDeterministic;
private final boolean isInternal;
private final Function<SqlOperatorBinding, SqlMonotonicity> monotonicity;
protected BuiltInScalarFunction(
String name,
int version,
SqlKind kind,
@Nullable SqlReturnTypeInference returnTypeInference,
@Nullable SqlOperandTypeInference operandTypeInference,
@Nullable SqlOperandTypeChecker operandTypeChecker,
SqlFunctionCategory category,
boolean isDeterministic,
boolean isInternal,
Function<SqlOperatorBinding, SqlMonotonicity> monotonicity) {
super(
checkNotNull(name),
checkNotNull(kind),
returnTypeInference,
operandTypeInference,
operandTypeChecker,
checkNotNull(category));
this.version = isInternal ? null : version;
this.isDeterministic = isDeterministic;
this.isInternal = isInternal;
this.monotonicity = monotonicity;
validateFunction(name, version, isInternal);
}
protected BuiltInScalarFunction(
String name,
SqlKind kind,
SqlReturnTypeInference returnTypeInference,
SqlOperandTypeInference operandTypeInference,
@Nullable SqlOperandTypeChecker operandTypeChecker,
SqlFunctionCategory category) {
this(
name,
DEFAULT_VERSION,
kind,
returnTypeInference,
operandTypeInference,
operandTypeChecker,
category,
true,
false,
call -> SqlMonotonicity.NOT_MONOTONIC);
}
/** Builder for configuring and creating instances of {@link BuiltInScalarFunction}. */
public static Builder newBuilder() {
return new Builder();
}
public final Optional<Integer> getVersion() {
return Optional.ofNullable(version);
}
public String getQualifiedName() {
if (isInternal) {
return getName();
}
assert version != null;
return qualifyFunctionName(getName(), version);
}
@Override
public boolean isDeterministic() {
return isDeterministic;
}
public final boolean isInternal() {
return isInternal;
}
@Override
public SqlMonotonicity getMonotonicity(SqlOperatorBinding call) {
return monotonicity.apply(call);
}
// --------------------------------------------------------------------------------------------
// Builder
// --------------------------------------------------------------------------------------------
/** Builder for fluent definition of built-in functions. */
public static class Builder {
private String name;
private int version = DEFAULT_VERSION;
private SqlKind kind = SqlKind.OTHER_FUNCTION;
private SqlReturnTypeInference returnTypeInference;
private SqlOperandTypeInference operandTypeInference;
private SqlOperandTypeChecker operandTypeChecker;
private SqlFunctionCategory category = SqlFunctionCategory.SYSTEM;
private boolean isInternal = false;
private boolean isDeterministic = true;
private Function<SqlOperatorBinding, SqlMonotonicity> monotonicity =
call -> SqlMonotonicity.NOT_MONOTONIC;
/** @see BuiltInFunctionDefinition.Builder#name(String) */
public Builder name(String name) {
this.name = name;
return this;
}
/** @see BuiltInFunctionDefinition.Builder#version(int) */
public Builder version(int version) {
this.version = version;
return this;
}
public Builder kind(SqlKind kind) {
this.kind = kind;
return this;
}
public Builder returnType(SqlReturnTypeInference returnTypeInference) {
this.returnTypeInference = returnTypeInference;
return this;
}
public Builder operandTypeInference(SqlOperandTypeInference operandTypeInference) {
this.operandTypeInference = operandTypeInference;
return this;
}
public Builder operandTypeChecker(SqlOperandTypeChecker operandTypeChecker) {
this.operandTypeChecker = operandTypeChecker;
return this;
}
public Builder category(SqlFunctionCategory category) {
this.category = category;
return this;
}
public Builder notDeterministic() {
this.isDeterministic = false;
return this;
}
/** @see BuiltInFunctionDefinition.Builder#internal() */
public Builder internal() {
this.isInternal = true;
return this;
}
public Builder monotonicity(SqlMonotonicity staticMonotonicity) {
this.monotonicity = call -> staticMonotonicity;
return this;
}
public Builder monotonicity(Function<SqlOperatorBinding, SqlMonotonicity> monotonicity) {
this.monotonicity = monotonicity;
return this;
}
public BuiltInScalarFunction build() {
return new BuiltInScalarFunction(
name,
version,
kind,
returnTypeInference,
operandTypeInference,
operandTypeChecker,
category,
isDeterministic,
isInternal,
monotonicity);
}
}
}

@ -0,0 +1,55 @@
/*
* 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.functions;
import org.apache.flink.annotation.Internal;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.sql.SqlOperatorBinding;
import org.apache.calcite.sql.fun.SqlAbstractTimeFunction;
import org.apache.calcite.sql.type.SqlTypeName;
/**
* Function that used to define SQL time function like LOCALTIMESTAMP, CURRENT_TIMESTAMP,
* CURRENT_ROW_TIMESTAMP(), NOW() in Flink, the function support configuring the return type and the
* precision of return type.
*/
@Internal
public class BuiltInTimestampFunction extends SqlAbstractTimeFunction {
private final SqlTypeName returnTypeName;
private final int precision;
public BuiltInTimestampFunction(
String functionName, SqlTypeName returnTypeName, int precision) {
// access protected constructor
super(functionName, returnTypeName);
this.returnTypeName = returnTypeName;
this.precision = precision;
}
@Override
public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
return opBinding.getTypeFactory().createSqlType(returnTypeName, precision);
}
@Override
public boolean isDeterministic() {
return false;
}
}

@ -0,0 +1,473 @@
/*
* 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.functions;
import org.apache.flink.cdc.common.utils.DateTimeUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.math.BigDecimal;
import java.math.RoundingMode;
import java.text.ParseException;
import java.text.SimpleDateFormat;
import java.util.Arrays;
import java.util.Calendar;
import java.util.Date;
import java.util.TimeZone;
import java.util.UUID;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
/** System function utils to support the call of flink cdc pipeline transform. */
public class SystemFunctionUtils {
private static final Logger LOG = LoggerFactory.getLogger(SystemFunctionUtils.class);
public static int localtime(long epochTime, String timezone) {
return DateTimeUtils.timestampMillisToTime(epochTime);
}
public static long localtimestamp(long epochTime, String timezone) {
return epochTime;
}
// synonym: localtime
public static int currentTime(long epochTime, String timezone) {
return localtime(epochTime, timezone);
}
public static int currentDate(long epochTime, String timezone) {
return DateTimeUtils.timestampMillisToDate(epochTime);
}
public static long currentTimestamp(long epochTime, String timezone) {
return epochTime + TimeZone.getTimeZone(timezone).getOffset(epochTime);
}
// synonym: currentTimestamp
public static long now(long epochTime, String timezone) {
return currentTimestamp(epochTime, timezone);
}
public static String dateFormat(long timestamp, String format) {
SimpleDateFormat dateFormat = new SimpleDateFormat(format);
return dateFormat.format(new Date(timestamp));
}
public static int toDate(String str) {
return toDate(str, "yyyy-MM-dd");
}
public static int toDate(String str, String format) {
return DateTimeUtils.parseDate(str, format);
}
public static long toTimestamp(String str) {
return toTimestamp(str, "yyyy-MM-dd HH:mm:ss");
}
public static long toTimestamp(String str, String format) {
SimpleDateFormat dateFormat = new SimpleDateFormat(format);
try {
return dateFormat.parse(str).getTime();
} catch (ParseException e) {
LOG.error("Unsupported date type convert: {}", str);
throw new RuntimeException(e);
}
}
public static int timestampDiff(String symbol, long fromDate, long toDate) {
Calendar from = Calendar.getInstance();
from.setTime(new Date(fromDate));
Calendar to = Calendar.getInstance();
to.setTime(new Date(toDate));
Long second = (to.getTimeInMillis() - from.getTimeInMillis()) / 1000;
switch (symbol) {
case "SECOND":
return second.intValue();
case "MINUTE":
return second.intValue() / 60;
case "HOUR":
return second.intValue() / 3600;
case "DAY":
return second.intValue() / (24 * 3600);
case "MONTH":
return to.get(Calendar.YEAR) * 12
+ to.get(Calendar.MONDAY)
- (from.get(Calendar.YEAR) * 12 + from.get(Calendar.MONDAY));
case "YEAR":
return to.get(Calendar.YEAR) - from.get(Calendar.YEAR);
default:
LOG.error("Unsupported timestamp diff: {}", symbol);
throw new RuntimeException("Unsupported timestamp diff: " + symbol);
}
}
public static boolean betweenAsymmetric(String value, String minValue, String maxValue) {
if (value == null) {
return false;
}
return value.compareTo(minValue) >= 0 && value.compareTo(maxValue) <= 0;
}
public static boolean betweenAsymmetric(Short value, short minValue, short maxValue) {
if (value == null) {
return false;
}
return value >= minValue && value <= maxValue;
}
public static boolean betweenAsymmetric(Integer value, int minValue, int maxValue) {
if (value == null) {
return false;
}
return value >= minValue && value <= maxValue;
}
public static boolean betweenAsymmetric(Long value, long minValue, long maxValue) {
if (value == null) {
return false;
}
return value >= minValue && value <= maxValue;
}
public static boolean betweenAsymmetric(Float value, float minValue, float maxValue) {
if (value == null) {
return false;
}
return value >= minValue && value <= maxValue;
}
public static boolean betweenAsymmetric(Double value, double minValue, double maxValue) {
if (value == null) {
return false;
}
return value >= minValue && value <= maxValue;
}
public static boolean betweenAsymmetric(
BigDecimal value, BigDecimal minValue, BigDecimal maxValue) {
if (value == null) {
return false;
}
return value.compareTo(minValue) >= 0 && value.compareTo(maxValue) <= 0;
}
public static boolean notBetweenAsymmetric(String value, String minValue, String maxValue) {
return !betweenAsymmetric(value, minValue, maxValue);
}
public static boolean notBetweenAsymmetric(Short value, short minValue, short maxValue) {
return !betweenAsymmetric(value, minValue, maxValue);
}
public static boolean notBetweenAsymmetric(Integer value, int minValue, int maxValue) {
return !betweenAsymmetric(value, minValue, maxValue);
}
public static boolean notBetweenAsymmetric(Long value, long minValue, long maxValue) {
return !betweenAsymmetric(value, minValue, maxValue);
}
public static boolean notBetweenAsymmetric(Float value, float minValue, float maxValue) {
return !betweenAsymmetric(value, minValue, maxValue);
}
public static boolean notBetweenAsymmetric(Double value, double minValue, double maxValue) {
return !betweenAsymmetric(value, minValue, maxValue);
}
public static boolean notBetweenAsymmetric(
BigDecimal value, BigDecimal minValue, BigDecimal maxValue) {
return !betweenAsymmetric(value, minValue, maxValue);
}
public static boolean in(String value, String... str) {
return Arrays.stream(str).anyMatch(item -> value.equals(item));
}
public static boolean in(Short value, Short... values) {
return Arrays.stream(values).anyMatch(item -> value.equals(item));
}
public static boolean in(Integer value, Integer... values) {
return Arrays.stream(values).anyMatch(item -> value.equals(item));
}
public static boolean in(Long value, Long... values) {
return Arrays.stream(values).anyMatch(item -> value.equals(item));
}
public static boolean in(Float value, Float... values) {
return Arrays.stream(values).anyMatch(item -> value.equals(item));
}
public static boolean in(Double value, Double... values) {
return Arrays.stream(values).anyMatch(item -> value.equals(item));
}
public static boolean in(BigDecimal value, BigDecimal... values) {
return Arrays.stream(values).anyMatch(item -> value.equals(item));
}
public static boolean notIn(String value, String... values) {
return !in(value, values);
}
public static boolean notIn(Short value, Short... values) {
return !in(value, values);
}
public static boolean notIn(Integer value, Integer... values) {
return !in(value, values);
}
public static boolean notIn(Long value, Long... values) {
return !in(value, values);
}
public static boolean notIn(Float value, Float... values) {
return !in(value, values);
}
public static boolean notIn(Double value, Double... values) {
return !in(value, values);
}
public static boolean notIn(BigDecimal value, BigDecimal... values) {
return !in(value, values);
}
public static int charLength(String str) {
return str.length();
}
public static String trim(String symbol, String target, String str) {
return str.trim();
}
/**
* Returns a string resulting from replacing all substrings that match the regular expression
* with replacement.
*/
public static String regexpReplace(String str, String regex, String replacement) {
if (str == null || regex == null || replacement == null) {
return null;
}
try {
return str.replaceAll(regex, Matcher.quoteReplacement(replacement));
} catch (Exception e) {
LOG.error(
String.format(
"Exception in regexpReplace('%s', '%s', '%s')",
str, regex, replacement),
e);
// return null if exception in regex replace
return null;
}
}
public static String concat(String... str) {
return String.join("", str);
}
public static boolean like(String str, String regex) {
return Pattern.compile(regex).matcher(str).find();
}
public static boolean notLike(String str, String regex) {
return !like(str, regex);
}
public static String substr(String str, int beginIndex) {
return str.substring(beginIndex);
}
public static String substr(String str, int beginIndex, int length) {
return str.substring(beginIndex, beginIndex + length);
}
public static String upper(String str) {
return str.toUpperCase();
}
public static String lower(String str) {
return str.toLowerCase();
}
/** SQL <code>ABS</code> operator applied to byte values. */
public static byte abs(byte b0) {
return (byte) Math.abs(b0);
}
/** SQL <code>ABS</code> operator applied to short values. */
public static short abs(short b0) {
return (short) Math.abs(b0);
}
/** SQL <code>ABS</code> operator applied to int values. */
public static int abs(int b0) {
return Math.abs(b0);
}
/** SQL <code>ABS</code> operator applied to long values. */
public static long abs(long b0) {
return Math.abs(b0);
}
/** SQL <code>ABS</code> operator applied to float values. */
public static float abs(float b0) {
return Math.abs(b0);
}
/** SQL <code>ABS</code> operator applied to double values. */
public static double abs(double b0) {
return Math.abs(b0);
}
public static double floor(double b0) {
return Math.floor(b0);
}
public static float floor(float b0) {
return (float) Math.floor(b0);
}
/** SQL <code>FLOOR</code> operator applied to int values. */
public static int floor(int b0, int b1) {
int r = b0 % b1;
if (r < 0) {
r += b1;
}
return b0 - r;
}
/** SQL <code>FLOOR</code> operator applied to long values. */
public static long floor(long b0, long b1) {
long r = b0 % b1;
if (r < 0) {
r += b1;
}
return b0 - r;
}
public static double ceil(double b0) {
return Math.ceil(b0);
}
public static float ceil(float b0) {
return (float) Math.ceil(b0);
}
/** SQL <code>CEIL</code> operator applied to int values. */
public static int ceil(int b0, int b1) {
int r = b0 % b1;
if (r > 0) {
r -= b1;
}
return b0 - r;
}
/** SQL <code>CEIL</code> operator applied to long values. */
public static long ceil(long b0, long b1) {
return floor(b0 + b1 - 1, b1);
}
// SQL ROUND
/** SQL <code>ROUND</code> operator applied to byte values. */
public static byte round(byte b0) {
return round(b0, 0);
}
/** SQL <code>ROUND</code> operator applied to byte values. */
public static byte round(byte b0, int b1) {
return round(BigDecimal.valueOf(b0), b1).byteValue();
}
/** SQL <code>ROUND</code> operator applied to short values. */
public static short round(short b0) {
return round(b0, 0);
}
/** SQL <code>ROUND</code> operator applied to short values. */
public static short round(short b0, int b1) {
return round(BigDecimal.valueOf(b0), b1).shortValue();
}
/** SQL <code>ROUND</code> operator applied to int values. */
public static int round(int b0) {
return round(b0, 0);
}
/** SQL <code>ROUND</code> operator applied to int values. */
public static int round(int b0, int b1) {
return round(BigDecimal.valueOf(b0), b1).intValue();
}
/** SQL <code>ROUND</code> operator applied to long values. */
public static long round(long b0) {
return round(b0, 0);
}
/** SQL <code>ROUND</code> operator applied to long values. */
public static long round(long b0, int b1) {
return round(BigDecimal.valueOf(b0), b1).longValue();
}
/** SQL <code>ROUND</code> operator applied to BigDecimal values. */
public static BigDecimal round(BigDecimal b0) {
return round(b0, 0);
}
/** SQL <code>ROUND</code> operator applied to BigDecimal values. */
public static BigDecimal round(BigDecimal b0, int b1) {
return b0.movePointRight(b1).setScale(0, RoundingMode.HALF_UP).movePointLeft(b1);
}
/** SQL <code>ROUND</code> operator applied to float values. */
public static float round(float b0) {
return round(b0, 0);
}
/** SQL <code>ROUND</code> operator applied to float values. */
public static float round(float b0, int b1) {
return round(BigDecimal.valueOf(b0), b1).floatValue();
}
/** SQL <code>ROUND</code> operator applied to double values. */
public static double round(double b0) {
return round(b0, 0);
}
/** SQL <code>ROUND</code> operator applied to double values. */
public static double round(double b0, int b1) {
return round(BigDecimal.valueOf(b0), b1).doubleValue();
}
public static String uuid() {
return UUID.randomUUID().toString();
}
public static String uuid(byte[] b) {
return UUID.nameUUIDFromBytes(b).toString();
}
public static boolean valueEquals(Object object1, Object object2) {
return (object1 != null && object2 != null) && object1.equals(object2);
}
}

@ -0,0 +1,106 @@
/*
* 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.transform;
import org.apache.flink.cdc.common.schema.Column;
import org.apache.flink.cdc.common.types.DataType;
import org.apache.flink.cdc.common.utils.StringUtils;
import java.io.Serializable;
import java.util.List;
/**
* The ProjectionColumn applies to describe the information of the transformation column. If it only
* has column info, it describes the data column. If it has column info and expression info, it
* describes the user-defined computed columns.
*
* <p>A projection column contains:
*
* <ul>
* <li>column: column information parsed from projection.
* <li>expression: a string for column expression split from the user-defined projection.
* <li>scriptExpression: a string for column script expression compiled from the column
* expression.
* <li>originalColumnNames: a list for recording the name of all columns used by the column
* expression.
* </ul>
*/
public class ProjectionColumn implements Serializable {
private static final long serialVersionUID = 1L;
private final Column column;
private final String expression;
private final String scriptExpression;
private final List<String> originalColumnNames;
private TransformExpressionKey transformExpressionKey;
public ProjectionColumn(
Column column,
String expression,
String scriptExpression,
List<String> originalColumnNames) {
this.column = column;
this.expression = expression;
this.scriptExpression = scriptExpression;
this.originalColumnNames = originalColumnNames;
}
public Column getColumn() {
return column;
}
public String getColumnName() {
return column.getName();
}
public DataType getDataType() {
return column.getType();
}
public String getScriptExpression() {
return scriptExpression;
}
public List<String> getOriginalColumnNames() {
return originalColumnNames;
}
public void setTransformExpressionKey(TransformExpressionKey transformExpressionKey) {
this.transformExpressionKey = transformExpressionKey;
}
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);
}
public static ProjectionColumn of(
String columnName,
DataType dataType,
String expression,
String scriptExpression,
List<String> originalColumnNames) {
return new ProjectionColumn(
Column.physicalColumn(columnName, dataType),
expression,
scriptExpression,
originalColumnNames);
}
}

@ -0,0 +1,153 @@
/*
* 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.transform;
import org.apache.flink.cdc.common.data.RecordData;
import org.apache.flink.cdc.common.data.binary.BinaryRecordData;
import org.apache.flink.cdc.common.schema.Column;
import org.apache.flink.cdc.runtime.parser.JaninoCompiler;
import org.apache.flink.cdc.runtime.parser.TransformParser;
import org.apache.flink.cdc.runtime.typeutils.DataTypeConverter;
import org.codehaus.janino.ExpressionEvaluator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.lang.reflect.InvocationTargetException;
import java.util.ArrayList;
import java.util.List;
/**
* The processor of the projection column. It processes the data column and the user-defined
* computed columns.
*/
public class ProjectionColumnProcessor {
private static final Logger LOG = LoggerFactory.getLogger(ProjectionColumnProcessor.class);
private TableInfo tableInfo;
private ProjectionColumn projectionColumn;
private String timezone;
private TransformExpressionKey transformExpressionKey;
public ProjectionColumnProcessor(
TableInfo tableInfo, ProjectionColumn projectionColumn, String timezone) {
this.tableInfo = tableInfo;
this.projectionColumn = projectionColumn;
this.timezone = timezone;
this.transformExpressionKey = generateTransformExpressionKey();
}
public static ProjectionColumnProcessor of(
TableInfo tableInfo, ProjectionColumn projectionColumn, String timezone) {
return new ProjectionColumnProcessor(tableInfo, projectionColumn, timezone);
}
public Object evaluate(BinaryRecordData after, long epochTime) {
ExpressionEvaluator expressionEvaluator =
TransformExpressionCompiler.compileExpression(transformExpressionKey);
try {
return expressionEvaluator.evaluate(generateParams(after, epochTime));
} catch (InvocationTargetException e) {
LOG.error(
"Table:{} column:{} projection:{} execute failed. {}",
tableInfo.getName(),
projectionColumn.getColumnName(),
projectionColumn.getScriptExpression(),
e);
throw new RuntimeException(e);
}
}
private Object[] generateParams(BinaryRecordData after, long epochTime) {
List<Object> params = new ArrayList<>();
List<Column> columns = tableInfo.getSchema().getColumns();
RecordData.FieldGetter[] fieldGetters = tableInfo.getFieldGetters();
for (String originalColumnName : projectionColumn.getOriginalColumnNames()) {
if (originalColumnName.equals(TransformParser.DEFAULT_NAMESPACE_NAME)) {
params.add(tableInfo.getNamespace());
continue;
}
if (originalColumnName.equals(TransformParser.DEFAULT_SCHEMA_NAME)) {
params.add(tableInfo.getSchemaName());
continue;
}
if (originalColumnName.equals(TransformParser.DEFAULT_TABLE_NAME)) {
params.add(tableInfo.getTableName());
continue;
}
for (int i = 0; i < columns.size(); i++) {
Column column = columns.get(i);
if (column.getName().equals(originalColumnName)) {
params.add(
DataTypeConverter.convertToOriginal(
fieldGetters[i].getFieldOrNull(after), column.getType()));
break;
}
}
}
params.add(timezone);
params.add(epochTime);
return params.toArray();
}
private TransformExpressionKey generateTransformExpressionKey() {
List<String> argumentNames = new ArrayList<>();
List<Class<?>> paramTypes = new ArrayList<>();
List<Column> columns = tableInfo.getSchema().getColumns();
String scriptExpression = projectionColumn.getScriptExpression();
List<String> originalColumnNames = projectionColumn.getOriginalColumnNames();
for (String originalColumnName : originalColumnNames) {
for (int i = 0; i < columns.size(); i++) {
Column column = columns.get(i);
if (column.getName().equals(originalColumnName)) {
argumentNames.add(originalColumnName);
paramTypes.add(DataTypeConverter.convertOriginalClass(column.getType()));
break;
}
}
}
if (scriptExpression.contains(TransformParser.DEFAULT_NAMESPACE_NAME)
&& !argumentNames.contains(TransformParser.DEFAULT_NAMESPACE_NAME)) {
argumentNames.add(TransformParser.DEFAULT_NAMESPACE_NAME);
paramTypes.add(String.class);
}
if (scriptExpression.contains(TransformParser.DEFAULT_SCHEMA_NAME)
&& !argumentNames.contains(TransformParser.DEFAULT_SCHEMA_NAME)) {
argumentNames.add(TransformParser.DEFAULT_SCHEMA_NAME);
paramTypes.add(String.class);
}
if (scriptExpression.contains(TransformParser.DEFAULT_TABLE_NAME)
&& !argumentNames.contains(TransformParser.DEFAULT_TABLE_NAME)) {
argumentNames.add(TransformParser.DEFAULT_TABLE_NAME);
paramTypes.add(String.class);
}
argumentNames.add(JaninoCompiler.DEFAULT_TIME_ZONE);
paramTypes.add(String.class);
argumentNames.add(JaninoCompiler.DEFAULT_EPOCH_TIME);
paramTypes.add(Long.class);
return TransformExpressionKey.of(
JaninoCompiler.loadSystemFunction(scriptExpression),
argumentNames,
paramTypes,
DataTypeConverter.convertOriginalClass(projectionColumn.getDataType()));
}
}

@ -0,0 +1,85 @@
/*
* 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.transform;
import org.apache.flink.cdc.common.schema.Schema;
import org.apache.flink.cdc.common.utils.StringUtils;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* a Pojo class to describe the information of the primaryKeys/partitionKeys/options transformation
* of {@link Schema}.
*/
public class SchemaMetadataTransform implements Serializable {
private static final long serialVersionUID = 1L;
private List<String> primaryKeys = new ArrayList<>();
private List<String> partitionKeys = new ArrayList<>();
private Map<String, String> options = new HashMap<>();
public SchemaMetadataTransform(
String primaryKeyString, String partitionKeyString, String tableOptionString) {
if (!StringUtils.isNullOrWhitespaceOnly(primaryKeyString)) {
String[] primaryKeyArr = primaryKeyString.split(",");
for (int i = 0; i < primaryKeyArr.length; i++) {
primaryKeyArr[i] = primaryKeyArr[i].trim();
}
primaryKeys = Arrays.asList(primaryKeyArr);
}
if (!StringUtils.isNullOrWhitespaceOnly(partitionKeyString)) {
String[] partitionKeyArr = partitionKeyString.split(",");
for (int i = 0; i < partitionKeyArr.length; i++) {
partitionKeyArr[i] = partitionKeyArr[i].trim();
}
partitionKeys = Arrays.asList(partitionKeyArr);
}
if (!StringUtils.isNullOrWhitespaceOnly(tableOptionString)) {
for (String tableOption : tableOptionString.split(",")) {
String[] kv = tableOption.split("=");
if (kv.length != 2) {
throw new IllegalArgumentException(
"table option format error: "
+ tableOptionString
+ ", it should be like `key1=value1,key2=value2`.");
}
options.put(kv[0].trim(), kv[1].trim());
}
}
}
public List<String> getPrimaryKeys() {
return primaryKeys;
}
public List<String> getPartitionKeys() {
return partitionKeys;
}
public Map<String, String> getOptions() {
return options;
}
}

@ -0,0 +1,150 @@
/*
* 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.transform;
import org.apache.flink.cdc.common.data.RecordData;
import org.apache.flink.cdc.common.event.TableId;
import org.apache.flink.cdc.common.schema.Schema;
import org.apache.flink.cdc.common.utils.SchemaUtils;
import org.apache.flink.cdc.runtime.serializer.TableIdSerializer;
import org.apache.flink.cdc.runtime.serializer.schema.SchemaSerializer;
import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
import org.apache.flink.cdc.runtime.typeutils.DataTypeConverter;
import org.apache.flink.core.io.SimpleVersionedSerializer;
import org.apache.flink.core.memory.DataInputViewStreamWrapper;
import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.List;
/** The TableInfo applies to cache schema change and fieldGetters. */
public class TableChangeInfo {
private TableId tableId;
private Schema originalSchema;
private Schema transformedSchema;
private RecordData.FieldGetter[] fieldGetters;
private BinaryRecordDataGenerator recordDataGenerator;
public static final TableChangeInfo.Serializer SERIALIZER = new TableChangeInfo.Serializer();
public TableChangeInfo(
TableId tableId,
Schema originalSchema,
Schema transformedSchema,
RecordData.FieldGetter[] fieldGetters,
BinaryRecordDataGenerator recordDataGenerator) {
this.tableId = tableId;
this.originalSchema = originalSchema;
this.transformedSchema = transformedSchema;
this.fieldGetters = fieldGetters;
this.recordDataGenerator = recordDataGenerator;
}
public String getName() {
return tableId.identifier();
}
public String getTableName() {
return tableId.getTableName();
}
public String getSchemaName() {
return tableId.getSchemaName();
}
public TableId getTableId() {
return tableId;
}
public Schema getOriginalSchema() {
return originalSchema;
}
public Schema getTransformedSchema() {
return transformedSchema;
}
public RecordData.FieldGetter[] getFieldGetters() {
return fieldGetters;
}
public BinaryRecordDataGenerator getRecordDataGenerator() {
return recordDataGenerator;
}
public static TableChangeInfo of(
TableId tableId, Schema originalSchema, Schema transformedSchema) {
List<RecordData.FieldGetter> fieldGetters =
SchemaUtils.createFieldGetters(originalSchema.getColumns());
BinaryRecordDataGenerator recordDataGenerator =
new BinaryRecordDataGenerator(
DataTypeConverter.toRowType(transformedSchema.getColumns()));
return new TableChangeInfo(
tableId,
originalSchema,
transformedSchema,
fieldGetters.toArray(new RecordData.FieldGetter[0]),
recordDataGenerator);
}
/** Serializer for {@link TableChangeInfo}. */
public static class Serializer implements SimpleVersionedSerializer<TableChangeInfo> {
public static final int CURRENT_VERSION = 0;
@Override
public int getVersion() {
return CURRENT_VERSION;
}
@Override
public byte[] serialize(TableChangeInfo tableChangeInfo) throws IOException {
TableIdSerializer tableIdSerializer = TableIdSerializer.INSTANCE;
SchemaSerializer schemaSerializer = SchemaSerializer.INSTANCE;
try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream out = new DataOutputStream(baos)) {
tableIdSerializer.serialize(
tableChangeInfo.getTableId(), new DataOutputViewStreamWrapper(out));
schemaSerializer.serialize(
tableChangeInfo.originalSchema, new DataOutputViewStreamWrapper(out));
schemaSerializer.serialize(
tableChangeInfo.transformedSchema, new DataOutputViewStreamWrapper(out));
return baos.toByteArray();
}
}
@Override
public TableChangeInfo deserialize(int version, byte[] serialized) throws IOException {
TableIdSerializer tableIdSerializer = TableIdSerializer.INSTANCE;
SchemaSerializer schemaSerializer = SchemaSerializer.INSTANCE;
try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
DataInputStream in = new DataInputStream(bais)) {
TableId tableId = tableIdSerializer.deserialize(new DataInputViewStreamWrapper(in));
Schema originalSchema =
schemaSerializer.deserialize(new DataInputViewStreamWrapper(in));
Schema transformedSchema =
schemaSerializer.deserialize(new DataInputViewStreamWrapper(in));
return TableChangeInfo.of(tableId, originalSchema, transformedSchema);
}
}
}
}

@ -0,0 +1,90 @@
/*
* 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.transform;
import org.apache.flink.cdc.common.data.RecordData;
import org.apache.flink.cdc.common.event.TableId;
import org.apache.flink.cdc.common.schema.Schema;
import org.apache.flink.cdc.common.utils.SchemaUtils;
import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
import org.apache.flink.cdc.runtime.typeutils.DataTypeConverter;
import java.util.List;
/** The TableInfo applies to cache schema and fieldGetters. */
public class TableInfo {
private TableId tableId;
private Schema schema;
private RecordData.FieldGetter[] fieldGetters;
private BinaryRecordDataGenerator recordDataGenerator;
public TableInfo(
TableId tableId,
Schema schema,
RecordData.FieldGetter[] fieldGetters,
BinaryRecordDataGenerator recordDataGenerator) {
this.tableId = tableId;
this.schema = schema;
this.fieldGetters = fieldGetters;
this.recordDataGenerator = recordDataGenerator;
}
public String getName() {
return tableId.identifier();
}
public String getTableName() {
return tableId.getTableName();
}
public String getSchemaName() {
return tableId.getSchemaName();
}
public String getNamespace() {
return tableId.getNamespace();
}
public TableId getTableId() {
return tableId;
}
public Schema getSchema() {
return schema;
}
public RecordData.FieldGetter[] getFieldGetters() {
return fieldGetters;
}
public BinaryRecordDataGenerator getRecordDataGenerator() {
return recordDataGenerator;
}
public static TableInfo of(TableId tableId, Schema schema) {
List<RecordData.FieldGetter> fieldGetters =
SchemaUtils.createFieldGetters(schema.getColumns());
BinaryRecordDataGenerator recordDataGenerator =
new BinaryRecordDataGenerator(DataTypeConverter.toRowType(schema.getColumns()));
return new TableInfo(
tableId,
schema,
fieldGetters.toArray(new RecordData.FieldGetter[0]),
recordDataGenerator);
}
}

@ -0,0 +1,406 @@
/*
* 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.transform;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.api.java.tuple.Tuple4;
import org.apache.flink.cdc.common.data.binary.BinaryRecordData;
import org.apache.flink.cdc.common.event.CreateTableEvent;
import org.apache.flink.cdc.common.event.DataChangeEvent;
import org.apache.flink.cdc.common.event.Event;
import org.apache.flink.cdc.common.event.SchemaChangeEvent;
import org.apache.flink.cdc.common.event.TableId;
import org.apache.flink.cdc.common.pipeline.PipelineOptions;
import org.apache.flink.cdc.common.schema.Schema;
import org.apache.flink.cdc.common.schema.Selectors;
import org.apache.flink.cdc.common.utils.SchemaUtils;
import org.apache.flink.cdc.common.utils.StringUtils;
import org.apache.flink.cdc.runtime.operators.sink.SchemaEvolutionClient;
import org.apache.flink.cdc.runtime.parser.TransformParser;
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.state.StateInitializationContext;
import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.tasks.StreamTask;
import javax.annotation.Nullable;
import java.time.ZoneId;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.ConcurrentHashMap;
import java.util.stream.Collectors;
/** A data process function that applies user-defined transform logics. */
public class TransformDataOperator extends AbstractStreamOperator<Event>
implements OneInputStreamOperator<Event, Event> {
private SchemaEvolutionClient schemaEvolutionClient;
private final OperatorID schemaOperatorID;
private final String timezone;
private final List<Tuple3<String, String, String>> transformRules;
private transient List<
Tuple4<
Selectors,
Optional<TransformProjection>,
Optional<TransformFilter>,
Boolean>>
transforms;
/** keep the relationship of TableId and table information. */
private final Map<TableId, TableInfo> tableInfoMap;
private transient Map<TransformProjection, TransformProjectionProcessor>
transformProjectionProcessorMap;
private transient Map<TransformFilter, TransformFilterProcessor> transformFilterProcessorMap;
public static TransformDataOperator.Builder newBuilder() {
return new TransformDataOperator.Builder();
}
/** Builder of {@link TransformDataOperator}. */
public static class Builder {
private final List<Tuple3<String, String, String>> transformRules = new ArrayList<>();
private OperatorID schemaOperatorID;
private String timezone;
public TransformDataOperator.Builder addTransform(
String tableInclusions, @Nullable String projection, @Nullable String filter) {
transformRules.add(Tuple3.of(tableInclusions, projection, filter));
return this;
}
public TransformDataOperator.Builder addSchemaOperatorID(OperatorID schemaOperatorID) {
this.schemaOperatorID = schemaOperatorID;
return this;
}
public TransformDataOperator.Builder addTimezone(String timezone) {
if (PipelineOptions.PIPELINE_LOCAL_TIME_ZONE.defaultValue().equals(timezone)) {
this.timezone = ZoneId.systemDefault().toString();
} else {
this.timezone = timezone;
}
return this;
}
public TransformDataOperator build() {
return new TransformDataOperator(transformRules, schemaOperatorID, timezone);
}
}
private TransformDataOperator(
List<Tuple3<String, String, String>> transformRules,
OperatorID schemaOperatorID,
String timezone) {
this.transformRules = transformRules;
this.schemaOperatorID = schemaOperatorID;
this.timezone = timezone;
this.tableInfoMap = new ConcurrentHashMap<>();
this.transformFilterProcessorMap = new ConcurrentHashMap<>();
this.transformProjectionProcessorMap = new ConcurrentHashMap<>();
}
@Override
public void setup(
StreamTask<?, ?> containingTask,
StreamConfig config,
Output<StreamRecord<Event>> output) {
super.setup(containingTask, config, output);
schemaEvolutionClient =
new SchemaEvolutionClient(
containingTask.getEnvironment().getOperatorCoordinatorEventGateway(),
schemaOperatorID);
}
@Override
public void open() throws Exception {
super.open();
transforms =
transformRules.stream()
.map(
tuple3 -> {
String tableInclusions = tuple3.f0;
String projection = tuple3.f1;
String filterExpression = tuple3.f2;
Selectors selectors =
new Selectors.SelectorsBuilder()
.includeTables(tableInclusions)
.build();
return new Tuple4<>(
selectors,
TransformProjection.of(projection),
TransformFilter.of(filterExpression),
containFilteredComputedColumn(
projection, filterExpression));
})
.collect(Collectors.toList());
this.transformFilterProcessorMap = new ConcurrentHashMap<>();
this.transformProjectionProcessorMap = new ConcurrentHashMap<>();
}
@Override
public void finish() throws Exception {
super.finish();
clearOperator();
}
@Override
public void close() throws Exception {
super.close();
clearOperator();
}
@Override
public void initializeState(StateInitializationContext context) throws Exception {
schemaEvolutionClient.registerSubtask(getRuntimeContext().getIndexOfThisSubtask());
}
@Override
public void processElement(StreamRecord<Event> element) throws Exception {
Event event = element.getValue();
if (event instanceof SchemaChangeEvent) {
event = cacheSchema((SchemaChangeEvent) event);
output.collect(new StreamRecord<>(event));
} else if (event instanceof DataChangeEvent) {
Optional<DataChangeEvent> dataChangeEventOptional =
processDataChangeEvent(((DataChangeEvent) event));
if (dataChangeEventOptional.isPresent()) {
output.collect(new StreamRecord<>(dataChangeEventOptional.get()));
}
}
}
private SchemaChangeEvent cacheSchema(SchemaChangeEvent event) throws Exception {
TableId tableId = event.tableId();
Schema newSchema;
if (event instanceof CreateTableEvent) {
newSchema = ((CreateTableEvent) event).getSchema();
} else {
newSchema =
SchemaUtils.applySchemaChangeEvent(
getTableInfoFromSchemaEvolutionClient(tableId).getSchema(), event);
}
transformSchema(tableId, newSchema);
tableInfoMap.put(tableId, TableInfo.of(tableId, newSchema));
return event;
}
private TableInfo getTableInfoFromSchemaEvolutionClient(TableId tableId) throws Exception {
TableInfo tableInfo = tableInfoMap.get(tableId);
if (tableInfo == null) {
Optional<Schema> schemaOptional = schemaEvolutionClient.getLatestSchema(tableId);
if (schemaOptional.isPresent()) {
tableInfo = TableInfo.of(tableId, schemaOptional.get());
} else {
throw new RuntimeException(
"Could not find schema message from SchemaRegistry for " + tableId);
}
}
return tableInfo;
}
private void transformSchema(TableId tableId, Schema schema) throws Exception {
for (Tuple4<Selectors, Optional<TransformProjection>, Optional<TransformFilter>, Boolean>
transform : transforms) {
Selectors selectors = transform.f0;
if (selectors.isMatch(tableId) && transform.f1.isPresent()) {
TransformProjection transformProjection = transform.f1.get();
if (transformProjection.isValid()) {
if (!transformProjectionProcessorMap.containsKey(transformProjection)) {
transformProjectionProcessorMap.put(
transformProjection,
TransformProjectionProcessor.of(transformProjection));
}
TransformProjectionProcessor transformProjectionProcessor =
transformProjectionProcessorMap.get(transformProjection);
// update the columns of projection and add the column of projection into Schema
transformProjectionProcessor.processSchemaChangeEvent(schema);
}
}
}
}
private Optional<DataChangeEvent> processDataChangeEvent(DataChangeEvent dataChangeEvent)
throws Exception {
TableId tableId = dataChangeEvent.tableId();
List<Optional<DataChangeEvent>> transformedDataChangeEventOptionalList = new ArrayList<>();
long epochTime = System.currentTimeMillis();
for (Tuple4<Selectors, Optional<TransformProjection>, Optional<TransformFilter>, Boolean>
transform : transforms) {
Selectors selectors = transform.f0;
Boolean isPreProjection = transform.f3;
if (selectors.isMatch(tableId)) {
Optional<DataChangeEvent> dataChangeEventOptional = Optional.of(dataChangeEvent);
Optional<TransformProjection> transformProjectionOptional = transform.f1;
if (isPreProjection
&& transformProjectionOptional.isPresent()
&& transformProjectionOptional.get().isValid()) {
TransformProjection transformProjection = transformProjectionOptional.get();
if (!transformProjectionProcessorMap.containsKey(transformProjection)
|| !transformProjectionProcessorMap
.get(transformProjection)
.hasTableInfo()) {
transformProjectionProcessorMap.put(
transformProjection,
TransformProjectionProcessor.of(
getTableInfoFromSchemaEvolutionClient(tableId),
transformProjection,
timezone));
}
TransformProjectionProcessor transformProjectionProcessor =
transformProjectionProcessorMap.get(transformProjection);
dataChangeEventOptional =
processProjection(
transformProjectionProcessor,
dataChangeEventOptional.get(),
epochTime);
}
Optional<TransformFilter> transformFilterOptional = transform.f2;
if (transformFilterOptional.isPresent()
&& transformFilterOptional.get().isVaild()) {
TransformFilter transformFilter = transformFilterOptional.get();
if (!transformFilterProcessorMap.containsKey(transformFilter)) {
transformFilterProcessorMap.put(
transformFilter,
TransformFilterProcessor.of(
getTableInfoFromSchemaEvolutionClient(tableId),
transformFilter,
timezone));
}
TransformFilterProcessor transformFilterProcessor =
transformFilterProcessorMap.get(transformFilter);
dataChangeEventOptional =
processFilter(
transformFilterProcessor,
dataChangeEventOptional.get(),
epochTime);
}
if (!isPreProjection
&& dataChangeEventOptional.isPresent()
&& transformProjectionOptional.isPresent()
&& transformProjectionOptional.get().isValid()) {
TransformProjection transformProjection = transformProjectionOptional.get();
if (!transformProjectionProcessorMap.containsKey(transformProjection)
|| !transformProjectionProcessorMap
.get(transformProjection)
.hasTableInfo()) {
transformProjectionProcessorMap.put(
transformProjection,
TransformProjectionProcessor.of(
getTableInfoFromSchemaEvolutionClient(tableId),
transformProjection,
timezone));
}
TransformProjectionProcessor transformProjectionProcessor =
transformProjectionProcessorMap.get(transformProjection);
dataChangeEventOptional =
processProjection(
transformProjectionProcessor,
dataChangeEventOptional.get(),
epochTime);
}
transformedDataChangeEventOptionalList.add(dataChangeEventOptional);
}
}
if (transformedDataChangeEventOptionalList.isEmpty()) {
return Optional.of(dataChangeEvent);
} else {
for (Optional<DataChangeEvent> dataChangeEventOptional :
transformedDataChangeEventOptionalList) {
if (dataChangeEventOptional.isPresent()) {
return dataChangeEventOptional;
}
}
return Optional.empty();
}
}
private Optional<DataChangeEvent> processFilter(
TransformFilterProcessor transformFilterProcessor,
DataChangeEvent dataChangeEvent,
long epochTime)
throws Exception {
BinaryRecordData before = (BinaryRecordData) dataChangeEvent.before();
BinaryRecordData after = (BinaryRecordData) dataChangeEvent.after();
// insert and update event only process afterData, delete only process beforeData
if (after != null) {
if (transformFilterProcessor.process(after, epochTime)) {
return Optional.of(dataChangeEvent);
} else {
return Optional.empty();
}
} else if (before != null) {
if (transformFilterProcessor.process(before, epochTime)) {
return Optional.of(dataChangeEvent);
} else {
return Optional.empty();
}
}
return Optional.empty();
}
private Optional<DataChangeEvent> processProjection(
TransformProjectionProcessor transformProjectionProcessor,
DataChangeEvent dataChangeEvent,
long epochTime)
throws Exception {
BinaryRecordData before = (BinaryRecordData) dataChangeEvent.before();
BinaryRecordData after = (BinaryRecordData) dataChangeEvent.after();
if (before != null) {
BinaryRecordData projectedBefore =
transformProjectionProcessor.processData(before, epochTime);
dataChangeEvent = DataChangeEvent.projectBefore(dataChangeEvent, projectedBefore);
}
if (after != null) {
BinaryRecordData projectedAfter =
transformProjectionProcessor.processData(after, epochTime);
dataChangeEvent = DataChangeEvent.projectAfter(dataChangeEvent, projectedAfter);
}
return Optional.of(dataChangeEvent);
}
private boolean containFilteredComputedColumn(String projection, String filter) {
boolean contain = false;
if (StringUtils.isNullOrWhitespaceOnly(projection)
|| StringUtils.isNullOrWhitespaceOnly(filter)) {
return contain;
}
List<String> computedColumnNames = TransformParser.parseComputedColumnNames(projection);
List<String> filteredColumnNames = TransformParser.parseFilterColumnNameList(filter);
for (String computedColumnName : computedColumnNames) {
if (filteredColumnNames.contains(computedColumnName)) {
return true;
}
}
return contain;
}
private void clearOperator() {
this.transforms = null;
this.transformProjectionProcessorMap = null;
this.transformFilterProcessorMap = null;
TransformExpressionCompiler.cleanUp();
}
}

@ -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.transform;
import org.apache.flink.api.common.InvalidProgramException;
import org.apache.flink.util.FlinkRuntimeException;
import org.apache.flink.shaded.guava31.com.google.common.cache.Cache;
import org.apache.flink.shaded.guava31.com.google.common.cache.CacheBuilder;
import org.codehaus.commons.compiler.CompileException;
import org.codehaus.janino.ExpressionEvaluator;
/**
* The processor of the transform expression. It processes the expression of projections and
* filters.
*/
public class TransformExpressionCompiler {
static final Cache<TransformExpressionKey, ExpressionEvaluator> COMPILED_EXPRESSION_CACHE =
CacheBuilder.newBuilder().softValues().build();
/** Triggers internal garbage collection of expired cache entries. */
public static void cleanUp() {
COMPILED_EXPRESSION_CACHE.cleanUp();
}
/** Compiles an expression code to a janino {@link ExpressionEvaluator}. */
public static ExpressionEvaluator compileExpression(TransformExpressionKey key) {
try {
return COMPILED_EXPRESSION_CACHE.get(
key,
() -> {
ExpressionEvaluator expressionEvaluator = new ExpressionEvaluator();
// Input args
expressionEvaluator.setParameters(
key.getArgumentNames().toArray(new String[0]),
key.getArgumentClasses().toArray(new Class[0]));
// Result type
expressionEvaluator.setExpressionType(key.getReturnClass());
try {
// Compile
expressionEvaluator.cook(key.getExpression());
} catch (CompileException e) {
throw new InvalidProgramException(
"Expression cannot be compiled. This is a bug. Please file an issue.\nExpression: "
+ key.getExpression(),
e);
}
return expressionEvaluator;
});
} catch (Exception e) {
throw new FlinkRuntimeException(e.getMessage(), e);
}
}
}

@ -0,0 +1,97 @@
/*
* 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.transform;
import java.io.Serializable;
import java.util.List;
import java.util.Objects;
/**
* The key applies to describe the information of the transformation expression.
*
* <p>A transform expression key contains:
*
* <ul>
* <li>expression: a string for the transformation expression.
* <li>argumentNames: a list for the argument names in expression.
* <li>argumentClasses: a list for the argument classes in expression.
* <li>returnClass: a class for the return class in expression
* </ul>
*/
public class TransformExpressionKey implements Serializable {
private static final long serialVersionUID = 1L;
private final String expression;
private final List<String> argumentNames;
private final List<Class<?>> argumentClasses;
private final Class<?> returnClass;
private TransformExpressionKey(
String expression,
List<String> argumentNames,
List<Class<?>> argumentClasses,
Class<?> returnClass) {
this.expression = expression;
this.argumentNames = argumentNames;
this.argumentClasses = argumentClasses;
this.returnClass = returnClass;
}
public String getExpression() {
return expression;
}
public List<String> getArgumentNames() {
return argumentNames;
}
public List<Class<?>> getArgumentClasses() {
return argumentClasses;
}
public Class<?> getReturnClass() {
return returnClass;
}
public static TransformExpressionKey of(
String expression,
List<String> argumentNames,
List<Class<?>> argumentClasses,
Class<?> returnClass) {
return new TransformExpressionKey(expression, argumentNames, argumentClasses, returnClass);
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
TransformExpressionKey that = (TransformExpressionKey) o;
return expression.equals(that.expression)
&& argumentNames.equals(that.argumentNames)
&& argumentClasses.equals(that.argumentClasses)
&& returnClass.equals(that.returnClass);
}
@Override
public int hashCode() {
return Objects.hash(expression, argumentNames, argumentClasses, returnClass);
}
}

@ -0,0 +1,76 @@
/*
* 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.transform;
import org.apache.flink.cdc.common.utils.StringUtils;
import org.apache.flink.cdc.runtime.parser.TransformParser;
import java.io.Serializable;
import java.util.List;
import java.util.Optional;
/**
* The TransformFilter applies to describe the information of the filter row.
*
* <p>A filter row contains:
*
* <ul>
* <li>expression: a string for filter expression split from the user-defined filter.
* <li>scriptExpression: a string for filter script expression compiled from the column
* expression.
* <li>columnNames: a list for recording the name of all columns used by the filter expression.
* </ul>
*/
public class TransformFilter implements Serializable {
private static final long serialVersionUID = 1L;
private final String expression;
private final String scriptExpression;
private final List<String> columnNames;
public TransformFilter(String expression, String scriptExpression, List<String> columnNames) {
this.expression = expression;
this.scriptExpression = scriptExpression;
this.columnNames = columnNames;
}
public String getExpression() {
return expression;
}
public String getScriptExpression() {
return scriptExpression;
}
public List<String> getColumnNames() {
return columnNames;
}
public static Optional<TransformFilter> of(String filterExpression) {
if (StringUtils.isNullOrWhitespaceOnly(filterExpression)) {
return Optional.empty();
}
List<String> columnNames = TransformParser.parseFilterColumnNameList(filterExpression);
String scriptExpression =
TransformParser.translateFilterExpressionToJaninoExpression(filterExpression);
return Optional.of(new TransformFilter(filterExpression, scriptExpression, columnNames));
}
public boolean isVaild() {
return !columnNames.isEmpty();
}
}

@ -0,0 +1,148 @@
/*
* 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.transform;
import org.apache.flink.cdc.common.data.RecordData;
import org.apache.flink.cdc.common.data.binary.BinaryRecordData;
import org.apache.flink.cdc.common.schema.Column;
import org.apache.flink.cdc.runtime.parser.JaninoCompiler;
import org.apache.flink.cdc.runtime.parser.TransformParser;
import org.apache.flink.cdc.runtime.typeutils.DataTypeConverter;
import org.codehaus.janino.ExpressionEvaluator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.lang.reflect.InvocationTargetException;
import java.util.ArrayList;
import java.util.List;
/** The processor of the transform filter. It processes the data change event of matched table. */
public class TransformFilterProcessor {
private static final Logger LOG = LoggerFactory.getLogger(TransformFilterProcessor.class);
private TableInfo tableInfo;
private TransformFilter transformFilter;
private String timezone;
private TransformExpressionKey transformExpressionKey;
public TransformFilterProcessor(
TableInfo tableInfo, TransformFilter transformFilter, String timezone) {
this.tableInfo = tableInfo;
this.transformFilter = transformFilter;
this.timezone = timezone;
transformExpressionKey = generateTransformExpressionKey();
}
public static TransformFilterProcessor of(
TableInfo tableInfo, TransformFilter transformFilter, String timezone) {
return new TransformFilterProcessor(tableInfo, transformFilter, timezone);
}
public boolean process(BinaryRecordData after, long epochTime) {
ExpressionEvaluator expressionEvaluator =
TransformExpressionCompiler.compileExpression(transformExpressionKey);
try {
return (Boolean) expressionEvaluator.evaluate(generateParams(after, epochTime));
} catch (InvocationTargetException e) {
LOG.error(
"Table:{} filter:{} execute failed. {}",
tableInfo.getName(),
transformFilter.getExpression(),
e);
throw new RuntimeException(e);
}
}
private Object[] generateParams(BinaryRecordData after, long epochTime) {
List<Object> params = new ArrayList<>();
List<Column> columns = tableInfo.getSchema().getColumns();
RecordData.FieldGetter[] fieldGetters = tableInfo.getFieldGetters();
for (String columnName : transformFilter.getColumnNames()) {
if (columnName.equals(TransformParser.DEFAULT_NAMESPACE_NAME)) {
params.add(tableInfo.getNamespace());
continue;
}
if (columnName.equals(TransformParser.DEFAULT_SCHEMA_NAME)) {
params.add(tableInfo.getSchemaName());
continue;
}
if (columnName.equals(TransformParser.DEFAULT_TABLE_NAME)) {
params.add(tableInfo.getTableName());
continue;
}
for (int i = 0; i < columns.size(); i++) {
Column column = columns.get(i);
if (column.getName().equals(columnName)) {
params.add(
DataTypeConverter.convertToOriginal(
fieldGetters[i].getFieldOrNull(after), column.getType()));
break;
}
}
}
params.add(timezone);
params.add(epochTime);
return params.toArray();
}
private TransformExpressionKey generateTransformExpressionKey() {
List<String> argumentNames = new ArrayList<>();
List<Class<?>> paramTypes = new ArrayList<>();
List<Column> columns = tableInfo.getSchema().getColumns();
String scriptExpression = transformFilter.getScriptExpression();
List<String> columnNames = transformFilter.getColumnNames();
for (String columnName : columnNames) {
for (int i = 0; i < columns.size(); i++) {
Column column = columns.get(i);
if (column.getName().equals(columnName)) {
argumentNames.add(columnName);
paramTypes.add(DataTypeConverter.convertOriginalClass(column.getType()));
break;
}
}
}
if (scriptExpression.contains(TransformParser.DEFAULT_NAMESPACE_NAME)
&& !argumentNames.contains(TransformParser.DEFAULT_NAMESPACE_NAME)) {
argumentNames.add(TransformParser.DEFAULT_NAMESPACE_NAME);
paramTypes.add(String.class);
}
if (scriptExpression.contains(TransformParser.DEFAULT_SCHEMA_NAME)
&& !argumentNames.contains(TransformParser.DEFAULT_SCHEMA_NAME)) {
argumentNames.add(TransformParser.DEFAULT_SCHEMA_NAME);
paramTypes.add(String.class);
}
if (scriptExpression.contains(TransformParser.DEFAULT_TABLE_NAME)
&& !argumentNames.contains(TransformParser.DEFAULT_TABLE_NAME)) {
argumentNames.add(TransformParser.DEFAULT_TABLE_NAME);
paramTypes.add(String.class);
}
argumentNames.add(JaninoCompiler.DEFAULT_TIME_ZONE);
paramTypes.add(String.class);
argumentNames.add(JaninoCompiler.DEFAULT_EPOCH_TIME);
paramTypes.add(Long.class);
return TransformExpressionKey.of(
JaninoCompiler.loadSystemFunction(scriptExpression),
argumentNames,
paramTypes,
Boolean.class);
}
}

@ -0,0 +1,78 @@
/*
* 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.transform;
import org.apache.flink.cdc.common.schema.Column;
import org.apache.flink.cdc.common.utils.StringUtils;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
/**
* The projection of transform applies to describe a projection of filtering tables. Projection
* includes the original columns of the data table and the user-defined computed columns.
*
* <p>A transformation projection contains:
*
* <ul>
* <li>projection: a string for projecting the row of matched table as output.
* <li>projectionColumns: a list for recording all columns transformation of the projection.
* </ul>
*/
public class TransformProjection implements Serializable {
private static final long serialVersionUID = 1L;
private String projection;
private List<ProjectionColumn> projectionColumns;
public TransformProjection(String projection, List<ProjectionColumn> projectionColumns) {
this.projection = projection;
this.projectionColumns = projectionColumns;
}
public String getProjection() {
return projection;
}
public List<ProjectionColumn> getProjectionColumns() {
return projectionColumns;
}
public void setProjectionColumns(List<ProjectionColumn> projectionColumns) {
this.projectionColumns = projectionColumns;
}
public boolean isValid() {
return !StringUtils.isNullOrWhitespaceOnly(projection);
}
public static Optional<TransformProjection> of(String projection) {
if (StringUtils.isNullOrWhitespaceOnly(projection)) {
return Optional.empty();
}
return Optional.of(new TransformProjection(projection, new ArrayList<>()));
}
public List<Column> getAllColumnList() {
return projectionColumns.stream()
.map(ProjectionColumn::getColumn)
.collect(Collectors.toList());
}
}

@ -0,0 +1,186 @@
/*
* 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.transform;
import org.apache.flink.cdc.common.data.RecordData;
import org.apache.flink.cdc.common.data.binary.BinaryRecordData;
import org.apache.flink.cdc.common.event.CreateTableEvent;
import org.apache.flink.cdc.common.schema.Column;
import org.apache.flink.cdc.common.schema.Schema;
import org.apache.flink.cdc.runtime.parser.TransformParser;
import org.apache.flink.cdc.runtime.typeutils.DataTypeConverter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
/**
* The processor of transform projection applies to process a row of filtering tables.
*
* <p>A transform projection processor contains:
*
* <ul>
* <li>CreateTableEvent: add the user-defined computed columns into Schema.
* <li>SchemaChangeEvent: update the columns of TransformProjection.
* <li>DataChangeEvent: Fill data field to row in TransformSchemaOperator. Process the data column
* and the user-defined expression computed columns.
* </ul>
*/
public class TransformProjectionProcessor {
private static final Logger LOG = LoggerFactory.getLogger(TransformProjectionProcessor.class);
private TableInfo tableInfo;
private TableChangeInfo tableChangeInfo;
private TransformProjection transformProjection;
private String timezone;
private Map<String, ProjectionColumnProcessor> projectionColumnProcessorMap;
public TransformProjectionProcessor(
TableInfo tableInfo,
TableChangeInfo tableChangeInfo,
TransformProjection transformProjection,
String timezone) {
this.tableInfo = tableInfo;
this.tableChangeInfo = tableChangeInfo;
this.transformProjection = transformProjection;
this.timezone = timezone;
this.projectionColumnProcessorMap = new ConcurrentHashMap<>();
}
public boolean hasTableChangeInfo() {
return this.tableChangeInfo != null;
}
public boolean hasTableInfo() {
return this.tableInfo != null;
}
public static TransformProjectionProcessor of(
TableInfo tableInfo, TransformProjection transformProjection, String timezone) {
return new TransformProjectionProcessor(tableInfo, null, transformProjection, timezone);
}
public static TransformProjectionProcessor of(
TableChangeInfo tableChangeInfo, TransformProjection transformProjection) {
return new TransformProjectionProcessor(null, tableChangeInfo, transformProjection, null);
}
public static TransformProjectionProcessor of(TransformProjection transformProjection) {
return new TransformProjectionProcessor(null, null, transformProjection, null);
}
public CreateTableEvent processCreateTableEvent(CreateTableEvent createTableEvent) {
List<ProjectionColumn> projectionColumns =
TransformParser.generateProjectionColumns(
transformProjection.getProjection(),
createTableEvent.getSchema().getColumns());
transformProjection.setProjectionColumns(projectionColumns);
List<Column> allColumnList = transformProjection.getAllColumnList();
// add the column of projection into Schema
Schema schema = createTableEvent.getSchema().copy(allColumnList);
return new CreateTableEvent(createTableEvent.tableId(), schema);
}
public void processSchemaChangeEvent(Schema schema) {
List<ProjectionColumn> projectionColumns =
TransformParser.generateProjectionColumns(
transformProjection.getProjection(), schema.getColumns());
transformProjection.setProjectionColumns(projectionColumns);
}
public BinaryRecordData processFillDataField(BinaryRecordData data) {
List<Object> valueList = new ArrayList<>();
for (Column column : tableChangeInfo.getTransformedSchema().getColumns()) {
boolean isProjectionColumn = false;
for (ProjectionColumn projectionColumn : transformProjection.getProjectionColumns()) {
if (column.getName().equals(projectionColumn.getColumnName())
&& projectionColumn.isValidTransformedProjectionColumn()) {
valueList.add(null);
isProjectionColumn = true;
break;
}
}
if (!isProjectionColumn) {
valueList.add(
getValueFromBinaryRecordData(
column.getName(),
data,
tableChangeInfo.getOriginalSchema().getColumns(),
tableChangeInfo.getFieldGetters()));
}
}
return tableChangeInfo
.getRecordDataGenerator()
.generate(valueList.toArray(new Object[valueList.size()]));
}
public BinaryRecordData processData(BinaryRecordData after, long epochTime) {
List<Object> valueList = new ArrayList<>();
for (Column column : tableInfo.getSchema().getColumns()) {
boolean isProjectionColumn = false;
for (ProjectionColumn projectionColumn : transformProjection.getProjectionColumns()) {
if (column.getName().equals(projectionColumn.getColumnName())
&& projectionColumn.isValidTransformedProjectionColumn()) {
if (!projectionColumnProcessorMap.containsKey(
projectionColumn.getColumnName())) {
projectionColumnProcessorMap.put(
projectionColumn.getColumnName(),
ProjectionColumnProcessor.of(
tableInfo, projectionColumn, timezone));
}
ProjectionColumnProcessor projectionColumnProcessor =
projectionColumnProcessorMap.get(projectionColumn.getColumnName());
valueList.add(
DataTypeConverter.convert(
projectionColumnProcessor.evaluate(after, epochTime),
projectionColumn.getDataType()));
isProjectionColumn = true;
break;
}
}
if (!isProjectionColumn) {
valueList.add(
getValueFromBinaryRecordData(
column.getName(),
after,
tableInfo.getSchema().getColumns(),
tableInfo.getFieldGetters()));
}
}
return tableInfo
.getRecordDataGenerator()
.generate(valueList.toArray(new Object[valueList.size()]));
}
private Object getValueFromBinaryRecordData(
String columnName,
BinaryRecordData binaryRecordData,
List<Column> columns,
RecordData.FieldGetter[] fieldGetters) {
for (int i = 0; i < columns.size(); i++) {
if (columnName.equals(columns.get(i).getName())) {
return DataTypeConverter.convert(
fieldGetters[i].getFieldOrNull(binaryRecordData), columns.get(i).getType());
}
}
return null;
}
}

@ -0,0 +1,297 @@
/*
* 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.transform;
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.state.OperatorStateStore;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple5;
import org.apache.flink.cdc.common.data.binary.BinaryRecordData;
import org.apache.flink.cdc.common.event.CreateTableEvent;
import org.apache.flink.cdc.common.event.DataChangeEvent;
import org.apache.flink.cdc.common.event.Event;
import org.apache.flink.cdc.common.event.SchemaChangeEvent;
import org.apache.flink.cdc.common.event.TableId;
import org.apache.flink.cdc.common.schema.Schema;
import org.apache.flink.cdc.common.schema.Selectors;
import org.apache.flink.cdc.common.utils.SchemaUtils;
import org.apache.flink.runtime.state.StateInitializationContext;
import org.apache.flink.runtime.state.StateSnapshotContext;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.ChainingStrategy;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.ConcurrentHashMap;
import java.util.stream.Collectors;
/** A schema process function that applies user-defined transform logics. */
public class TransformSchemaOperator extends AbstractStreamOperator<Event>
implements OneInputStreamOperator<Event, Event> {
private final List<Tuple5<String, String, String, String, String>> transformRules;
private transient List<Tuple2<Selectors, Optional<TransformProjection>>> transforms;
private final Map<TableId, TableChangeInfo> tableChangeInfoMap;
private transient Map<TableId, TransformProjectionProcessor> processorMap;
private final List<Tuple2<Selectors, SchemaMetadataTransform>> schemaMetadataTransformers;
private transient ListState<byte[]> state;
public static TransformSchemaOperator.Builder newBuilder() {
return new TransformSchemaOperator.Builder();
}
/** Builder of {@link TransformSchemaOperator}. */
public static class Builder {
private final List<Tuple5<String, String, String, String, String>> transformRules =
new ArrayList<>();
public TransformSchemaOperator.Builder addTransform(
String tableInclusions,
@Nullable String projection,
String primaryKey,
String partitionKey,
String tableOption) {
transformRules.add(
Tuple5.of(tableInclusions, projection, primaryKey, partitionKey, tableOption));
return this;
}
public TransformSchemaOperator build() {
return new TransformSchemaOperator(transformRules);
}
}
private TransformSchemaOperator(
List<Tuple5<String, String, String, String, String>> transformRules) {
this.transformRules = transformRules;
this.tableChangeInfoMap = new ConcurrentHashMap<>();
this.processorMap = new ConcurrentHashMap<>();
this.schemaMetadataTransformers = new ArrayList<>();
this.chainingStrategy = ChainingStrategy.ALWAYS;
}
@Override
public void open() throws Exception {
super.open();
for (Tuple5<String, String, String, String, String> transformRule : transformRules) {
String tableInclusions = transformRule.f0;
String projection = transformRule.f1;
String primaryKeys = transformRule.f2;
String partitionKeys = transformRule.f3;
String tableOptions = transformRule.f4;
Selectors selectors =
new Selectors.SelectorsBuilder().includeTables(tableInclusions).build();
transforms = new ArrayList<>();
transforms.add(new Tuple2<>(selectors, TransformProjection.of(projection)));
schemaMetadataTransformers.add(
new Tuple2<>(
selectors,
new SchemaMetadataTransform(primaryKeys, partitionKeys, tableOptions)));
}
this.processorMap = new ConcurrentHashMap<>();
}
@Override
public void initializeState(StateInitializationContext context) throws Exception {
super.initializeState(context);
OperatorStateStore stateStore = context.getOperatorStateStore();
ListStateDescriptor<byte[]> descriptor =
new ListStateDescriptor<>("originalSchemaState", byte[].class);
state = stateStore.getUnionListState(descriptor);
if (context.isRestored()) {
for (byte[] serializedTableInfo : state.get()) {
TableChangeInfo stateTableChangeInfo =
TableChangeInfo.SERIALIZER.deserialize(
TableChangeInfo.SERIALIZER.getVersion(), serializedTableInfo);
tableChangeInfoMap.put(stateTableChangeInfo.getTableId(), stateTableChangeInfo);
}
}
}
@Override
public void snapshotState(StateSnapshotContext context) throws Exception {
super.snapshotState(context);
state.update(
new ArrayList<>(
tableChangeInfoMap.values().stream()
.map(
tableChangeInfo -> {
try {
return TableChangeInfo.SERIALIZER.serialize(
tableChangeInfo);
} catch (IOException e) {
throw new RuntimeException(e);
}
})
.collect(Collectors.toList())));
}
@Override
public void finish() throws Exception {
super.finish();
clearOperator();
}
@Override
public void close() throws Exception {
super.close();
clearOperator();
}
@Override
public void processElement(StreamRecord<Event> element) throws Exception {
Event event = element.getValue();
if (event instanceof CreateTableEvent) {
event = cacheCreateTable((CreateTableEvent) event);
output.collect(new StreamRecord<>(event));
} else if (event instanceof SchemaChangeEvent) {
event = cacheChangeSchema((SchemaChangeEvent) event);
output.collect(new StreamRecord<>(event));
} else if (event instanceof DataChangeEvent) {
output.collect(new StreamRecord<>(processDataChangeEvent(((DataChangeEvent) event))));
}
}
private SchemaChangeEvent cacheCreateTable(CreateTableEvent event) {
TableId tableId = event.tableId();
Schema originalSchema = event.getSchema();
event = transformCreateTableEvent(event);
Schema newSchema = (event).getSchema();
tableChangeInfoMap.put(tableId, TableChangeInfo.of(tableId, originalSchema, newSchema));
return event;
}
private SchemaChangeEvent cacheChangeSchema(SchemaChangeEvent event) {
TableId tableId = event.tableId();
TableChangeInfo tableChangeInfo = tableChangeInfoMap.get(tableId);
Schema originalSchema =
SchemaUtils.applySchemaChangeEvent(tableChangeInfo.getOriginalSchema(), event);
Schema newSchema =
SchemaUtils.applySchemaChangeEvent(tableChangeInfo.getTransformedSchema(), event);
tableChangeInfoMap.put(tableId, TableChangeInfo.of(tableId, originalSchema, newSchema));
return event;
}
private CreateTableEvent transformCreateTableEvent(CreateTableEvent createTableEvent) {
TableId tableId = createTableEvent.tableId();
for (Tuple2<Selectors, SchemaMetadataTransform> transform : schemaMetadataTransformers) {
Selectors selectors = transform.f0;
if (selectors.isMatch(tableId)) {
createTableEvent =
new CreateTableEvent(
tableId,
transformSchemaMetaData(
createTableEvent.getSchema(), transform.f1));
}
}
for (Tuple2<Selectors, Optional<TransformProjection>> transform : transforms) {
Selectors selectors = transform.f0;
if (selectors.isMatch(tableId) && transform.f1.isPresent()) {
TransformProjection transformProjection = transform.f1.get();
if (transformProjection.isValid()) {
if (!processorMap.containsKey(tableId)) {
processorMap.put(
tableId, TransformProjectionProcessor.of(transformProjection));
}
TransformProjectionProcessor transformProjectionProcessor =
processorMap.get(tableId);
// update the columns of projection and add the column of projection into Schema
return transformProjectionProcessor.processCreateTableEvent(createTableEvent);
}
}
}
return createTableEvent;
}
private Schema transformSchemaMetaData(
Schema schema, SchemaMetadataTransform schemaMetadataTransform) {
Schema.Builder schemaBuilder = Schema.newBuilder().setColumns(schema.getColumns());
if (!schemaMetadataTransform.getPrimaryKeys().isEmpty()) {
schemaBuilder.primaryKey(schemaMetadataTransform.getPrimaryKeys());
} else {
schemaBuilder.primaryKey(schema.primaryKeys());
}
if (!schemaMetadataTransform.getPartitionKeys().isEmpty()) {
schemaBuilder.partitionKey(schemaMetadataTransform.getPartitionKeys());
} else {
schemaBuilder.partitionKey(schema.partitionKeys());
}
if (!schemaMetadataTransform.getOptions().isEmpty()) {
schemaBuilder.options(schemaMetadataTransform.getOptions());
} else {
schemaBuilder.options(schema.options());
}
return schemaBuilder.build();
}
private DataChangeEvent processDataChangeEvent(DataChangeEvent dataChangeEvent)
throws Exception {
TableId tableId = dataChangeEvent.tableId();
for (Tuple2<Selectors, Optional<TransformProjection>> transform : transforms) {
Selectors selectors = transform.f0;
if (selectors.isMatch(tableId) && transform.f1.isPresent()) {
TransformProjection transformProjection = transform.f1.get();
if (transformProjection.isValid()) {
return processProjection(transformProjection, dataChangeEvent);
}
}
}
return dataChangeEvent;
}
private DataChangeEvent processProjection(
TransformProjection transformProjection, DataChangeEvent dataChangeEvent)
throws Exception {
TableId tableId = dataChangeEvent.tableId();
TableChangeInfo tableChangeInfo = tableChangeInfoMap.get(tableId);
if (!processorMap.containsKey(tableId) || !processorMap.get(tableId).hasTableChangeInfo()) {
processorMap.put(
tableId, TransformProjectionProcessor.of(tableChangeInfo, transformProjection));
}
TransformProjectionProcessor transformProjectionProcessor = processorMap.get(tableId);
BinaryRecordData before = (BinaryRecordData) dataChangeEvent.before();
BinaryRecordData after = (BinaryRecordData) dataChangeEvent.after();
if (before != null) {
BinaryRecordData projectedBefore =
transformProjectionProcessor.processFillDataField(before);
dataChangeEvent = DataChangeEvent.projectBefore(dataChangeEvent, projectedBefore);
}
if (after != null) {
BinaryRecordData projectedAfter =
transformProjectionProcessor.processFillDataField(after);
dataChangeEvent = DataChangeEvent.projectAfter(dataChangeEvent, projectedAfter);
}
return dataChangeEvent;
}
private void clearOperator() {
this.transforms = null;
this.processorMap = null;
this.state = null;
}
}

@ -0,0 +1,254 @@
/*
* 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.parser;
import org.apache.flink.api.common.InvalidProgramException;
import org.apache.flink.api.common.io.ParseException;
import org.apache.flink.cdc.common.utils.StringUtils;
import org.apache.calcite.sql.SqlBasicCall;
import org.apache.calcite.sql.SqlCharStringLiteral;
import org.apache.calcite.sql.SqlIdentifier;
import org.apache.calcite.sql.SqlLiteral;
import org.apache.calcite.sql.SqlNode;
import org.apache.calcite.sql.SqlNodeList;
import org.apache.calcite.sql.type.SqlTypeName;
import org.codehaus.commons.compiler.CompileException;
import org.codehaus.commons.compiler.Location;
import org.codehaus.janino.ExpressionEvaluator;
import org.codehaus.janino.Java;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
/**
* Use Janino compiler to compiler the statement of flink cdc pipeline transform into the executable
* code of Janino. For example, compiler 'string1 || string2' into 'concat(string1, string2)'. The
* core logic is to traverse SqlNode tree and transform to Atom tree. Janino documents:
* https://www.janino.net/index.html#properties
*/
public class JaninoCompiler {
private static final List<SqlTypeName> SQL_TYPE_NAME_IGNORE = Arrays.asList(SqlTypeName.SYMBOL);
private static final List<String> NO_OPERAND_TIMESTAMP_FUNCTIONS =
Arrays.asList(
"LOCALTIME",
"LOCALTIMESTAMP",
"CURRENT_TIME",
"CURRENT_DATE",
"CURRENT_TIMESTAMP");
public static final String DEFAULT_EPOCH_TIME = "__epoch_time__";
public static final String DEFAULT_TIME_ZONE = "__time_zone__";
public static String loadSystemFunction(String expression) {
return "import static org.apache.flink.cdc.runtime.functions.SystemFunctionUtils.*;"
+ expression;
}
public static ExpressionEvaluator compileExpression(
String expression,
List<String> argumentNames,
List<Class<?>> argumentClasses,
Class<?> returnClass) {
ExpressionEvaluator expressionEvaluator = new ExpressionEvaluator();
expressionEvaluator.setParameters(
argumentNames.toArray(new String[0]), argumentClasses.toArray(new Class[0]));
expressionEvaluator.setExpressionType(returnClass);
try {
expressionEvaluator.cook(expression);
return expressionEvaluator;
} catch (CompileException e) {
throw new InvalidProgramException(
"Expression cannot be compiled. This is a bug. Please file an issue.\nExpression: "
+ expression,
e);
}
}
public static String translateSqlNodeToJaninoExpression(SqlNode transform) {
if (transform instanceof SqlIdentifier) {
SqlIdentifier sqlIdentifier = (SqlIdentifier) transform;
return sqlIdentifier.names.get(sqlIdentifier.names.size() - 1);
} else if (transform instanceof SqlBasicCall) {
Java.Rvalue rvalue = translateJaninoAST((SqlBasicCall) transform);
return rvalue.toString();
}
return "";
}
private static Java.Rvalue translateJaninoAST(SqlBasicCall sqlBasicCall) {
List<SqlNode> operandList = sqlBasicCall.getOperandList();
List<Java.Rvalue> atoms = new ArrayList<>();
for (SqlNode sqlNode : operandList) {
translateSqlNodeToAtoms(sqlNode, atoms);
}
if (NO_OPERAND_TIMESTAMP_FUNCTIONS.contains(sqlBasicCall.getOperator().getName())) {
atoms.add(new Java.AmbiguousName(Location.NOWHERE, new String[] {DEFAULT_EPOCH_TIME}));
atoms.add(new Java.AmbiguousName(Location.NOWHERE, new String[] {DEFAULT_TIME_ZONE}));
}
return sqlBasicCallToJaninoRvalue(sqlBasicCall, atoms.toArray(new Java.Rvalue[0]));
}
private static void translateSqlNodeToAtoms(SqlNode sqlNode, List<Java.Rvalue> atoms) {
if (sqlNode instanceof SqlIdentifier) {
SqlIdentifier sqlIdentifier = (SqlIdentifier) sqlNode;
String columnName = sqlIdentifier.names.get(sqlIdentifier.names.size() - 1);
if (NO_OPERAND_TIMESTAMP_FUNCTIONS.contains(columnName)) {
atoms.add(generateNoOperandTimestampFunctionOperation(columnName));
} else {
atoms.add(new Java.AmbiguousName(Location.NOWHERE, new String[] {columnName}));
}
} else if (sqlNode instanceof SqlLiteral) {
SqlLiteral sqlLiteral = (SqlLiteral) sqlNode;
String value = sqlLiteral.getValue().toString();
if (sqlLiteral instanceof SqlCharStringLiteral) {
// Double quotation marks represent strings in Janino.
value = "\"" + value.substring(1, value.length() - 1) + "\"";
}
if (SQL_TYPE_NAME_IGNORE.contains(sqlLiteral.getTypeName())) {
value = "\"" + value + "\"";
}
atoms.add(new Java.AmbiguousName(Location.NOWHERE, new String[] {value}));
} else if (sqlNode instanceof SqlBasicCall) {
atoms.add(translateJaninoAST((SqlBasicCall) sqlNode));
} else if (sqlNode instanceof SqlNodeList) {
for (SqlNode node : (SqlNodeList) sqlNode) {
translateSqlNodeToAtoms(node, atoms);
}
}
}
private static Java.Rvalue sqlBasicCallToJaninoRvalue(
SqlBasicCall sqlBasicCall, Java.Rvalue[] atoms) {
switch (sqlBasicCall.getKind()) {
case AND:
return generateBinaryOperation(sqlBasicCall, atoms, "&&");
case OR:
return generateBinaryOperation(sqlBasicCall, atoms, "||");
case NOT:
return generateUnaryOperation("!", atoms[0]);
case EQUALS:
return generateEqualsOperation(sqlBasicCall, atoms);
case NOT_EQUALS:
return generateUnaryOperation("!", generateEqualsOperation(sqlBasicCall, atoms));
case IS_NULL:
return generateUnaryOperation("null == ", atoms[0]);
case IS_NOT_NULL:
return generateUnaryOperation("null != ", atoms[0]);
case IS_FALSE:
case IS_NOT_TRUE:
return generateUnaryOperation("false == ", atoms[0]);
case IS_TRUE:
case IS_NOT_FALSE:
return generateUnaryOperation("true == ", atoms[0]);
case BETWEEN:
case IN:
case NOT_IN:
case LIKE:
case CEIL:
case FLOOR:
case TRIM:
case OTHER_FUNCTION:
return generateOtherFunctionOperation(sqlBasicCall, atoms);
case PLUS:
return generateBinaryOperation(sqlBasicCall, atoms, "+");
case MINUS:
return generateBinaryOperation(sqlBasicCall, atoms, "-");
case TIMES:
return generateBinaryOperation(sqlBasicCall, atoms, "*");
case DIVIDE:
return generateBinaryOperation(sqlBasicCall, atoms, "/");
case MOD:
return generateBinaryOperation(sqlBasicCall, atoms, "%");
case LESS_THAN:
case GREATER_THAN:
case LESS_THAN_OR_EQUAL:
case GREATER_THAN_OR_EQUAL:
return generateBinaryOperation(sqlBasicCall, atoms, sqlBasicCall.getKind().sql);
case OTHER:
return generateOtherOperation(sqlBasicCall, atoms);
default:
throw new ParseException("Unrecognized expression: " + sqlBasicCall.toString());
}
}
private static Java.Rvalue generateUnaryOperation(String operator, Java.Rvalue atom) {
return new Java.UnaryOperation(Location.NOWHERE, operator, atom);
}
private static Java.Rvalue generateBinaryOperation(
SqlBasicCall sqlBasicCall, Java.Rvalue[] atoms, String operator) {
if (atoms.length != 2) {
throw new ParseException("Unrecognized expression: " + sqlBasicCall.toString());
}
return new Java.BinaryOperation(Location.NOWHERE, atoms[0], operator, atoms[1]);
}
private static Java.Rvalue generateEqualsOperation(
SqlBasicCall sqlBasicCall, Java.Rvalue[] atoms) {
if (atoms.length != 2) {
throw new ParseException("Unrecognized expression: " + sqlBasicCall.toString());
}
return new Java.MethodInvocation(
Location.NOWHERE, null, StringUtils.convertToCamelCase("VALUE_EQUALS"), atoms);
}
private static Java.Rvalue generateOtherOperation(
SqlBasicCall sqlBasicCall, Java.Rvalue[] atoms) {
if (sqlBasicCall.getOperator().getName().equals("||")) {
return new Java.MethodInvocation(
Location.NOWHERE, null, StringUtils.convertToCamelCase("CONCAT"), atoms);
}
throw new ParseException("Unrecognized expression: " + sqlBasicCall.toString());
}
private static Java.Rvalue generateOtherFunctionOperation(
SqlBasicCall sqlBasicCall, Java.Rvalue[] atoms) {
String operationName = sqlBasicCall.getOperator().getName().toUpperCase();
if (operationName.equals("IF")) {
if (atoms.length == 3) {
return new Java.ConditionalExpression(
Location.NOWHERE, atoms[0], atoms[1], atoms[2]);
} else {
throw new ParseException("Unrecognized expression: " + sqlBasicCall.toString());
}
} else if (operationName.equals("NOW")) {
return generateNoOperandTimestampFunctionOperation(operationName);
} else {
return new Java.MethodInvocation(
Location.NOWHERE,
null,
StringUtils.convertToCamelCase(sqlBasicCall.getOperator().getName()),
atoms);
}
}
private static Java.Rvalue generateNoOperandTimestampFunctionOperation(String operationName) {
List<Java.Rvalue> timestampFunctionParam = new ArrayList<>();
timestampFunctionParam.add(
new Java.AmbiguousName(Location.NOWHERE, new String[] {DEFAULT_EPOCH_TIME}));
timestampFunctionParam.add(
new Java.AmbiguousName(Location.NOWHERE, new String[] {DEFAULT_TIME_ZONE}));
return new Java.MethodInvocation(
Location.NOWHERE,
null,
StringUtils.convertToCamelCase(operationName),
timestampFunctionParam.toArray(new Java.Rvalue[0]));
}
}

@ -0,0 +1,378 @@
/*
* 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.parser;
import org.apache.flink.api.common.io.ParseException;
import org.apache.flink.cdc.common.schema.Column;
import org.apache.flink.cdc.common.types.DataTypes;
import org.apache.flink.cdc.common.utils.StringUtils;
import org.apache.flink.cdc.runtime.operators.transform.ProjectionColumn;
import org.apache.flink.cdc.runtime.parser.metadata.TransformSchemaFactory;
import org.apache.flink.cdc.runtime.parser.metadata.TransformSqlOperatorTable;
import org.apache.flink.cdc.runtime.typeutils.DataTypeConverter;
import org.apache.calcite.config.CalciteConnectionConfigImpl;
import org.apache.calcite.config.Lex;
import org.apache.calcite.jdbc.CalciteSchema;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.hep.HepPlanner;
import org.apache.calcite.plan.hep.HepProgramBuilder;
import org.apache.calcite.prepare.CalciteCatalogReader;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.RelRoot;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeField;
import org.apache.calcite.rel.type.RelDataTypeSystem;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.sql.SqlBasicCall;
import org.apache.calcite.sql.SqlIdentifier;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.SqlNode;
import org.apache.calcite.sql.SqlSelect;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.calcite.sql.parser.SqlParseException;
import org.apache.calcite.sql.parser.SqlParser;
import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
import org.apache.calcite.sql.util.SqlOperatorTables;
import org.apache.calcite.sql.validate.SqlConformanceEnum;
import org.apache.calcite.sql.validate.SqlValidator;
import org.apache.calcite.sql.validate.SqlValidatorUtil;
import org.apache.calcite.sql2rel.SqlToRelConverter;
import org.apache.calcite.sql2rel.StandardConvertletTable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Properties;
import java.util.stream.Collectors;
/** Use Flink's calcite parser to parse the statement of flink cdc pipeline transform. */
public class TransformParser {
private static final Logger LOG = LoggerFactory.getLogger(TransformParser.class);
private static final String DEFAULT_SCHEMA = "default_schema";
private static final String DEFAULT_TABLE = "TB";
public static final String DEFAULT_NAMESPACE_NAME = "__namespace_name__";
public static final String DEFAULT_SCHEMA_NAME = "__schema_name__";
public static final String DEFAULT_TABLE_NAME = "__table_name__";
private static SqlParser getCalciteParser(String sql) {
return SqlParser.create(
sql,
SqlParser.Config.DEFAULT
.withConformance(SqlConformanceEnum.MYSQL_5)
.withCaseSensitive(true)
.withLex(Lex.JAVA));
}
private static RelNode sqlToRel(List<Column> columns, SqlNode sqlNode) {
List<Column> columnsWithMetadata = copyFillMetadataColumn(sqlNode.toString(), columns);
CalciteSchema rootSchema = CalciteSchema.createRootSchema(true);
Map<String, Object> operand = new HashMap<>();
operand.put("tableName", DEFAULT_TABLE);
operand.put("columns", columnsWithMetadata);
rootSchema.add(
DEFAULT_SCHEMA,
TransformSchemaFactory.INSTANCE.create(rootSchema.plus(), DEFAULT_SCHEMA, operand));
SqlTypeFactoryImpl factory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
CalciteCatalogReader calciteCatalogReader =
new CalciteCatalogReader(
rootSchema,
rootSchema.path(DEFAULT_SCHEMA),
factory,
new CalciteConnectionConfigImpl(new Properties()));
TransformSqlOperatorTable transformSqlOperatorTable = TransformSqlOperatorTable.instance();
SqlStdOperatorTable sqlStdOperatorTable = SqlStdOperatorTable.instance();
SqlValidator validator =
SqlValidatorUtil.newValidator(
SqlOperatorTables.chain(sqlStdOperatorTable, transformSqlOperatorTable),
calciteCatalogReader,
factory,
SqlValidator.Config.DEFAULT.withIdentifierExpansion(true));
SqlNode validateSqlNode = validator.validate(sqlNode);
SqlToRelConverter sqlToRelConverter =
new SqlToRelConverter(
null,
validator,
calciteCatalogReader,
RelOptCluster.create(
new HepPlanner(new HepProgramBuilder().build()),
new RexBuilder(factory)),
StandardConvertletTable.INSTANCE,
SqlToRelConverter.config().withTrimUnusedFields(false));
RelRoot relRoot = sqlToRelConverter.convertQuery(validateSqlNode, false, true);
return relRoot.rel;
}
public static SqlSelect parseSelect(String statement) {
SqlNode sqlNode = null;
try {
sqlNode = getCalciteParser(statement).parseQuery();
} catch (SqlParseException e) {
LOG.error("Statements can not be parsed. {} \n {}", statement, e);
throw new ParseException("Statements can not be parsed.", e);
}
if (sqlNode instanceof SqlSelect) {
return (SqlSelect) sqlNode;
} else {
throw new ParseException("Only select statements can be parsed.");
}
}
// Parse all columns
public static List<ProjectionColumn> generateProjectionColumns(
String projectionExpression, List<Column> columns) {
if (StringUtils.isNullOrWhitespaceOnly(projectionExpression)) {
return new ArrayList<>();
}
SqlSelect sqlSelect = parseProjectionExpression(projectionExpression);
if (sqlSelect.getSelectList().isEmpty()) {
return new ArrayList<>();
}
RelNode relNode = sqlToRel(columns, sqlSelect);
Map<String, RelDataType> relDataTypeMap =
relNode.getRowType().getFieldList().stream()
.collect(
Collectors.toMap(
RelDataTypeField::getName, RelDataTypeField::getType));
List<ProjectionColumn> projectionColumns = new ArrayList<>();
for (SqlNode sqlNode : sqlSelect.getSelectList()) {
if (sqlNode instanceof SqlBasicCall) {
SqlBasicCall sqlBasicCall = (SqlBasicCall) sqlNode;
if (SqlKind.AS.equals(sqlBasicCall.getOperator().kind)) {
Optional<SqlNode> transformOptional = Optional.empty();
String columnName = null;
List<SqlNode> operandList = sqlBasicCall.getOperandList();
if (operandList.size() == 2) {
transformOptional = Optional.of(operandList.get(0));
SqlNode sqlNode1 = operandList.get(1);
if (sqlNode1 instanceof SqlIdentifier) {
SqlIdentifier sqlIdentifier = (SqlIdentifier) sqlNode1;
columnName = sqlIdentifier.names.get(sqlIdentifier.names.size() - 1);
}
}
if (isMetadataColumn(columnName)) {
continue;
}
ProjectionColumn projectionColumn =
transformOptional.isPresent()
? ProjectionColumn.of(
columnName,
DataTypeConverter.convertCalciteRelDataTypeToDataType(
relDataTypeMap.get(columnName)),
transformOptional.get().toString(),
JaninoCompiler.translateSqlNodeToJaninoExpression(
transformOptional.get()),
parseColumnNameList(transformOptional.get()))
: ProjectionColumn.of(
columnName,
DataTypeConverter.convertCalciteRelDataTypeToDataType(
relDataTypeMap.get(columnName)));
boolean hasReplacedDuplicateColumn = false;
for (int i = 0; i < projectionColumns.size(); i++) {
if (projectionColumns.get(i).getColumnName().equals(columnName)
&& !projectionColumns.get(i).isValidTransformedProjectionColumn()) {
hasReplacedDuplicateColumn = true;
projectionColumns.set(i, projectionColumn);
break;
}
}
if (!hasReplacedDuplicateColumn) {
projectionColumns.add(projectionColumn);
}
} else {
throw new ParseException("Unrecognized projection: " + sqlBasicCall.toString());
}
} else if (sqlNode instanceof SqlIdentifier) {
SqlIdentifier sqlIdentifier = (SqlIdentifier) sqlNode;
String columnName = sqlIdentifier.names.get(sqlIdentifier.names.size() - 1);
if (isMetadataColumn(columnName)) {
projectionColumns.add(
ProjectionColumn.of(
columnName,
DataTypeConverter.convertCalciteRelDataTypeToDataType(
relDataTypeMap.get(columnName)),
columnName,
columnName,
Arrays.asList(columnName)));
} else {
projectionColumns.add(
ProjectionColumn.of(
columnName,
DataTypeConverter.convertCalciteRelDataTypeToDataType(
relDataTypeMap.get(columnName))));
}
} else {
throw new ParseException("Unrecognized projection: " + sqlNode.toString());
}
}
return projectionColumns;
}
public static String translateFilterExpressionToJaninoExpression(String filterExpression) {
if (StringUtils.isNullOrWhitespaceOnly(filterExpression)) {
return "";
}
SqlSelect sqlSelect = TransformParser.parseFilterExpression(filterExpression);
if (!sqlSelect.hasWhere()) {
return "";
}
SqlNode where = sqlSelect.getWhere();
if (!(where instanceof SqlBasicCall)) {
throw new ParseException("Unrecognized where: " + where.toString());
}
return JaninoCompiler.translateSqlNodeToJaninoExpression((SqlBasicCall) where);
}
public static List<String> parseComputedColumnNames(String projection) {
List<String> columnNames = new ArrayList<>();
if (StringUtils.isNullOrWhitespaceOnly(projection)) {
return columnNames;
}
SqlSelect sqlSelect = parseProjectionExpression(projection);
if (sqlSelect.getSelectList().isEmpty()) {
return columnNames;
}
for (SqlNode sqlNode : sqlSelect.getSelectList()) {
if (sqlNode instanceof SqlBasicCall) {
SqlBasicCall sqlBasicCall = (SqlBasicCall) sqlNode;
if (SqlKind.AS.equals(sqlBasicCall.getOperator().kind)) {
String columnName = null;
List<SqlNode> operandList = sqlBasicCall.getOperandList();
for (SqlNode operand : operandList) {
if (operand instanceof SqlIdentifier) {
SqlIdentifier sqlIdentifier = (SqlIdentifier) operand;
columnName = sqlIdentifier.names.get(sqlIdentifier.names.size() - 1);
}
}
if (columnNames.contains(columnName)) {
throw new ParseException("Duplicate column definitions: " + columnName);
}
columnNames.add(columnName);
} else {
throw new ParseException("Unrecognized projection: " + sqlBasicCall.toString());
}
} else if (sqlNode instanceof SqlIdentifier) {
String columnName = sqlNode.toString();
if (isMetadataColumn(columnName) && !columnNames.contains(columnName)) {
columnNames.add(columnName);
} else {
continue;
}
} else {
throw new ParseException("Unrecognized projection: " + sqlNode.toString());
}
}
return columnNames;
}
public static List<String> parseFilterColumnNameList(String filterExpression) {
if (StringUtils.isNullOrWhitespaceOnly(filterExpression)) {
return new ArrayList<>();
}
SqlSelect sqlSelect = parseFilterExpression(filterExpression);
if (!sqlSelect.hasWhere()) {
return new ArrayList<>();
}
SqlNode where = sqlSelect.getWhere();
if (!(where instanceof SqlBasicCall)) {
throw new ParseException("Unrecognized where: " + where.toString());
}
SqlBasicCall sqlBasicCall = (SqlBasicCall) where;
return parseColumnNameList(sqlBasicCall);
}
private static List<String> parseColumnNameList(SqlNode sqlNode) {
List<String> columnNameList = new ArrayList<>();
if (sqlNode instanceof SqlIdentifier) {
SqlIdentifier sqlIdentifier = (SqlIdentifier) sqlNode;
String columnName = sqlIdentifier.names.get(sqlIdentifier.names.size() - 1);
columnNameList.add(columnName);
} else if (sqlNode instanceof SqlBasicCall) {
SqlBasicCall sqlBasicCall = (SqlBasicCall) sqlNode;
findSqlIdentifier(sqlBasicCall.getOperandList(), columnNameList);
}
return columnNameList;
}
private static void findSqlIdentifier(List<SqlNode> sqlNodes, List<String> columnNameList) {
for (SqlNode sqlNode : sqlNodes) {
if (sqlNode instanceof SqlIdentifier) {
SqlIdentifier sqlIdentifier = (SqlIdentifier) sqlNode;
String columnName = sqlIdentifier.names.get(sqlIdentifier.names.size() - 1);
columnNameList.add(columnName);
} else if (sqlNode instanceof SqlBasicCall) {
SqlBasicCall sqlBasicCall = (SqlBasicCall) sqlNode;
findSqlIdentifier(sqlBasicCall.getOperandList(), columnNameList);
}
}
}
private static SqlSelect parseProjectionExpression(String projection) {
StringBuilder statement = new StringBuilder();
statement.append("SELECT ");
statement.append(projection);
statement.append(" FROM ");
statement.append(DEFAULT_TABLE);
return parseSelect(statement.toString());
}
private static List<Column> copyFillMetadataColumn(
String transformStatement, List<Column> columns) {
List<Column> columnsWithMetadata = new ArrayList<>(columns);
if (transformStatement.contains(DEFAULT_NAMESPACE_NAME)
&& !containsMetadataColumn(columnsWithMetadata, DEFAULT_NAMESPACE_NAME)) {
columnsWithMetadata.add(
Column.physicalColumn(DEFAULT_NAMESPACE_NAME, DataTypes.STRING()));
}
if (transformStatement.contains(DEFAULT_SCHEMA_NAME)
&& !containsMetadataColumn(columnsWithMetadata, DEFAULT_SCHEMA_NAME)) {
columnsWithMetadata.add(Column.physicalColumn(DEFAULT_SCHEMA_NAME, DataTypes.STRING()));
}
if (transformStatement.contains(DEFAULT_TABLE_NAME)
&& !containsMetadataColumn(columnsWithMetadata, DEFAULT_TABLE_NAME)) {
columnsWithMetadata.add(Column.physicalColumn(DEFAULT_TABLE_NAME, DataTypes.STRING()));
}
return columnsWithMetadata;
}
private static boolean containsMetadataColumn(List<Column> columns, String columnName) {
return columns.stream().anyMatch(column -> column.getName().equals(columnName));
}
private static boolean isMetadataColumn(String columnName) {
return DEFAULT_TABLE_NAME.equals(columnName)
|| DEFAULT_SCHEMA_NAME.equals(columnName)
|| DEFAULT_NAMESPACE_NAME.equals(columnName);
}
public static SqlSelect parseFilterExpression(String filterExpression) {
StringBuilder statement = new StringBuilder();
statement.append("SELECT * FROM ");
statement.append(DEFAULT_TABLE);
if (!StringUtils.isNullOrWhitespaceOnly(filterExpression)) {
statement.append(" WHERE ");
statement.append(filterExpression);
}
return parseSelect(statement.toString());
}
}

@ -0,0 +1,91 @@
/*
* 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.parser.metadata;
import org.apache.calcite.sql.SqlCallBinding;
import org.apache.calcite.sql.SqlOperandCountRange;
import org.apache.calcite.sql.SqlOperator;
import org.apache.calcite.sql.SqlUtil;
import org.apache.calcite.sql.type.SqlOperandCountRanges;
import org.apache.calcite.sql.type.SqlOperandTypeChecker;
import org.apache.calcite.sql.type.SqlTypeUtil;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
/**
* Parameter type-checking strategy where all operand types except first one must be numeric type.
*/
public class TransformNumericExceptFirstOperandChecker implements SqlOperandTypeChecker {
private int nOperands;
public TransformNumericExceptFirstOperandChecker(int nOperands) {
this.nOperands = nOperands;
}
@Override
public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) {
for (int i = 1; i < callBinding.getOperandCount(); i++) {
if (!SqlTypeUtil.isNumeric(callBinding.getOperandType(i))) {
if (!throwOnFailure) {
return false;
}
throw callBinding.newValidationSignatureError();
}
}
return true;
}
@Override
public SqlOperandCountRange getOperandCountRange() {
if (nOperands == -1) {
return SqlOperandCountRanges.any();
} else {
return SqlOperandCountRanges.of(nOperands);
}
}
@Override
public String getAllowedSignatures(SqlOperator op, String opName) {
final String anyType = "ANY_TYPE";
final String numericType = "NUMERIC_TYPE";
if (nOperands == -1) {
return SqlUtil.getAliasedSignature(
op, opName, Arrays.asList(anyType, numericType, "..."));
} else {
List<String> types = new ArrayList<>();
types.add(anyType);
types.addAll(Collections.nCopies(nOperands - 1, numericType));
return SqlUtil.getAliasedSignature(op, opName, types);
}
}
@Override
public Consistency getConsistency() {
return Consistency.NONE;
}
@Override
public boolean isOptional(int i) {
return false;
}
}

@ -0,0 +1,42 @@
/*
* 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.parser.metadata;
import org.apache.calcite.schema.Table;
import org.apache.calcite.schema.impl.AbstractSchema;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/** TransformSchema to generate the metadata of calcite. */
public class TransformSchema extends AbstractSchema {
private String name;
private List<TransformTable> tables;
public TransformSchema(String name, List<TransformTable> tables) {
this.name = name;
this.tables = tables;
}
@Override
public Map<String, Table> getTableMap() {
return tables.stream().collect(Collectors.toMap(TransformTable::getName, t -> t));
}
}

@ -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.runtime.parser.metadata;
import org.apache.flink.cdc.common.schema.Column;
import org.apache.flink.cdc.common.utils.StringUtils;
import org.apache.calcite.schema.Schema;
import org.apache.calcite.schema.SchemaFactory;
import org.apache.calcite.schema.SchemaPlus;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
/** TransformSchemaFactory to generate the metadata of calcite. */
public class TransformSchemaFactory implements SchemaFactory {
public static final TransformSchemaFactory INSTANCE = new TransformSchemaFactory();
private TransformSchemaFactory() {}
@Override
public Schema create(SchemaPlus schemaPlus, String schemaName, Map<String, Object> operand) {
if (StringUtils.isNullOrWhitespaceOnly(schemaName)) {
schemaName = "default_schema";
}
String tableName = String.valueOf(operand.get("tableName"));
List<Column> columns = (List<Column>) operand.get("columns");
return new TransformSchema(
schemaName, Arrays.asList(new TransformTable(tableName, columns)));
}
}

@ -0,0 +1,249 @@
/*
* 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.parser.metadata;
import org.apache.flink.cdc.runtime.functions.BuiltInScalarFunction;
import org.apache.flink.cdc.runtime.functions.BuiltInTimestampFunction;
import org.apache.calcite.sql.SqlFunction;
import org.apache.calcite.sql.SqlFunctionCategory;
import org.apache.calcite.sql.SqlIdentifier;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.SqlOperator;
import org.apache.calcite.sql.SqlSyntax;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.calcite.sql.type.InferTypes;
import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlOperandCountRanges;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.calcite.sql.type.SqlTypeTransforms;
import org.apache.calcite.sql.util.ReflectiveSqlOperatorTable;
import org.apache.calcite.sql.validate.SqlNameMatcher;
import org.apache.calcite.sql.validate.SqlNameMatchers;
import javax.annotation.Nullable;
import java.util.List;
/** TransformSqlOperatorTable to generate the metadata of calcite. */
public class TransformSqlOperatorTable extends ReflectiveSqlOperatorTable {
private static TransformSqlOperatorTable instance;
private TransformSqlOperatorTable() {}
public static synchronized TransformSqlOperatorTable instance() {
if (instance == null) {
instance = new TransformSqlOperatorTable();
instance.init();
}
return instance;
}
@Override
public void lookupOperatorOverloads(
SqlIdentifier opName,
@Nullable SqlFunctionCategory sqlFunctionCategory,
SqlSyntax syntax,
List<SqlOperator> operatorList,
SqlNameMatcher nameMatcher) {
// set caseSensitive=false to make sure the behavior is same with before.
super.lookupOperatorOverloads(
opName,
sqlFunctionCategory,
syntax,
operatorList,
SqlNameMatchers.withCaseSensitive(false));
}
public static final SqlFunction CONCAT_FUNCTION =
BuiltInScalarFunction.newBuilder()
.name("CONCAT")
.returnType(
ReturnTypes.cascade(
ReturnTypes.explicit(SqlTypeName.VARCHAR),
SqlTypeTransforms.TO_NULLABLE))
.operandTypeChecker(
OperandTypes.repeat(SqlOperandCountRanges.from(1), OperandTypes.STRING))
.build();
public static final SqlFunction LOCALTIMESTAMP =
new BuiltInTimestampFunction("LOCALTIMESTAMP", SqlTypeName.TIMESTAMP, 3);
public static final SqlFunction CURRENT_TIMESTAMP =
new BuiltInTimestampFunction(
"CURRENT_TIMESTAMP", SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE, 3);
public static final SqlFunction CURRENT_DATE =
new BuiltInTimestampFunction("CURRENT_DATE", SqlTypeName.DATE, 0);
public static final SqlFunction NOW =
new BuiltInTimestampFunction("NOW", SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE, 3) {
@Override
public SqlSyntax getSyntax() {
return SqlSyntax.FUNCTION;
}
};
public static final SqlFunction TO_DATE =
new SqlFunction(
"TO_DATE",
SqlKind.OTHER_FUNCTION,
ReturnTypes.cascade(
ReturnTypes.explicit(SqlTypeName.DATE),
SqlTypeTransforms.FORCE_NULLABLE),
null,
OperandTypes.or(
OperandTypes.family(SqlTypeFamily.STRING),
OperandTypes.family(SqlTypeFamily.STRING, SqlTypeFamily.STRING)),
SqlFunctionCategory.TIMEDATE);
public static final SqlFunction TO_TIMESTAMP =
new SqlFunction(
"TO_TIMESTAMP",
SqlKind.OTHER_FUNCTION,
ReturnTypes.cascade(
ReturnTypes.explicit(SqlTypeName.TIMESTAMP, 3),
SqlTypeTransforms.FORCE_NULLABLE),
null,
OperandTypes.or(
OperandTypes.family(SqlTypeFamily.CHARACTER),
OperandTypes.family(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER)),
SqlFunctionCategory.TIMEDATE);
public static final SqlFunction TIMESTAMP_DIFF =
new SqlFunction(
"TIMESTAMP_DIFF",
SqlKind.OTHER_FUNCTION,
ReturnTypes.cascade(
ReturnTypes.explicit(SqlTypeName.INTEGER),
SqlTypeTransforms.FORCE_NULLABLE),
null,
OperandTypes.family(
SqlTypeFamily.ANY, SqlTypeFamily.TIMESTAMP, SqlTypeFamily.TIMESTAMP),
SqlFunctionCategory.TIMEDATE);
public static final SqlFunction REGEXP_REPLACE =
new SqlFunction(
"REGEXP_REPLACE",
SqlKind.OTHER_FUNCTION,
ReturnTypes.cascade(
ReturnTypes.explicit(SqlTypeName.VARCHAR),
SqlTypeTransforms.TO_NULLABLE),
null,
OperandTypes.family(
SqlTypeFamily.STRING, SqlTypeFamily.STRING, SqlTypeFamily.STRING),
SqlFunctionCategory.STRING);
public static final SqlFunction SUBSTR =
new SqlFunction(
"SUBSTR",
SqlKind.OTHER_FUNCTION,
TransformSqlReturnTypes.ARG0_VARCHAR_FORCE_NULLABLE,
null,
OperandTypes.or(
OperandTypes.family(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER),
OperandTypes.family(
SqlTypeFamily.CHARACTER,
SqlTypeFamily.INTEGER,
SqlTypeFamily.INTEGER)),
SqlFunctionCategory.STRING);
public static final SqlFunction ROUND =
new SqlFunction(
"ROUND",
SqlKind.OTHER_FUNCTION,
TransformSqlReturnTypes.ROUND_FUNCTION_NULLABLE,
null,
OperandTypes.or(OperandTypes.NUMERIC_INTEGER, OperandTypes.NUMERIC),
SqlFunctionCategory.NUMERIC);
public static final SqlFunction UUID =
BuiltInScalarFunction.newBuilder()
.name("UUID")
.returnType(ReturnTypes.explicit(SqlTypeName.CHAR, 36))
.operandTypeChecker(OperandTypes.NILADIC)
.notDeterministic()
.build();
public static final SqlFunction MOD = SqlStdOperatorTable.MOD;
public static final SqlFunction LOCALTIME = SqlStdOperatorTable.LOCALTIME;
public static final SqlFunction YEAR = SqlStdOperatorTable.YEAR;
public static final SqlFunction QUARTER = SqlStdOperatorTable.QUARTER;
public static final SqlFunction MONTH = SqlStdOperatorTable.MONTH;
public static final SqlFunction WEEK = SqlStdOperatorTable.WEEK;
public static final SqlFunction TIMESTAMP_ADD = SqlStdOperatorTable.TIMESTAMP_ADD;
public static final SqlOperator BETWEEN = SqlStdOperatorTable.BETWEEN;
public static final SqlOperator SYMMETRIC_BETWEEN = SqlStdOperatorTable.SYMMETRIC_BETWEEN;
public static final SqlOperator NOT_BETWEEN = SqlStdOperatorTable.NOT_BETWEEN;
public static final SqlOperator IN = SqlStdOperatorTable.IN;
public static final SqlOperator NOT_IN = SqlStdOperatorTable.NOT_IN;
public static final SqlFunction CHAR_LENGTH = SqlStdOperatorTable.CHAR_LENGTH;
public static final SqlFunction TRIM = SqlStdOperatorTable.TRIM;
public static final SqlOperator NOT_LIKE = SqlStdOperatorTable.NOT_LIKE;
public static final SqlOperator LIKE = SqlStdOperatorTable.LIKE;
public static final SqlFunction UPPER = SqlStdOperatorTable.UPPER;
public static final SqlFunction LOWER = SqlStdOperatorTable.LOWER;
public static final SqlFunction ABS = SqlStdOperatorTable.ABS;
public static final SqlFunction IF =
new SqlFunction(
"IF",
SqlKind.OTHER_FUNCTION,
TransformSqlReturnTypes.NUMERIC_FROM_ARG1_DEFAULT1_NULLABLE,
null,
OperandTypes.or(
OperandTypes.and(
// cannot only use `family(BOOLEAN, NUMERIC, NUMERIC)` here,
// as we don't want non-numeric types to be implicitly casted to
// numeric types.
new TransformNumericExceptFirstOperandChecker(3),
OperandTypes.family(
SqlTypeFamily.BOOLEAN,
SqlTypeFamily.NUMERIC,
SqlTypeFamily.NUMERIC)),
// used for a more explicit exception message
OperandTypes.family(
SqlTypeFamily.BOOLEAN,
SqlTypeFamily.STRING,
SqlTypeFamily.STRING),
OperandTypes.family(
SqlTypeFamily.BOOLEAN,
SqlTypeFamily.BOOLEAN,
SqlTypeFamily.BOOLEAN),
OperandTypes.family(
SqlTypeFamily.BOOLEAN,
SqlTypeFamily.CHARACTER,
SqlTypeFamily.CHARACTER),
OperandTypes.family(
SqlTypeFamily.BOOLEAN,
SqlTypeFamily.BINARY,
SqlTypeFamily.BINARY),
OperandTypes.family(
SqlTypeFamily.BOOLEAN, SqlTypeFamily.DATE, SqlTypeFamily.DATE),
OperandTypes.family(
SqlTypeFamily.BOOLEAN,
SqlTypeFamily.TIMESTAMP,
SqlTypeFamily.TIMESTAMP),
OperandTypes.family(
SqlTypeFamily.BOOLEAN, SqlTypeFamily.TIME, SqlTypeFamily.TIME)),
SqlFunctionCategory.NUMERIC);
public static final SqlFunction NULLIF = SqlStdOperatorTable.NULLIF;
public static final SqlFunction FLOOR = SqlStdOperatorTable.FLOOR;
public static final SqlFunction CEIL = SqlStdOperatorTable.CEIL;
public static final SqlFunction DATE_FORMAT =
new SqlFunction(
"DATE_FORMAT",
SqlKind.OTHER_FUNCTION,
TransformSqlReturnTypes.VARCHAR_FORCE_NULLABLE,
InferTypes.RETURN_TYPE,
OperandTypes.or(
OperandTypes.family(SqlTypeFamily.TIMESTAMP, SqlTypeFamily.STRING),
OperandTypes.family(SqlTypeFamily.STRING, SqlTypeFamily.STRING)),
SqlFunctionCategory.TIMEDATE);
}

@ -0,0 +1,191 @@
/*
* 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.parser.metadata;
import org.apache.flink.table.types.logical.DecimalType;
import org.apache.flink.table.types.logical.utils.LogicalTypeMerging;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.sql.SqlOperatorBinding;
import org.apache.calcite.sql.type.OrdinalReturnTypeInference;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlReturnTypeInference;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.calcite.sql.type.SqlTypeTransforms;
import org.apache.calcite.sql.type.SqlTypeUtil;
import java.math.BigDecimal;
import java.util.ArrayList;
import java.util.List;
/**
* This is the return type definition of the call in the Transform expression, which references the
* class of FlinkReturnTypes in the Flink. In addition, its difference is that it is specifically
* referenced for FlinkCDC's Transform, and there may be new extensions here in the future.
*/
public class TransformSqlReturnTypes {
/** ROUND(num [,len]) type inference. */
public static final SqlReturnTypeInference ROUND_FUNCTION =
new SqlReturnTypeInference() {
@Override
public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
final RelDataType numType = opBinding.getOperandType(0);
if (numType.getSqlTypeName() != SqlTypeName.DECIMAL) {
return numType;
}
final BigDecimal lenVal;
if (opBinding.getOperandCount() == 1) {
lenVal = BigDecimal.ZERO;
} else if (opBinding.getOperandCount() == 2) {
lenVal = getArg1Literal(opBinding); // may return null
} else {
throw new AssertionError();
}
if (lenVal == null) {
return numType; //
}
// ROUND( decimal(p,s), r )
final int p = numType.getPrecision();
final int s = numType.getScale();
final int r = lenVal.intValueExact();
DecimalType dt = LogicalTypeMerging.findRoundDecimalType(p, s, r);
return opBinding
.getTypeFactory()
.createSqlType(SqlTypeName.DECIMAL, dt.getPrecision(), dt.getScale());
}
private BigDecimal getArg1Literal(SqlOperatorBinding opBinding) {
try {
return opBinding.getOperandLiteralValue(1, BigDecimal.class);
} catch (Throwable e) {
return null;
}
}
};
/**
* Type-inference strategy whereby the result type of a call is the type of the operand #0
* (0-based), with nulls always allowed.
*/
public static final SqlReturnTypeInference ARG0_VARCHAR_FORCE_NULLABLE =
new OrdinalReturnTypeInference(0) {
@Override
public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
RelDataType type = super.inferReturnType(opBinding);
RelDataType newType;
switch (type.getSqlTypeName()) {
case CHAR:
newType =
opBinding
.getTypeFactory()
.createSqlType(
SqlTypeName.VARCHAR, type.getPrecision());
break;
case VARCHAR:
newType = type;
break;
default:
throw new UnsupportedOperationException("Unsupported type: " + type);
}
return opBinding.getTypeFactory().createTypeWithNullability(newType, true);
}
};
public static final SqlReturnTypeInference VARCHAR_FORCE_NULLABLE =
ReturnTypes.cascade(
ReturnTypes.explicit(SqlTypeName.VARCHAR), SqlTypeTransforms.FORCE_NULLABLE);
public static final SqlReturnTypeInference VARCHAR_NOT_NULL =
ReturnTypes.cascade(
ReturnTypes.explicit(SqlTypeName.VARCHAR), SqlTypeTransforms.TO_NOT_NULLABLE);
public static final SqlReturnTypeInference ROUND_FUNCTION_NULLABLE =
ReturnTypes.cascade(ROUND_FUNCTION, SqlTypeTransforms.TO_NULLABLE);
/**
* Determine the return type of IF functions with arguments that has the least restrictive (eg:
* numeric, character, binary). The return type is the type of the argument with the largest
* range. We start to consider the arguments from the first one. If one of the arguments is not
* of the type that has the least restrictive (eg: numeric, character, binary), we return the
* type of the first argument instead.
*/
public static final SqlReturnTypeInference IF_NULLABLE =
ReturnTypes.cascade(
new SqlReturnTypeInference() {
@Override
public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
int nOperands = opBinding.getOperandCount();
List<RelDataType> types = new ArrayList<>();
for (int i = 1; i < nOperands; i++) {
RelDataType type = opBinding.getOperandType(i);
// the RelDataTypeFactory.leastRestrictive() will check that all
// types are identical.
if (SqlTypeUtil.isNumeric(type)
|| SqlTypeUtil.isCharacter(type)
|| SqlTypeUtil.isBinary(type)) {
types.add(type);
} else {
return opBinding.getOperandType(1);
}
}
return opBinding.getTypeFactory().leastRestrictive(types);
}
},
SqlTypeTransforms.TO_NULLABLE);
public static final SqlReturnTypeInference NUMERIC_FROM_ARG1_DEFAULT1 =
new NumericOrDefaultReturnTypeInference(1, 1);
public static final SqlReturnTypeInference NUMERIC_FROM_ARG1_DEFAULT1_NULLABLE =
ReturnTypes.cascade(NUMERIC_FROM_ARG1_DEFAULT1, SqlTypeTransforms.TO_NULLABLE);
private static class NumericOrDefaultReturnTypeInference implements SqlReturnTypeInference {
// Default argument whose type is returned
// when one of the arguments from the `startTypeIdx`-th isn't of numeric type.
private int defaultTypeIdx;
// We check from the `startTypeIdx`-th argument that
// if all the following arguments are of numeric type.
// Previous arguments are ignored.
private int startTypeIdx;
public NumericOrDefaultReturnTypeInference(int defaultTypeIdx) {
this(defaultTypeIdx, 0);
}
public NumericOrDefaultReturnTypeInference(int defaultTypeIdx, int startTypeIdx) {
this.defaultTypeIdx = defaultTypeIdx;
this.startTypeIdx = startTypeIdx;
}
@Override
public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
int nOperands = opBinding.getOperandCount();
List<RelDataType> types = new ArrayList<>();
for (int i = startTypeIdx; i < nOperands; i++) {
RelDataType type = opBinding.getOperandType(i);
if (SqlTypeUtil.isNumeric(type)) {
types.add(type);
} else {
return opBinding.getOperandType(defaultTypeIdx);
}
}
return opBinding.getTypeFactory().leastRestrictive(types);
}
}
}

@ -0,0 +1,60 @@
/*
* 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.parser.metadata;
import org.apache.flink.cdc.common.schema.Column;
import org.apache.flink.cdc.runtime.typeutils.DataTypeConverter;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.schema.impl.AbstractTable;
import org.apache.calcite.util.Pair;
import java.util.ArrayList;
import java.util.List;
/** TransformTable to generate the metadata of calcite. */
public class TransformTable extends AbstractTable {
private String name;
private List<Column> columns;
public TransformTable(String name, List<Column> columns) {
this.name = name;
this.columns = columns;
}
public String getName() {
return name;
}
@Override
public RelDataType getRowType(RelDataTypeFactory relDataTypeFactory) {
List<String> names = new ArrayList<>();
List<RelDataType> types = new ArrayList<>();
for (Column column : columns) {
names.add(column.getName());
RelDataType sqlType =
relDataTypeFactory.createSqlType(
DataTypeConverter.convertCalciteType(column.getType()));
types.add(sqlType);
}
return relDataTypeFactory.createStructType(Pair.zip(names, types));
}
}

@ -43,6 +43,8 @@ public class SchemaSerializer extends TypeSerializerSingleton<Schema> {
new ListSerializer<>(ColumnSerializer.INSTANCE);
private final ListSerializer<String> primaryKeysSerializer =
new ListSerializer<>(StringSerializer.INSTANCE);
private final ListSerializer<String> partitionKeysSerializer =
new ListSerializer<>(StringSerializer.INSTANCE);
private final MapSerializer<String, String> optionsSerializer =
new MapSerializer<>(StringSerializer.INSTANCE, StringSerializer.INSTANCE);
private final StringSerializer stringSerializer = StringSerializer.INSTANCE;
@ -62,6 +64,7 @@ public class SchemaSerializer extends TypeSerializerSingleton<Schema> {
return Schema.newBuilder()
.setColumns(columnsSerializer.copy(from.getColumns()))
.primaryKey(primaryKeysSerializer.copy(from.primaryKeys()))
.partitionKey(partitionKeysSerializer.copy(from.partitionKeys()))
.options(optionsSerializer.copy(from.options()))
.comment(stringSerializer.copy(from.comment()))
.build();
@ -81,6 +84,7 @@ public class SchemaSerializer extends TypeSerializerSingleton<Schema> {
public void serialize(Schema record, DataOutputView target) throws IOException {
columnsSerializer.serialize(record.getColumns(), target);
primaryKeysSerializer.serialize(record.primaryKeys(), target);
partitionKeysSerializer.serialize(record.partitionKeys(), target);
optionsSerializer.serialize(record.options(), target);
stringSerializer.serialize(record.comment(), target);
}
@ -90,6 +94,7 @@ public class SchemaSerializer extends TypeSerializerSingleton<Schema> {
return Schema.newBuilder()
.setColumns(columnsSerializer.deserialize(source))
.primaryKey(primaryKeysSerializer.deserialize(source))
.partitionKey(partitionKeysSerializer.deserialize(source))
.options(optionsSerializer.deserialize(source))
.comment(stringSerializer.deserialize(source))
.build();

@ -0,0 +1,508 @@
/*
* 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.typeutils;
import org.apache.flink.cdc.common.data.DecimalData;
import org.apache.flink.cdc.common.data.LocalZonedTimestampData;
import org.apache.flink.cdc.common.data.TimestampData;
import org.apache.flink.cdc.common.data.binary.BinaryStringData;
import org.apache.flink.cdc.common.schema.Column;
import org.apache.flink.cdc.common.types.BinaryType;
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.common.types.VarBinaryType;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.sql.type.SqlTypeName;
import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.sql.Timestamp;
import java.time.Duration;
import java.time.Instant;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.LocalTime;
import java.util.List;
import java.util.concurrent.TimeUnit;
/** A data type converter. */
public class DataTypeConverter {
static final long MILLISECONDS_PER_SECOND = TimeUnit.SECONDS.toMillis(1);
static final long NANOSECONDS_PER_MILLISECOND = TimeUnit.MILLISECONDS.toNanos(1);
static final long NANOSECONDS_PER_DAY = TimeUnit.DAYS.toNanos(1);
public static RowType toRowType(List<Column> columnList) {
DataType[] dataTypes = columnList.stream().map(Column::getType).toArray(DataType[]::new);
String[] columnNames = columnList.stream().map(Column::getName).toArray(String[]::new);
return RowType.of(dataTypes, columnNames);
}
public static Class<?> convertOriginalClass(DataType dataType) {
switch (dataType.getTypeRoot()) {
case BOOLEAN:
return Boolean.class;
case TINYINT:
return Byte.class;
case SMALLINT:
return Short.class;
case INTEGER:
return Integer.class;
case BIGINT:
return Long.class;
case DATE:
return Integer.class;
case TIME_WITHOUT_TIME_ZONE:
return Integer.class;
case TIMESTAMP_WITHOUT_TIME_ZONE:
return TimestampData.class;
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
return LocalZonedTimestampData.class;
case FLOAT:
return Float.class;
case DOUBLE:
return Double.class;
case CHAR:
case VARCHAR:
return String.class;
case BINARY:
case VARBINARY:
return byte[].class;
case DECIMAL:
return BigDecimal.class;
case ROW:
return Object.class;
case ARRAY:
case MAP:
default:
throw new UnsupportedOperationException("Unsupported type: " + dataType);
}
}
public static SqlTypeName convertCalciteType(DataType dataType) {
switch (dataType.getTypeRoot()) {
case BOOLEAN:
return SqlTypeName.BOOLEAN;
case TINYINT:
return SqlTypeName.TINYINT;
case SMALLINT:
return SqlTypeName.SMALLINT;
case INTEGER:
return SqlTypeName.INTEGER;
case BIGINT:
return SqlTypeName.BIGINT;
case DATE:
return SqlTypeName.DATE;
case TIME_WITHOUT_TIME_ZONE:
return SqlTypeName.TIME_WITH_LOCAL_TIME_ZONE;
case TIMESTAMP_WITHOUT_TIME_ZONE:
return SqlTypeName.TIMESTAMP;
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
return SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE;
case FLOAT:
return SqlTypeName.FLOAT;
case DOUBLE:
return SqlTypeName.DOUBLE;
case CHAR:
return SqlTypeName.CHAR;
case VARCHAR:
return SqlTypeName.VARCHAR;
case BINARY:
return SqlTypeName.BINARY;
case VARBINARY:
return SqlTypeName.VARBINARY;
case DECIMAL:
return SqlTypeName.DECIMAL;
case ROW:
return SqlTypeName.ROW;
case ARRAY:
case MAP:
default:
throw new UnsupportedOperationException("Unsupported type: " + dataType);
}
}
public static DataType convertCalciteRelDataTypeToDataType(RelDataType relDataType) {
switch (relDataType.getSqlTypeName()) {
case BOOLEAN:
return DataTypes.BOOLEAN();
case TINYINT:
return DataTypes.TINYINT();
case SMALLINT:
return DataTypes.SMALLINT();
case INTEGER:
return DataTypes.INT();
case BIGINT:
return DataTypes.BIGINT();
case DATE:
return DataTypes.DATE();
case TIME:
case TIME_WITH_LOCAL_TIME_ZONE:
return DataTypes.TIME(relDataType.getPrecision());
case TIMESTAMP:
return DataTypes.TIMESTAMP(relDataType.getPrecision());
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
return DataTypes.TIMESTAMP_LTZ(relDataType.getPrecision());
case FLOAT:
return DataTypes.FLOAT();
case DOUBLE:
return DataTypes.DOUBLE();
case CHAR:
case VARCHAR:
return DataTypes.STRING();
case BINARY:
return DataTypes.BINARY(BinaryType.MAX_LENGTH);
case VARBINARY:
return DataTypes.VARBINARY(VarBinaryType.MAX_LENGTH);
case DECIMAL:
return DataTypes.DECIMAL(relDataType.getPrecision(), relDataType.getScale());
case ROW:
case ARRAY:
case MAP:
default:
throw new UnsupportedOperationException(
"Unsupported type: " + relDataType.getSqlTypeName());
}
}
public static Object convert(Object value, DataType dataType) {
if (value == null) {
return null;
}
switch (dataType.getTypeRoot()) {
case BOOLEAN:
return convertToBoolean(value);
case TINYINT:
return convertToByte(value);
case SMALLINT:
return convertToShort(value);
case INTEGER:
return convertToInt(value);
case BIGINT:
return convertToLong(value);
case DATE:
return convertToDate(value);
case TIME_WITHOUT_TIME_ZONE:
return convertToTime(value);
case TIMESTAMP_WITHOUT_TIME_ZONE:
return convertToTimestamp(value);
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
return convertToLocalTimeZoneTimestamp(value);
case FLOAT:
return convertToFloat(value);
case DOUBLE:
return convertToDouble(value);
case CHAR:
case VARCHAR:
return convertToString(value);
case BINARY:
case VARBINARY:
return convertToBinary(value);
case DECIMAL:
return convertToDecimal(value);
case ROW:
return value;
case ARRAY:
case MAP:
default:
throw new UnsupportedOperationException("Unsupported type: " + dataType);
}
}
public static Object convertToOriginal(Object value, DataType dataType) {
if (value == null) {
return null;
}
switch (dataType.getTypeRoot()) {
case BOOLEAN:
return convertToBoolean(value);
case TINYINT:
return convertToByte(value);
case SMALLINT:
return convertToShort(value);
case INTEGER:
return convertToInt(value);
case BIGINT:
return convertToLong(value);
case DATE:
return convertToDate(value);
case TIME_WITHOUT_TIME_ZONE:
return convertToTime(value);
case TIMESTAMP_WITHOUT_TIME_ZONE:
return convertToTimestamp(value);
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
return convertToLocalTimeZoneTimestamp(value);
case FLOAT:
return convertToFloat(value);
case DOUBLE:
return convertToDouble(value);
case CHAR:
case VARCHAR:
return convertToStringOriginal(value);
case BINARY:
case VARBINARY:
return convertToBinary(value);
case DECIMAL:
return convertToDecimalOriginal(value);
case ROW:
return value;
case ARRAY:
case MAP:
default:
throw new UnsupportedOperationException("Unsupported type: " + dataType);
}
}
private static Object convertToBoolean(Object obj) {
if (obj instanceof Boolean) {
return obj;
} else if (obj instanceof Byte) {
return (byte) obj == 1;
} else if (obj instanceof Short) {
return (short) obj == 1;
} else {
return Boolean.parseBoolean(obj.toString());
}
}
private static Object convertToByte(Object obj) {
return Byte.parseByte(obj.toString());
}
private static Object convertToShort(Object obj) {
return Short.parseShort(obj.toString());
}
private static Object convertToInt(Object obj) {
if (obj instanceof Integer) {
return obj;
} else if (obj instanceof Long) {
return ((Long) obj).intValue();
} else {
return Integer.parseInt(obj.toString());
}
}
private static Object convertToLong(Object obj) {
if (obj instanceof Integer) {
return ((Integer) obj).longValue();
} else if (obj instanceof Long) {
return obj;
} else {
return Long.parseLong(obj.toString());
}
}
private static Object convertToFloat(Object obj) {
if (obj instanceof Float) {
return obj;
} else if (obj instanceof Double) {
return ((Double) obj).floatValue();
} else {
return Float.parseFloat(obj.toString());
}
}
private static Object convertToDouble(Object obj) {
if (obj instanceof Float) {
return ((Float) obj).doubleValue();
} else if (obj instanceof Double) {
return obj;
} else {
return Double.parseDouble(obj.toString());
}
}
private static Object convertToDate(Object obj) {
return (int) toLocalDate(obj).toEpochDay();
}
private static LocalDate toLocalDate(Object obj) {
if (obj == null) {
return null;
}
if (obj instanceof LocalDate) {
return (LocalDate) obj;
}
if (obj instanceof LocalDateTime) {
return ((LocalDateTime) obj).toLocalDate();
}
if (obj instanceof java.sql.Date) {
return ((java.sql.Date) obj).toLocalDate();
}
if (obj instanceof java.sql.Time) {
throw new IllegalArgumentException(
"Unable to convert to LocalDate from a java.sql.Time value '" + obj + "'");
}
if (obj instanceof java.util.Date) {
java.util.Date date = (java.util.Date) obj;
return LocalDate.of(date.getYear() + 1900, date.getMonth() + 1, date.getDate());
}
if (obj instanceof Long) {
// Assume the value is the epoch day number
return LocalDate.ofEpochDay((Long) obj);
}
if (obj instanceof Integer) {
// Assume the value is the epoch day number
return LocalDate.ofEpochDay((Integer) obj);
}
throw new IllegalArgumentException(
"Unable to convert to LocalDate from unexpected value '"
+ obj
+ "' of type "
+ obj.getClass().getName());
}
private static Object convertToTime(Object obj) {
if (obj instanceof Integer) {
return obj;
}
// get number of milliseconds of the day
return toLocalTime(obj).toSecondOfDay() * 1000;
}
private static LocalTime toLocalTime(Object obj) {
if (obj == null) {
return null;
}
if (obj instanceof LocalTime) {
return (LocalTime) obj;
}
if (obj instanceof LocalDateTime) {
return ((LocalDateTime) obj).toLocalTime();
}
if (obj instanceof java.sql.Date) {
throw new IllegalArgumentException(
"Unable to convert to LocalDate from a java.sql.Date value '" + obj + "'");
}
if (obj instanceof java.sql.Time) {
java.sql.Time time = (java.sql.Time) obj;
long millis = (int) (time.getTime() % MILLISECONDS_PER_SECOND);
int nanosOfSecond = (int) (millis * NANOSECONDS_PER_MILLISECOND);
return LocalTime.of(
time.getHours(), time.getMinutes(), time.getSeconds(), nanosOfSecond);
}
if (obj instanceof java.sql.Timestamp) {
java.sql.Timestamp timestamp = (java.sql.Timestamp) obj;
return LocalTime.of(
timestamp.getHours(),
timestamp.getMinutes(),
timestamp.getSeconds(),
timestamp.getNanos());
}
if (obj instanceof java.util.Date) {
java.util.Date date = (java.util.Date) obj;
long millis = (int) (date.getTime() % MILLISECONDS_PER_SECOND);
int nanosOfSecond = (int) (millis * NANOSECONDS_PER_MILLISECOND);
return LocalTime.of(
date.getHours(), date.getMinutes(), date.getSeconds(), nanosOfSecond);
}
if (obj instanceof Duration) {
Long value = ((Duration) obj).toNanos();
if (value >= 0 && value <= NANOSECONDS_PER_DAY) {
return LocalTime.ofNanoOfDay(value);
} else {
throw new IllegalArgumentException(
"Time values must use number of milliseconds greater than 0 and less than 86400000000000");
}
}
throw new IllegalArgumentException(
"Unable to convert to LocalTime from unexpected value '"
+ obj
+ "' of type "
+ obj.getClass().getName());
}
private static Object convertToTimestamp(Object obj) {
if (obj instanceof Long) {
return TimestampData.fromMillis((Long) obj);
} else if (obj instanceof Timestamp) {
return TimestampData.fromTimestamp((Timestamp) obj);
} else if (obj instanceof TimestampData) {
return obj;
}
throw new IllegalArgumentException(
"Unable to convert to TIMESTAMP from unexpected value '"
+ obj
+ "' of type "
+ obj.getClass().getName());
}
private static Object convertToLocalTimeZoneTimestamp(Object obj) {
if (obj instanceof String) {
String str = (String) obj;
// TIMESTAMP_LTZ type is encoded in string type
Instant instant = Instant.parse(str);
return LocalZonedTimestampData.fromInstant(instant);
}
throw new IllegalArgumentException(
"Unable to convert to TIMESTAMP_LTZ from unexpected value '"
+ obj
+ "' of type "
+ obj.getClass().getName());
}
private static Object convertToString(Object obj) {
return BinaryStringData.fromString(obj.toString());
}
private static Object convertToStringOriginal(Object obj) {
return String.valueOf(obj);
}
private static Object convertToBinary(Object obj) {
if (obj instanceof byte[]) {
return obj;
} else if (obj instanceof ByteBuffer) {
ByteBuffer byteBuffer = (ByteBuffer) obj;
byte[] bytes = new byte[byteBuffer.remaining()];
byteBuffer.get(bytes);
return bytes;
} else {
throw new UnsupportedOperationException(
"Unsupported BYTES value type: " + obj.getClass().getSimpleName());
}
}
private static Object convertToDecimal(Object obj) {
if (obj instanceof BigDecimal) {
BigDecimal bigDecimalValue = (BigDecimal) obj;
return DecimalData.fromBigDecimal(
bigDecimalValue, bigDecimalValue.precision(), bigDecimalValue.scale());
} else if (obj instanceof DecimalData) {
return obj;
} else {
throw new UnsupportedOperationException(
"Unsupported Decimal value type: " + obj.getClass().getSimpleName());
}
}
private static Object convertToDecimalOriginal(Object obj) {
if (obj instanceof BigDecimal) {
return obj;
} else if (obj instanceof DecimalData) {
DecimalData decimalData = (DecimalData) obj;
return decimalData.toBigDecimal();
} else {
throw new UnsupportedOperationException(
"Unsupported Decimal value type: " + obj.getClass().getSimpleName());
}
}
}

@ -0,0 +1,624 @@
/*
* 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.transform;
import org.apache.flink.cdc.common.data.DecimalData;
import org.apache.flink.cdc.common.data.TimestampData;
import org.apache.flink.cdc.common.data.binary.BinaryStringData;
import org.apache.flink.cdc.common.event.CreateTableEvent;
import org.apache.flink.cdc.common.event.DataChangeEvent;
import org.apache.flink.cdc.common.event.Event;
import org.apache.flink.cdc.common.event.TableId;
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.runtime.testutils.operators.EventOperatorTestHarness;
import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.assertj.core.api.Assertions;
import org.junit.jupiter.api.Test;
import java.math.BigDecimal;
/** Unit tests for the {@link TransformDataOperator}. */
public class TransformDataOperatorTest {
private static final TableId CUSTOMERS_TABLEID =
TableId.tableId("my_company", "my_branch", "customers");
private static final Schema CUSTOMERS_SCHEMA =
Schema.newBuilder()
.physicalColumn("col1", DataTypes.STRING())
.physicalColumn("col2", DataTypes.STRING())
.physicalColumn("col12", DataTypes.STRING())
.primaryKey("col1")
.build();
private static final TableId DATATYPE_TABLEID =
TableId.tableId("my_company", "my_branch", "data_types");
private static final Schema DATATYPE_SCHEMA =
Schema.newBuilder()
.physicalColumn("colString", DataTypes.STRING())
.physicalColumn("colBoolean", DataTypes.BOOLEAN())
.physicalColumn("colTinyint", DataTypes.TINYINT())
.physicalColumn("colSmallint", DataTypes.SMALLINT())
.physicalColumn("colInt", DataTypes.INT())
.physicalColumn("colBigint", DataTypes.BIGINT())
.physicalColumn("colDate", DataTypes.DATE())
.physicalColumn("colTime", DataTypes.TIME())
.physicalColumn("colTimestamp", DataTypes.TIMESTAMP())
.physicalColumn("colFloat", DataTypes.FLOAT())
.physicalColumn("colDouble", DataTypes.DOUBLE())
.physicalColumn("colDecimal", DataTypes.DECIMAL(6, 2))
.primaryKey("colString")
.build();
private static final TableId METADATA_TABLEID =
TableId.tableId("my_company", "my_branch", "metadata_table");
private static final Schema METADATA_SCHEMA =
Schema.newBuilder()
.physicalColumn("col1", DataTypes.STRING())
.physicalColumn("identifier_name", DataTypes.STRING())
.physicalColumn("__namespace_name__", DataTypes.STRING())
.physicalColumn("__schema_name__", DataTypes.STRING())
.physicalColumn("__table_name__", DataTypes.STRING())
.primaryKey("col1")
.build();
private static final TableId METADATA_AS_TABLEID =
TableId.tableId("my_company", "my_branch", "metadata_as_table");
private static final Schema METADATA_AS_SCHEMA =
Schema.newBuilder()
.physicalColumn("sid", DataTypes.INT())
.physicalColumn("name", DataTypes.STRING())
.physicalColumn("name_upper", DataTypes.STRING())
.physicalColumn("tbname", DataTypes.STRING())
.primaryKey("sid")
.build();
private static final TableId TIMESTAMP_TABLEID =
TableId.tableId("my_company", "my_branch", "timestamp_table");
private static final Schema TIMESTAMP_SCHEMA =
Schema.newBuilder()
.physicalColumn("col1", DataTypes.STRING())
.physicalColumn("time_equal", DataTypes.INT())
.physicalColumn("timestamp_equal", DataTypes.INT())
.physicalColumn("date_equal", DataTypes.INT())
.primaryKey("col1")
.build();
private static final TableId TIMESTAMPDIFF_TABLEID =
TableId.tableId("my_company", "my_branch", "timestampdiff_table");
private static final Schema TIMESTAMPDIFF_SCHEMA =
Schema.newBuilder()
.physicalColumn("col1", DataTypes.STRING())
.physicalColumn("second_diff", DataTypes.INT())
.physicalColumn("minute_diff", DataTypes.INT())
.physicalColumn("hour_diff", DataTypes.INT())
.physicalColumn("day_diff", DataTypes.INT())
.physicalColumn("month_diff", DataTypes.INT())
.physicalColumn("year_diff", DataTypes.INT())
.primaryKey("col1")
.build();
private static final TableId CONDITION_TABLEID =
TableId.tableId("my_company", "my_branch", "condition_table");
private static final Schema CONDITION_SCHEMA =
Schema.newBuilder()
.physicalColumn("col1", DataTypes.STRING())
.physicalColumn("condition_result", DataTypes.BOOLEAN())
.primaryKey("col1")
.build();
@Test
void testDataChangeEventTransform() throws Exception {
TransformDataOperator transform =
TransformDataOperator.newBuilder()
.addTransform(
CUSTOMERS_TABLEID.identifier(),
"*, concat(col1,col2) col12",
"col1 = '1'")
.build();
EventOperatorTestHarness<TransformDataOperator, Event>
transformFunctionEventEventOperatorTestHarness =
new EventOperatorTestHarness<>(transform, 1);
// Initialization
transformFunctionEventEventOperatorTestHarness.open();
// Create table
CreateTableEvent createTableEvent =
new CreateTableEvent(CUSTOMERS_TABLEID, CUSTOMERS_SCHEMA);
BinaryRecordDataGenerator recordDataGenerator =
new BinaryRecordDataGenerator(((RowType) CUSTOMERS_SCHEMA.toRowDataType()));
// Insert
DataChangeEvent insertEvent =
DataChangeEvent.insertEvent(
CUSTOMERS_TABLEID,
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("1"), new BinaryStringData("2"), null
}));
DataChangeEvent insertEventExpect =
DataChangeEvent.insertEvent(
CUSTOMERS_TABLEID,
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("1"),
new BinaryStringData("2"),
new BinaryStringData("12")
}));
// Insert will be ignored
DataChangeEvent insertEventIgnored =
DataChangeEvent.insertEvent(
CUSTOMERS_TABLEID,
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("2"), new BinaryStringData("2"), null
}));
// Update
DataChangeEvent updateEvent =
DataChangeEvent.updateEvent(
CUSTOMERS_TABLEID,
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("1"), new BinaryStringData("2"), null
}),
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("1"), new BinaryStringData("3"), null
}));
DataChangeEvent updateEventExpect =
DataChangeEvent.updateEvent(
CUSTOMERS_TABLEID,
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("1"),
new BinaryStringData("2"),
new BinaryStringData("12")
}),
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("1"),
new BinaryStringData("3"),
new BinaryStringData("13")
}));
transform.processElement(new StreamRecord<>(createTableEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(
new StreamRecord<>(
new CreateTableEvent(CUSTOMERS_TABLEID, CUSTOMERS_SCHEMA)));
transform.processElement(new StreamRecord<>(insertEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(new StreamRecord<>(insertEventExpect));
transform.processElement(new StreamRecord<>(insertEventIgnored));
transform.processElement(new StreamRecord<>(updateEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(new StreamRecord<>(updateEventExpect));
}
@Test
void testDataChangeEventTransformTwice() throws Exception {
TransformDataOperator transform =
TransformDataOperator.newBuilder()
.addTransform(
CUSTOMERS_TABLEID.identifier(),
"*, concat(col1, '1') col12",
"col1 = '1'")
.addTransform(
CUSTOMERS_TABLEID.identifier(),
"*, concat(col1, '2') col12",
"col1 = '2'")
.build();
EventOperatorTestHarness<TransformDataOperator, Event>
transformFunctionEventEventOperatorTestHarness =
new EventOperatorTestHarness<>(transform, 1);
// Initialization
transformFunctionEventEventOperatorTestHarness.open();
// Create table
CreateTableEvent createTableEvent =
new CreateTableEvent(CUSTOMERS_TABLEID, CUSTOMERS_SCHEMA);
BinaryRecordDataGenerator recordDataGenerator =
new BinaryRecordDataGenerator(((RowType) CUSTOMERS_SCHEMA.toRowDataType()));
// Insert
DataChangeEvent insertEvent =
DataChangeEvent.insertEvent(
CUSTOMERS_TABLEID,
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("1"), new BinaryStringData("2"), null
}));
DataChangeEvent insertEventExpect =
DataChangeEvent.insertEvent(
CUSTOMERS_TABLEID,
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("1"),
new BinaryStringData("2"),
new BinaryStringData("11")
}));
// Insert
DataChangeEvent insertEvent2 =
DataChangeEvent.insertEvent(
CUSTOMERS_TABLEID,
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("2"), new BinaryStringData("2"), null
}));
DataChangeEvent insertEvent2Expect =
DataChangeEvent.insertEvent(
CUSTOMERS_TABLEID,
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("2"),
new BinaryStringData("2"),
new BinaryStringData("22")
}));
// Update
DataChangeEvent updateEvent =
DataChangeEvent.updateEvent(
CUSTOMERS_TABLEID,
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("1"), new BinaryStringData("2"), null
}),
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("1"), new BinaryStringData("3"), null
}));
DataChangeEvent updateEventExpect =
DataChangeEvent.updateEvent(
CUSTOMERS_TABLEID,
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("1"),
new BinaryStringData("2"),
new BinaryStringData("11")
}),
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("1"),
new BinaryStringData("3"),
new BinaryStringData("11")
}));
transform.processElement(new StreamRecord<>(createTableEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(
new StreamRecord<>(
new CreateTableEvent(CUSTOMERS_TABLEID, CUSTOMERS_SCHEMA)));
transform.processElement(new StreamRecord<>(insertEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(new StreamRecord<>(insertEventExpect));
transform.processElement(new StreamRecord<>(insertEvent2));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(new StreamRecord<>(insertEvent2Expect));
transform.processElement(new StreamRecord<>(updateEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(new StreamRecord<>(updateEventExpect));
}
@Test
void testDataChangeEventTransformProjectionDataTypeConvert() throws Exception {
TransformDataOperator transform =
TransformDataOperator.newBuilder()
.addTransform(DATATYPE_TABLEID.identifier(), "*", null)
.build();
EventOperatorTestHarness<TransformDataOperator, Event>
transformFunctionEventEventOperatorTestHarness =
new EventOperatorTestHarness<>(transform, 1);
// Initialization
transformFunctionEventEventOperatorTestHarness.open();
// Create table
CreateTableEvent createTableEvent = new CreateTableEvent(DATATYPE_TABLEID, DATATYPE_SCHEMA);
BinaryRecordDataGenerator recordDataGenerator =
new BinaryRecordDataGenerator(((RowType) DATATYPE_SCHEMA.toRowDataType()));
// Insert
DataChangeEvent insertEvent =
DataChangeEvent.insertEvent(
DATATYPE_TABLEID,
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("3.14"),
new Boolean(true),
new Byte("1"),
new Short("1"),
new Integer(1),
new Long(1),
new Integer(1704471599),
new Integer(1704471599),
TimestampData.fromMillis(1704471599),
new Float(3.14f),
new Double(3.14d),
DecimalData.fromBigDecimal(new BigDecimal(3.14), 6, 2),
}));
transform.processElement(new StreamRecord<>(createTableEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(
new StreamRecord<>(
new CreateTableEvent(DATATYPE_TABLEID, DATATYPE_SCHEMA)));
transform.processElement(new StreamRecord<>(insertEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(new StreamRecord<>(insertEvent));
}
@Test
void testMetadataTransform() throws Exception {
TransformDataOperator transform =
TransformDataOperator.newBuilder()
.addTransform(
METADATA_TABLEID.identifier(),
"*, __namespace_name__ || '.' || __schema_name__ || '.' || __table_name__ identifier_name, __namespace_name__, __schema_name__, __table_name__",
" __table_name__ = 'metadata_table' ")
.build();
EventOperatorTestHarness<TransformDataOperator, Event>
transformFunctionEventEventOperatorTestHarness =
new EventOperatorTestHarness<>(transform, 1);
// Initialization
transformFunctionEventEventOperatorTestHarness.open();
// Create table
CreateTableEvent createTableEvent = new CreateTableEvent(METADATA_TABLEID, METADATA_SCHEMA);
BinaryRecordDataGenerator recordDataGenerator =
new BinaryRecordDataGenerator(((RowType) METADATA_SCHEMA.toRowDataType()));
// Insert
DataChangeEvent insertEvent =
DataChangeEvent.insertEvent(
METADATA_TABLEID,
recordDataGenerator.generate(
new Object[] {new BinaryStringData("1"), null, null, null, null}));
DataChangeEvent insertEventExpect =
DataChangeEvent.insertEvent(
METADATA_TABLEID,
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("1"),
new BinaryStringData("my_company.my_branch.metadata_table"),
new BinaryStringData("my_company"),
new BinaryStringData("my_branch"),
new BinaryStringData("metadata_table")
}));
transform.processElement(new StreamRecord<>(createTableEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(
new StreamRecord<>(
new CreateTableEvent(METADATA_TABLEID, METADATA_SCHEMA)));
transform.processElement(new StreamRecord<>(insertEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(new StreamRecord<>(insertEventExpect));
}
@Test
void testMetadataASTransform() throws Exception {
TransformDataOperator transform =
TransformDataOperator.newBuilder()
.addTransform(
METADATA_AS_TABLEID.identifier(),
"sid, name, UPPER(name) as name_upper, __table_name__ as tbname",
"sid < 3")
.build();
EventOperatorTestHarness<TransformDataOperator, Event>
transformFunctionEventEventOperatorTestHarness =
new EventOperatorTestHarness<>(transform, 1);
// Initialization
transformFunctionEventEventOperatorTestHarness.open();
// Create table
CreateTableEvent createTableEvent =
new CreateTableEvent(METADATA_AS_TABLEID, METADATA_AS_SCHEMA);
BinaryRecordDataGenerator recordDataGenerator =
new BinaryRecordDataGenerator(((RowType) METADATA_AS_SCHEMA.toRowDataType()));
// Insert
DataChangeEvent insertEvent =
DataChangeEvent.insertEvent(
METADATA_AS_TABLEID,
recordDataGenerator.generate(
new Object[] {1, new BinaryStringData("abc"), null, null}));
DataChangeEvent insertEventExpect =
DataChangeEvent.insertEvent(
METADATA_AS_TABLEID,
recordDataGenerator.generate(
new Object[] {
1,
new BinaryStringData("abc"),
new BinaryStringData("ABC"),
new BinaryStringData("metadata_as_table")
}));
transform.processElement(new StreamRecord<>(createTableEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(
new StreamRecord<>(
new CreateTableEvent(METADATA_AS_TABLEID, METADATA_AS_SCHEMA)));
transform.processElement(new StreamRecord<>(insertEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(new StreamRecord<>(insertEventExpect));
}
@Test
void testTimestampTransform() throws Exception {
TransformDataOperator transform =
TransformDataOperator.newBuilder()
.addTransform(
TIMESTAMP_TABLEID.identifier(),
"col1, IF(LOCALTIME = CURRENT_TIME, 1, 0) as time_equal,"
+ " IF(LOCALTIMESTAMP = CURRENT_TIMESTAMP and NOW() = LOCALTIMESTAMP, 1, 0) as timestamp_equal,"
+ " IF(TO_DATE(DATE_FORMAT(LOCALTIMESTAMP, 'yyyy-MM-dd')) = CURRENT_DATE, 1, 0) as date_equal",
"LOCALTIMESTAMP = CURRENT_TIMESTAMP")
.addTimezone("GMT")
.build();
EventOperatorTestHarness<TransformDataOperator, Event>
transformFunctionEventEventOperatorTestHarness =
new EventOperatorTestHarness<>(transform, 1);
// Initialization
transformFunctionEventEventOperatorTestHarness.open();
// Create table
CreateTableEvent createTableEvent =
new CreateTableEvent(TIMESTAMP_TABLEID, TIMESTAMP_SCHEMA);
BinaryRecordDataGenerator recordDataGenerator =
new BinaryRecordDataGenerator(((RowType) TIMESTAMP_SCHEMA.toRowDataType()));
// Insert
DataChangeEvent insertEvent =
DataChangeEvent.insertEvent(
TIMESTAMP_TABLEID,
recordDataGenerator.generate(
new Object[] {new BinaryStringData("1"), null, null, null}));
DataChangeEvent insertEventExpect =
DataChangeEvent.insertEvent(
TIMESTAMP_TABLEID,
recordDataGenerator.generate(
new Object[] {new BinaryStringData("1"), 1, 1, 1}));
transform.processElement(new StreamRecord<>(createTableEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(
new StreamRecord<>(
new CreateTableEvent(TIMESTAMP_TABLEID, TIMESTAMP_SCHEMA)));
transform.processElement(new StreamRecord<>(insertEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(new StreamRecord<>(insertEventExpect));
}
@Test
void testTimestampDiffTransform() throws Exception {
TransformDataOperator transform =
TransformDataOperator.newBuilder()
.addTransform(
TIMESTAMPDIFF_TABLEID.identifier(),
"col1, TIMESTAMP_DIFF('SECOND', LOCALTIMESTAMP, CURRENT_TIMESTAMP) as second_diff,"
+ " TIMESTAMP_DIFF('MINUTE', LOCALTIMESTAMP, CURRENT_TIMESTAMP) as minute_diff,"
+ " TIMESTAMP_DIFF('HOUR', LOCALTIMESTAMP, CURRENT_TIMESTAMP) as hour_diff,"
+ " TIMESTAMP_DIFF('DAY', LOCALTIMESTAMP, CURRENT_TIMESTAMP) as day_diff,"
+ " TIMESTAMP_DIFF('MONTH', LOCALTIMESTAMP, CURRENT_TIMESTAMP) as month_diff,"
+ " TIMESTAMP_DIFF('YEAR', LOCALTIMESTAMP, CURRENT_TIMESTAMP) as year_diff",
null)
.addTimezone("GMT-8:00")
.build();
EventOperatorTestHarness<TransformDataOperator, Event>
transformFunctionEventEventOperatorTestHarness =
new EventOperatorTestHarness<>(transform, 1);
// Initialization
transformFunctionEventEventOperatorTestHarness.open();
// Create table
CreateTableEvent createTableEvent =
new CreateTableEvent(TIMESTAMPDIFF_TABLEID, TIMESTAMPDIFF_SCHEMA);
BinaryRecordDataGenerator recordDataGenerator =
new BinaryRecordDataGenerator(((RowType) TIMESTAMPDIFF_SCHEMA.toRowDataType()));
// Insert
DataChangeEvent insertEvent =
DataChangeEvent.insertEvent(
TIMESTAMPDIFF_TABLEID,
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("1"), null, null, null, null, null, null
}));
DataChangeEvent insertEventExpect =
DataChangeEvent.insertEvent(
TIMESTAMPDIFF_TABLEID,
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("1"), -28800, -480, -8, 0, 0, 0
}));
transform.processElement(new StreamRecord<>(createTableEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(
new StreamRecord<>(
new CreateTableEvent(TIMESTAMPDIFF_TABLEID, TIMESTAMPDIFF_SCHEMA)));
transform.processElement(new StreamRecord<>(insertEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(new StreamRecord<>(insertEventExpect));
}
@Test
void testBuildInFunctionTransform() throws Exception {
testExpressionConditionTransform(
"TO_TIMESTAMP('1970-01-01 00:00:00') = TO_TIMESTAMP('1970-01-01', 'yyyy-MM-dd')");
testExpressionConditionTransform(
"TIMESTAMP_DIFF('DAY', TO_TIMESTAMP('1970-01-01 00:00:00'), TO_TIMESTAMP('1970-01-02 00:00:00')) = 1");
testExpressionConditionTransform("2 between 1 and 3");
testExpressionConditionTransform("4 not between 1 and 3");
testExpressionConditionTransform("2 in (1, 2, 3)");
testExpressionConditionTransform("4 not in (1, 2, 3)");
testExpressionConditionTransform("CHAR_LENGTH('abc') = 3");
testExpressionConditionTransform("trim(' abc ') = 'abc'");
testExpressionConditionTransform("REGEXP_REPLACE('123abc', '[a-zA-Z]', '') = '123'");
testExpressionConditionTransform("concat('123', 'abc') = '123abc'");
testExpressionConditionTransform("upper('abc') = 'ABC'");
testExpressionConditionTransform("lower('ABC') = 'abc'");
testExpressionConditionTransform("SUBSTR('ABC', 1, 1) = 'B'");
testExpressionConditionTransform("'ABC' like '^[a-zA-Z]'");
testExpressionConditionTransform("'123' not like '^[a-zA-Z]'");
testExpressionConditionTransform("abs(2) = 2");
testExpressionConditionTransform("ceil(2.4) = 3.0");
testExpressionConditionTransform("floor(2.5) = 2.0");
testExpressionConditionTransform("round(3.1415926,2) = 3.14");
}
void testExpressionConditionTransform(String expression) throws Exception {
TransformDataOperator transform =
TransformDataOperator.newBuilder()
.addTransform(
CONDITION_TABLEID.identifier(),
"col1, IF(" + expression + ", true, false) as condition_result",
expression)
.addTimezone("GMT")
.build();
EventOperatorTestHarness<TransformDataOperator, Event>
transformFunctionEventEventOperatorTestHarness =
new EventOperatorTestHarness<>(transform, 1);
// Initialization
transformFunctionEventEventOperatorTestHarness.open();
// Create table
CreateTableEvent createTableEvent =
new CreateTableEvent(CONDITION_TABLEID, CONDITION_SCHEMA);
BinaryRecordDataGenerator recordDataGenerator =
new BinaryRecordDataGenerator(((RowType) CONDITION_SCHEMA.toRowDataType()));
// Insert
DataChangeEvent insertEvent =
DataChangeEvent.insertEvent(
CONDITION_TABLEID,
recordDataGenerator.generate(
new Object[] {new BinaryStringData("1"), null}));
DataChangeEvent insertEventExpect =
DataChangeEvent.insertEvent(
CONDITION_TABLEID,
recordDataGenerator.generate(
new Object[] {new BinaryStringData("1"), true}));
transform.processElement(new StreamRecord<>(createTableEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(
new StreamRecord<>(
new CreateTableEvent(CONDITION_TABLEID, CONDITION_SCHEMA)));
transform.processElement(new StreamRecord<>(insertEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(new StreamRecord<>(insertEventExpect));
}
}

@ -0,0 +1,179 @@
/*
* 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.transform;
import org.apache.flink.cdc.common.data.binary.BinaryStringData;
import org.apache.flink.cdc.common.event.AddColumnEvent;
import org.apache.flink.cdc.common.event.CreateTableEvent;
import org.apache.flink.cdc.common.event.DataChangeEvent;
import org.apache.flink.cdc.common.event.Event;
import org.apache.flink.cdc.common.event.TableId;
import org.apache.flink.cdc.common.schema.Column;
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.runtime.testutils.operators.EventOperatorTestHarness;
import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.assertj.core.api.Assertions;
import org.junit.jupiter.api.Test;
import org.testcontainers.shaded.com.google.common.collect.ImmutableMap;
import java.util.Collections;
/** Unit tests for the {@link TransformSchemaOperator}. */
public class TransformSchemaOperatorTest {
private static final TableId CUSTOMERS_TABLEID =
TableId.tableId("my_company", "my_branch", "customers");
private static final Schema CUSTOMERS_SCHEMA =
Schema.newBuilder()
.physicalColumn("col1", DataTypes.STRING())
.physicalColumn("col2", DataTypes.STRING())
.primaryKey("col1")
.build();
private static final Schema CUSTOMERS_LATEST_SCHEMA =
Schema.newBuilder()
.physicalColumn("col1", DataTypes.STRING())
.physicalColumn("col2", DataTypes.STRING())
.physicalColumn("col3", DataTypes.STRING())
.primaryKey("col1")
.build();
private static final Schema EXPECT_SCHEMA =
Schema.newBuilder()
.physicalColumn("col1", DataTypes.STRING())
.physicalColumn("col2", DataTypes.STRING())
.physicalColumn("col12", DataTypes.STRING())
.primaryKey("col2")
.partitionKey("col12")
.options(ImmutableMap.of("key1", "value1", "key2", "value2"))
.build();
private static final Schema EXPECT_LATEST_SCHEMA =
Schema.newBuilder()
.physicalColumn("col1", DataTypes.STRING())
.physicalColumn("col2", DataTypes.STRING())
.physicalColumn("col12", DataTypes.STRING())
.physicalColumn("col3", DataTypes.STRING())
.primaryKey("col2")
.partitionKey("col12")
.options(ImmutableMap.of("key1", "value1", "key2", "value2"))
.build();
@Test
void testEventTransform() throws Exception {
TransformSchemaOperator transform =
TransformSchemaOperator.newBuilder()
.addTransform(
CUSTOMERS_TABLEID.identifier(),
"*, concat(col1,col2) col12",
"col2",
"col12",
"key1=value1,key2=value2")
.build();
EventOperatorTestHarness<TransformSchemaOperator, Event>
transformFunctionEventEventOperatorTestHarness =
new EventOperatorTestHarness<>(transform, 1);
// Initialization
transformFunctionEventEventOperatorTestHarness.open();
// Create table
CreateTableEvent createTableEvent =
new CreateTableEvent(CUSTOMERS_TABLEID, CUSTOMERS_SCHEMA);
// Add column
AddColumnEvent.ColumnWithPosition columnWithPosition =
new AddColumnEvent.ColumnWithPosition(
Column.physicalColumn("col3", DataTypes.STRING()));
AddColumnEvent addColumnEvent =
new AddColumnEvent(
CUSTOMERS_TABLEID, Collections.singletonList(columnWithPosition));
BinaryRecordDataGenerator recordDataGenerator =
new BinaryRecordDataGenerator(((RowType) CUSTOMERS_LATEST_SCHEMA.toRowDataType()));
BinaryRecordDataGenerator recordDataGeneratorExpect =
new BinaryRecordDataGenerator(((RowType) EXPECT_LATEST_SCHEMA.toRowDataType()));
// Insert
DataChangeEvent insertEvent =
DataChangeEvent.insertEvent(
CUSTOMERS_TABLEID,
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("1"),
new BinaryStringData("2"),
new BinaryStringData("3"),
}));
DataChangeEvent insertEventExpect =
DataChangeEvent.insertEvent(
CUSTOMERS_TABLEID,
recordDataGeneratorExpect.generate(
new Object[] {
new BinaryStringData("1"),
new BinaryStringData("2"),
null,
new BinaryStringData("3")
}));
// Update
DataChangeEvent updateEvent =
DataChangeEvent.updateEvent(
CUSTOMERS_TABLEID,
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("1"),
new BinaryStringData("2"),
new BinaryStringData("3")
}),
recordDataGenerator.generate(
new Object[] {
new BinaryStringData("1"),
new BinaryStringData("3"),
new BinaryStringData("3")
}));
DataChangeEvent updateEventExpect =
DataChangeEvent.updateEvent(
CUSTOMERS_TABLEID,
recordDataGeneratorExpect.generate(
new Object[] {
new BinaryStringData("1"),
new BinaryStringData("2"),
null,
new BinaryStringData("3")
}),
recordDataGeneratorExpect.generate(
new Object[] {
new BinaryStringData("1"),
new BinaryStringData("3"),
null,
new BinaryStringData("3")
}));
transform.processElement(new StreamRecord<>(createTableEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(
new StreamRecord<>(new CreateTableEvent(CUSTOMERS_TABLEID, EXPECT_SCHEMA)));
transform.processElement(new StreamRecord<>(addColumnEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(new StreamRecord<>(addColumnEvent));
transform.processElement(new StreamRecord<>(insertEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(new StreamRecord<>(insertEventExpect));
transform.processElement(new StreamRecord<>(updateEvent));
Assertions.assertThat(
transformFunctionEventEventOperatorTestHarness.getOutputRecords().poll())
.isEqualTo(new StreamRecord<>(updateEventExpect));
}
}

@ -0,0 +1,142 @@
/*
* 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.parser;
import org.codehaus.commons.compiler.CompileException;
import org.codehaus.commons.compiler.Location;
import org.codehaus.janino.ExpressionEvaluator;
import org.codehaus.janino.Java;
import org.codehaus.janino.Parser;
import org.codehaus.janino.Scanner;
import org.codehaus.janino.Unparser;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.io.StringReader;
import java.io.StringWriter;
import java.lang.reflect.InvocationTargetException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.TimeZone;
/** Unit tests for the {@link JaninoCompiler}. */
public class JaninoCompilerTest {
@Test
public void testJaninoParser() throws CompileException, IOException, InvocationTargetException {
String expression = "1==2";
Parser parser = new Parser(new Scanner(null, new StringReader(expression)));
ExpressionEvaluator expressionEvaluator = new ExpressionEvaluator();
expressionEvaluator.cook(parser);
Object evaluate = expressionEvaluator.evaluate();
Assert.assertEquals(false, evaluate);
}
@Test
public void testJaninoUnParser() {
String expression = "1 <= 2";
String[] values = new String[1];
values[0] = "1";
String value2 = "2";
Java.AmbiguousName ambiguousName1 = new Java.AmbiguousName(Location.NOWHERE, values);
Java.AmbiguousName ambiguousName2 =
new Java.AmbiguousName(Location.NOWHERE, new String[] {value2});
Java.BinaryOperation binaryOperation =
new Java.BinaryOperation(Location.NOWHERE, ambiguousName1, "<=", ambiguousName2);
StringWriter writer = new StringWriter();
Unparser unparser = new Unparser(writer);
unparser.unparseAtom(binaryOperation);
unparser.close();
Assert.assertEquals(expression, writer.toString());
}
@Test
public void testJaninoNumericCompare() throws InvocationTargetException {
String expression = "col1==3.14";
List<String> columnNames = Arrays.asList("col1");
List<Class<?>> paramTypes = Arrays.asList(Double.class);
List<Object> params = Arrays.asList(3.14);
ExpressionEvaluator expressionEvaluator =
JaninoCompiler.compileExpression(
expression, columnNames, paramTypes, Boolean.class);
Object evaluate = expressionEvaluator.evaluate(params.toArray());
Assert.assertEquals(true, evaluate);
}
@Test
public void testJaninoCharCompare() throws InvocationTargetException {
String expression = "String.valueOf('2').equals(col1)";
List<String> columnNames = Arrays.asList("col1");
List<Class<?>> paramTypes = Arrays.asList(String.class);
List<Object> params = Arrays.asList("2");
ExpressionEvaluator expressionEvaluator =
JaninoCompiler.compileExpression(
expression, columnNames, paramTypes, Boolean.class);
Object evaluate = expressionEvaluator.evaluate(params.toArray());
Assert.assertEquals(true, evaluate);
}
@Test
public void testJaninoStringCompare() throws InvocationTargetException {
String expression = "String.valueOf(\"metadata_table\").equals(__table_name__)";
List<String> columnNames = Arrays.asList("__table_name__");
List<Class<?>> paramTypes = Arrays.asList(String.class);
List<Object> params = Arrays.asList("metadata_table");
ExpressionEvaluator expressionEvaluator =
JaninoCompiler.compileExpression(
expression, columnNames, paramTypes, Boolean.class);
Object evaluate = expressionEvaluator.evaluate(params.toArray());
Assert.assertEquals(true, evaluate);
}
@Test
public void testJaninoTimestampFunction() throws InvocationTargetException {
long epochTime = System.currentTimeMillis();
long localTime = epochTime + TimeZone.getTimeZone("GMT-8:00").getOffset(epochTime);
String expression = "currentTimestamp(epochTime, \"GMT-8:00\")";
List<String> columnNames = Arrays.asList("epochTime");
List<Class<?>> paramTypes = Arrays.asList(Long.class);
List<Object> params = Arrays.asList(epochTime);
ExpressionEvaluator expressionEvaluator =
JaninoCompiler.compileExpression(
JaninoCompiler.loadSystemFunction(expression),
columnNames,
paramTypes,
Long.class);
Object evaluate = expressionEvaluator.evaluate(params.toArray());
Assert.assertEquals(localTime, evaluate);
}
@Test
public void testBuildInFunction() throws InvocationTargetException {
String expression = "ceil(2.4)";
List<String> columnNames = new ArrayList<>();
List<Class<?>> paramTypes = new ArrayList<>();
List<Object> params = new ArrayList<>();
ExpressionEvaluator expressionEvaluator =
JaninoCompiler.compileExpression(
JaninoCompiler.loadSystemFunction(expression),
columnNames,
paramTypes,
Double.class);
Object evaluate = expressionEvaluator.evaluate(params.toArray());
Assert.assertEquals(3.0, evaluate);
}
}

@ -0,0 +1,277 @@
/*
* 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.parser;
import org.apache.flink.cdc.common.schema.Schema;
import org.apache.flink.cdc.common.types.DataTypes;
import org.apache.flink.cdc.runtime.parser.metadata.TransformSchemaFactory;
import org.apache.flink.cdc.runtime.parser.metadata.TransformSqlOperatorTable;
import org.apache.flink.table.api.ApiExpression;
import org.apache.flink.table.api.Expressions;
import org.apache.calcite.config.CalciteConnectionConfigImpl;
import org.apache.calcite.jdbc.CalciteSchema;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.hep.HepPlanner;
import org.apache.calcite.plan.hep.HepProgramBuilder;
import org.apache.calcite.prepare.CalciteCatalogReader;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.RelRoot;
import org.apache.calcite.rel.type.RelDataTypeSystem;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.sql.SqlNode;
import org.apache.calcite.sql.SqlSelect;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
import org.apache.calcite.sql.util.SqlOperatorTables;
import org.apache.calcite.sql.validate.SqlValidator;
import org.apache.calcite.sql.validate.SqlValidatorUtil;
import org.apache.calcite.sql2rel.RelDecorrelator;
import org.apache.calcite.sql2rel.SqlToRelConverter;
import org.apache.calcite.sql2rel.StandardConvertletTable;
import org.apache.calcite.tools.RelBuilder;
import org.junit.Assert;
import org.junit.Test;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
/** Unit tests for the {@link TransformParser}. */
public class TransformParserTest {
private static final Schema CUSTOMERS_SCHEMA =
Schema.newBuilder()
.physicalColumn("id", DataTypes.STRING())
.physicalColumn("order_id", DataTypes.STRING())
.primaryKey("id")
.build();
@Test
public void testCalciteParser() {
SqlSelect parse =
TransformParser.parseSelect(
"select CONCAT(id, order_id) as uniq_id, * from tb where uniq_id > 10 and id is not null");
Assert.assertEquals(
"`CONCAT`(`id`, `order_id`) AS `uniq_id`, *", parse.getSelectList().toString());
Assert.assertEquals("`uniq_id` > 10 AND `id` IS NOT NULL", parse.getWhere().toString());
}
@Test
public void testTransformCalciteValidate() {
SqlSelect parse =
TransformParser.parseSelect(
"select SUBSTR(id, 1) as uniq_id, * from tb where id is not null");
CalciteSchema rootSchema = CalciteSchema.createRootSchema(true);
Map<String, Object> operand = new HashMap<>();
operand.put("tableName", "tb");
operand.put("columns", CUSTOMERS_SCHEMA.getColumns());
org.apache.calcite.schema.Schema schema =
TransformSchemaFactory.INSTANCE.create(
rootSchema.plus(), "default_schema", operand);
rootSchema.add("default_schema", schema);
SqlTypeFactoryImpl factory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
CalciteCatalogReader calciteCatalogReader =
new CalciteCatalogReader(
rootSchema,
rootSchema.path("default_schema"),
factory,
new CalciteConnectionConfigImpl(new Properties()));
TransformSqlOperatorTable transformSqlOperatorTable = TransformSqlOperatorTable.instance();
SqlStdOperatorTable sqlStdOperatorTable = SqlStdOperatorTable.instance();
SqlValidator validator =
SqlValidatorUtil.newValidator(
SqlOperatorTables.chain(sqlStdOperatorTable, transformSqlOperatorTable),
calciteCatalogReader,
factory,
SqlValidator.Config.DEFAULT.withIdentifierExpansion(true));
SqlNode validateSqlNode = validator.validate(parse);
Assert.assertEquals(
"SUBSTR(`tb`.`id`, 1) AS `uniq_id`, `tb`.`id`, `tb`.`order_id`",
parse.getSelectList().toString());
Assert.assertEquals("`tb`.`id` IS NOT NULL", parse.getWhere().toString());
Assert.assertEquals(
"SELECT SUBSTR(`tb`.`id`, 1) AS `uniq_id`, `tb`.`id`, `tb`.`order_id`\n"
+ "FROM `default_schema`.`tb` AS `tb`\n"
+ "WHERE `tb`.`id` IS NOT NULL",
validateSqlNode.toString().replaceAll("\r\n", "\n"));
}
@Test
public void testCalciteRelNode() {
SqlSelect parse =
TransformParser.parseSelect(
"select SUBSTR(id, 1) as uniq_id, * from tb where id is not null");
CalciteSchema rootSchema = CalciteSchema.createRootSchema(true);
Map<String, Object> operand = new HashMap<>();
operand.put("tableName", "tb");
operand.put("columns", CUSTOMERS_SCHEMA.getColumns());
org.apache.calcite.schema.Schema schema =
TransformSchemaFactory.INSTANCE.create(
rootSchema.plus(), "default_schema", operand);
rootSchema.add("default_schema", schema);
SqlTypeFactoryImpl factory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
CalciteCatalogReader calciteCatalogReader =
new CalciteCatalogReader(
rootSchema,
rootSchema.path("default_schema"),
factory,
new CalciteConnectionConfigImpl(new Properties()));
TransformSqlOperatorTable transformSqlOperatorTable = TransformSqlOperatorTable.instance();
SqlStdOperatorTable sqlStdOperatorTable = SqlStdOperatorTable.instance();
SqlValidator validator =
SqlValidatorUtil.newValidator(
SqlOperatorTables.chain(sqlStdOperatorTable, transformSqlOperatorTable),
calciteCatalogReader,
factory,
SqlValidator.Config.DEFAULT.withIdentifierExpansion(true));
SqlNode validateSqlNode = validator.validate(parse);
RexBuilder rexBuilder = new RexBuilder(factory);
HepProgramBuilder builder = new HepProgramBuilder();
HepPlanner planner = new HepPlanner(builder.build());
RelOptCluster cluster = RelOptCluster.create(planner, rexBuilder);
SqlToRelConverter.Config config = SqlToRelConverter.config().withTrimUnusedFields(false);
SqlToRelConverter sqlToRelConverter =
new SqlToRelConverter(
null,
validator,
calciteCatalogReader,
cluster,
StandardConvertletTable.INSTANCE,
config);
RelRoot relRoot = sqlToRelConverter.convertQuery(validateSqlNode, false, true);
relRoot = relRoot.withRel(sqlToRelConverter.flattenTypes(relRoot.rel, true));
RelBuilder relBuilder = config.getRelBuilderFactory().create(cluster, null);
relRoot = relRoot.withRel(RelDecorrelator.decorrelateQuery(relRoot.rel, relBuilder));
RelNode relNode = relRoot.rel;
Assert.assertEquals(
"SUBSTR(`tb`.`id`, 1) AS `uniq_id`, `tb`.`id`, `tb`.`order_id`",
parse.getSelectList().toString());
Assert.assertEquals("`tb`.`id` IS NOT NULL", parse.getWhere().toString());
Assert.assertEquals(
"SELECT SUBSTR(`tb`.`id`, 1) AS `uniq_id`, `tb`.`id`, `tb`.`order_id`\n"
+ "FROM `default_schema`.`tb` AS `tb`\n"
+ "WHERE `tb`.`id` IS NOT NULL",
validateSqlNode.toString().replaceAll("\r\n", "\n"));
}
@Test
public void testParseComputedColumnNames() {
List<String> computedColumnNames =
TransformParser.parseComputedColumnNames("CONCAT(id, order_id) as uniq_id, *");
Assert.assertEquals(new String[] {"uniq_id"}, computedColumnNames.toArray());
}
@Test
public void testParseFilterColumnNameList() {
List<String> computedColumnNames =
TransformParser.parseFilterColumnNameList(" uniq_id > 10 and id is not null");
Assert.assertEquals(new String[] {"uniq_id", "id"}, computedColumnNames.toArray());
}
@Test
public void testTranslateFilterToJaninoExpression() {
testFilterExpression("id is not null", "null != id");
testFilterExpression("id is null", "null == id");
testFilterExpression("id = 1 and uid = 2", "valueEquals(id, 1) && valueEquals(uid, 2)");
testFilterExpression("id = 1 or id = 2", "valueEquals(id, 1) || valueEquals(id, 2)");
testFilterExpression("not (id = 1)", "!valueEquals(id, 1)");
testFilterExpression("id = '1'", "valueEquals(id, \"1\")");
testFilterExpression("id <> '1'", "!valueEquals(id, \"1\")");
testFilterExpression("d between d1 and d2", "betweenAsymmetric(d, d1, d2)");
testFilterExpression("d not between d1 and d2", "notBetweenAsymmetric(d, d1, d2)");
testFilterExpression("d in (d1, d2)", "in(d, d1, d2)");
testFilterExpression("d not in (d1, d2)", "notIn(d, d1, d2)");
testFilterExpression("id is false", "false == id");
testFilterExpression("id is not false", "true == id");
testFilterExpression("id is true", "true == id");
testFilterExpression("id is not true", "false == id");
testFilterExpression("a || b", "concat(a, b)");
testFilterExpression("CHAR_LENGTH(id)", "charLength(id)");
testFilterExpression("trim(id)", "trim(\"BOTH\", \" \", id)");
testFilterExpression(
"REGEXP_REPLACE(id, '[a-zA-Z]', '')", "regexpReplace(id, \"[a-zA-Z]\", \"\")");
testFilterExpression("upper(id)", "upper(id)");
testFilterExpression("lower(id)", "lower(id)");
testFilterExpression("concat(a,b)", "concat(a, b)");
testFilterExpression("SUBSTR(a,1)", "substr(a, 1)");
testFilterExpression("id like '^[a-zA-Z]'", "like(id, \"^[a-zA-Z]\")");
testFilterExpression("id not like '^[a-zA-Z]'", "notLike(id, \"^[a-zA-Z]\")");
testFilterExpression("abs(2)", "abs(2)");
testFilterExpression("ceil(2)", "ceil(2)");
testFilterExpression("floor(2)", "floor(2)");
testFilterExpression("round(2,2)", "round(2, 2)");
testFilterExpression("uuid()", "uuid()");
testFilterExpression(
"id = LOCALTIME", "valueEquals(id, localtime(__epoch_time__, __time_zone__))");
testFilterExpression(
"id = LOCALTIMESTAMP",
"valueEquals(id, localtimestamp(__epoch_time__, __time_zone__))");
testFilterExpression(
"id = CURRENT_TIME", "valueEquals(id, currentTime(__epoch_time__, __time_zone__))");
testFilterExpression(
"id = CURRENT_DATE", "valueEquals(id, currentDate(__epoch_time__, __time_zone__))");
testFilterExpression(
"id = CURRENT_TIMESTAMP",
"valueEquals(id, currentTimestamp(__epoch_time__, __time_zone__))");
testFilterExpression("NOW()", "now(__epoch_time__, __time_zone__)");
testFilterExpression("YEAR(dt)", "year(dt)");
testFilterExpression("QUARTER(dt)", "quarter(dt)");
testFilterExpression("MONTH(dt)", "month(dt)");
testFilterExpression("WEEK(dt)", "week(dt)");
testFilterExpression("DATE_FORMAT(dt,'yyyy-MM-dd')", "dateFormat(dt, \"yyyy-MM-dd\")");
testFilterExpression("TO_DATE(dt, 'yyyy-MM-dd')", "toDate(dt, \"yyyy-MM-dd\")");
testFilterExpression("TO_TIMESTAMP(dt)", "toTimestamp(dt)");
testFilterExpression("TIMESTAMP_DIFF('DAY', dt1, dt2)", "timestampDiff(\"DAY\", dt1, dt2)");
testFilterExpression("IF(a>b,a,b)", "a > b ? a : b");
testFilterExpression("NULLIF(a,b)", "nullif(a, b)");
testFilterExpression("COALESCE(a,b,c)", "coalesce(a, b, c)");
testFilterExpression("id + 2", "id + 2");
testFilterExpression("id - 2", "id - 2");
testFilterExpression("id * 2", "id * 2");
testFilterExpression("id / 2", "id / 2");
testFilterExpression("id % 2", "id % 2");
testFilterExpression("a < b", "a < b");
testFilterExpression("a <= b", "a <= b");
testFilterExpression("a > b", "a > b");
testFilterExpression("a >= b", "a >= b");
testFilterExpression("__table_name__ = 'tb'", "valueEquals(__table_name__, \"tb\")");
testFilterExpression("__schema_name__ = 'tb'", "valueEquals(__schema_name__, \"tb\")");
testFilterExpression(
"__namespace_name__ = 'tb'", "valueEquals(__namespace_name__, \"tb\")");
testFilterExpression("upper(lower(id))", "upper(lower(id))");
testFilterExpression(
"abs(uniq_id) > 10 and id is not null", "abs(uniq_id) > 10 && null != id");
}
@Test
public void testSqlCall() {
ApiExpression apiExpression = Expressions.concat("1", "2");
ApiExpression substring = apiExpression.substring(1);
System.out.println(substring);
}
private void testFilterExpression(String expression, String expressionExpect) {
String janinoExpression =
TransformParser.translateFilterExpressionToJaninoExpression(expression);
Assert.assertEquals(expressionExpect, janinoExpression);
}
}

@ -53,6 +53,7 @@ public class SchemaSerializerTest extends SerializerTestBase<Schema> {
.metadataColumn("m3", DataTypes.TIMESTAMP_LTZ(), "mKey", "comment")
.option("option", "fake")
.primaryKey(Collections.singletonList("col1"))
.partitionKey(Collections.singletonList("m2"))
.build(),
Schema.newBuilder()
.physicalColumn("col1", DataTypes.BIGINT())

@ -93,6 +93,11 @@ limitations under the License.
<assertj.version>3.24.2</assertj.version>
<markBundledAsOptional>true</markBundledAsOptional>
<flatten-maven-plugin.version>1.5.0</flatten-maven-plugin.version>
<calcite.version>1.32.0</calcite.version>
<!-- Calcite 1.32.0 depends on 3.1.8,
at the same time minimum 3.1.x Janino version passing Flink tests without WAs is 3.1.10,
more details are in FLINK-27995 -->
<janino.version>3.1.10</janino.version>
</properties>
<dependencyManagement>
@ -165,6 +170,125 @@ limitations under the License.
<optional>true</optional>
</dependency>
<dependency>
<!-- Used for code generation -->
<groupId>org.codehaus.janino</groupId>
<artifactId>janino</artifactId>
<version>${janino.version}</version>
</dependency>
<dependency>
<!-- Used for translation of expression -->
<groupId>org.apache.calcite</groupId>
<artifactId>calcite-core</artifactId>
<!-- When updating the Calcite version, make sure to update the dependency exclusions -->
<version>${calcite.version}</version>
<exclusions>
<!--
"mvn dependency:tree" as of Calcite 1.32.0:
[INFO] +- org.apache.calcite:calcite-core:jar:1.32.0:compile
[INFO] | +- org.apache.calcite:calcite-linq4j:jar:1.32.0:compile
[INFO] | +- org.locationtech.jts:jts-core:jar:1.19.0:compile
[INFO] | +- com.fasterxml.jackson.core:jackson-annotations:jar:2.14.3:compile
[INFO] | +- org.apache.calcite.avatica:avatica-core:jar:1.22.0:compile
[INFO] | +- org.apiguardian:apiguardian-api:jar:1.1.2:compile
[INFO] | +- com.fasterxml.jackson.core:jackson-core:jar:2.14.3:compile
[INFO] | +- com.fasterxml.jackson.core:jackson-databind:jar:2.14.3:compile
[INFO] | +- com.jayway.jsonpath:json-path:jar:2.7.0:runtime
[INFO] | | \- net.minidev:json-smart:jar:2.4.7:runtime
[INFO] | | \- net.minidev:accessors-smart:jar:2.4.7:runtime
[INFO] | | \- org.ow2.asm:asm:jar:9.1:runtime
[INFO] | +- commons-codec:commons-codec:jar:1.15:runtime
[INFO] | \- commons-io:commons-io:jar:2.11.0:runtime
Dependencies that are not needed for how we use Calcite right now.
-->
<exclusion>
<groupId>org.apache.calcite.avatica</groupId>
<artifactId>avatica-server</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.calcite.avatica</groupId>
<artifactId>avatica-metrics</artifactId>
</exclusion>
<exclusion>
<groupId>com.google.protobuf</groupId>
<artifactId>protobuf-java</artifactId>
</exclusion>
<exclusion>
<groupId>com.google.uzaygezen</groupId>
<artifactId>uzaygezen-core</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpclient</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpcore</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.commons</groupId>
<artifactId>commons-dbcp2</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
</exclusion>
<exclusion>
<groupId>com.fasterxml.jackson.dataformat</groupId>
<artifactId>jackson-dataformat-yaml</artifactId>
</exclusion>
<exclusion>
<groupId>com.yahoo.datasketches</groupId>
<artifactId>sketches-core</artifactId>
</exclusion>
<exclusion>
<groupId>net.hydromatic</groupId>
<artifactId>aggdesigner-algorithm</artifactId>
</exclusion>
<exclusion>
<groupId>com.google.errorprone</groupId>
<artifactId>error_prone_annotations</artifactId>
</exclusion>
<exclusion>
<groupId>org.jetbrains</groupId>
<artifactId>annotations</artifactId>
</exclusion>
<exclusion>
<groupId>org.jetbrains.kotlin</groupId>
<artifactId>kotlin-stdlib-jdk8</artifactId>
</exclusion>
<exclusion>
<groupId>org.codehaus.janino</groupId>
<artifactId>janino</artifactId>
</exclusion>
<exclusion>
<groupId>org.codehaus.janino</groupId>
<artifactId>commons-compiler</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.httpcomponents.client5</groupId>
<artifactId>httpclient5</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.httpcomponents.core5</groupId>
<artifactId>httpcore5</artifactId>
</exclusion>
<exclusion>
<groupId>org.checkerframework</groupId>
<artifactId>checker-qual</artifactId>
</exclusion>
<exclusion>
<groupId>org.locationtech.jts.io</groupId>
<artifactId>jts-io-common</artifactId>
</exclusion>
<exclusion>
<groupId>org.locationtech.proj4j</groupId>
<artifactId>proj4j</artifactId>
</exclusion>
</exclusions>
</dependency>
<!-- test dependencies -->
<dependency>
<groupId>org.hamcrest</groupId>

Loading…
Cancel
Save