[cdc-runtime] Introduce TransformSchemaOperator and TransformDataOperator to support transformation
parent
c92903016d
commit
bcad5d9d11
@ -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));
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue