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

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1979,7 +1979,7 @@ void testTransformWithLargeLiterals() throws Exception {
+ "2147483648 AS greater_than_int_max, "
+ "-2147483648 AS int_min, "
+ "-2147483649 AS less_than_int_min, "
+ "CAST(1234567890123456789 AS DECIMAL(20, 0)) AS really_big_decimal",
+ "CAST(1234567890123456789 AS DECIMAL(19, 0)) AS really_big_decimal",
"CAST(id AS BIGINT) + 2147483648 > 2147483649", // equivalent to id > 1
null,
null,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -841,7 +841,7 @@ void testComplicatedFlinkUdf(ValuesDataSink.SinkApi sinkApi, String language) th
@ParameterizedTest
@MethodSource("testParams")
@Disabled("For manual test as there is a limit for quota.")
void testTransformWithModel(ValuesDataSink.SinkApi sinkApi) throws Exception {
void testTransformWithModel(ValuesDataSink.SinkApi sinkApi, String language) throws Exception {
FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster();

// Setup value source
Expand Down Expand Up @@ -909,6 +909,112 @@ void testTransformWithModel(ValuesDataSink.SinkApi sinkApi) throws Exception {
.hasSize(9);
}

@ParameterizedTest
@MethodSource("testParams")
void testComplicatedUdfReturnTypes(ValuesDataSink.SinkApi sinkApi, String language)
throws Exception {
FlinkPipelineComposer composer = FlinkPipelineComposer.ofMiniCluster();

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

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

// Setup transform
TransformDef transformDef =
new TransformDef(
"default_namespace.default_schema.table1",
"*, get_char() AS char_col, get_varchar() AS varchar_col, get_binary() AS binary_col, get_varbinary() AS varbinary_col, get_ts() AS ts_col, get_ts_ltz() AS ts_ltz_col, get_decimal() AS decimal_col, get_non_null() AS non_null_col",
null,
"col1",
null,
"key1=value1",
"",
null);

// Setup pipeline
Configuration pipelineConfig = new Configuration();
pipelineConfig.set(PipelineOptions.PIPELINE_PARALLELISM, 1);
pipelineConfig.set(PipelineOptions.PIPELINE_LOCAL_TIME_ZONE, "America/Los_Angeles");
pipelineConfig.set(
PipelineOptions.PIPELINE_SCHEMA_CHANGE_BEHAVIOR, SchemaChangeBehavior.EVOLVE);
PipelineDef pipelineDef =
new PipelineDef(
sourceDef,
sinkDef,
Collections.emptyList(),
Collections.singletonList(transformDef),
Arrays.asList(
new UdfDef(
"get_char",
String.format(
"org.apache.flink.cdc.udf.examples.%s.precision.CharTypeReturningClass",
language)),
new UdfDef(
"get_varchar",
String.format(
"org.apache.flink.cdc.udf.examples.%s.precision.VarCharTypeReturningClass",
language)),
new UdfDef(
"get_binary",
String.format(
"org.apache.flink.cdc.udf.examples.%s.precision.BinaryTypeReturningClass",
language)),
new UdfDef(
"get_varbinary",
String.format(
"org.apache.flink.cdc.udf.examples.%s.precision.VarBinaryTypeReturningClass",
language)),
new UdfDef(
"get_ts",
String.format(
"org.apache.flink.cdc.udf.examples.%s.precision.TimestampTypeReturningClass",
language)),
new UdfDef(
"get_ts_ltz",
String.format(
"org.apache.flink.cdc.udf.examples.%s.precision.LocalZonedTimestampTypeReturningClass",
language)),
new UdfDef(
"get_decimal",
String.format(
"org.apache.flink.cdc.udf.examples.%s.precision.DecimalTypeReturningClass",
language)),
new UdfDef(
"get_non_null",
String.format(
"org.apache.flink.cdc.udf.examples.%s.precision.DecimalTypeNonNullReturningClass",
language))),
pipelineConfig);

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

// Check the order and content of all received events
String[] outputEvents = outCaptor.toString().trim().split("\n");
assertThat(outputEvents)
.contains(
"CreateTableEvent{tableId=default_namespace.default_schema.table1, schema=columns={`col1` STRING NOT NULL,`col2` STRING,`char_col` STRING,`varchar_col` STRING,`binary_col` BINARY(17),`varbinary_col` VARBINARY(17),`ts_col` TIMESTAMP(2),`ts_ltz_col` TIMESTAMP_LTZ(2),`decimal_col` DECIMAL(10, 3),`non_null_col` DECIMAL(10, 3)}, primaryKeys=col1, options=({key1=value1})}",
"DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[1, 1, This is a string., This is a string., eHl6enk=, eHl6enk=, 1970-01-02T00:00, 1970-01-02T00:00, 12.315, 12.315], op=INSERT, meta=()}",
"DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[2, 2, This is a string., This is a string., eHl6enk=, eHl6enk=, 1970-01-02T00:00, 1970-01-02T00:00, 12.315, 12.315], op=INSERT, meta=()}",
"DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[], after=[3, 3, This is a string., This is a string., eHl6enk=, eHl6enk=, 1970-01-02T00:00, 1970-01-02T00:00, 12.315, 12.315], op=INSERT, meta=()}",
"AddColumnEvent{tableId=default_namespace.default_schema.table1, addedColumns=[ColumnWithPosition{column=`col3` STRING, position=AFTER, existedColumnName=col2}]}",
"RenameColumnEvent{tableId=default_namespace.default_schema.table1, nameMapping={col2=newCol2, col3=newCol3}}",
"DropColumnEvent{tableId=default_namespace.default_schema.table1, droppedColumnNames=[newCol2]}",
"DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[1, 1, This is a string., This is a string., eHl6enk=, eHl6enk=, 1970-01-02T00:00, 1970-01-02T00:00, 12.315, 12.315], after=[], op=DELETE, meta=()}",
"DataChangeEvent{tableId=default_namespace.default_schema.table1, before=[2, , This is a string., This is a string., eHl6enk=, eHl6enk=, 1970-01-02T00:00, 1970-01-02T00:00, 12.315, 12.315], after=[2, x, This is a string., This is a string., eHl6enk=, eHl6enk=, 1970-01-02T00:00, 1970-01-02T00:00, 12.315, 12.315], op=UPDATE, meta=()}");
}

private static Stream<Arguments> testParams() {
return Stream.of(
arguments(ValuesDataSink.SinkApi.SINK_FUNCTION, "java"),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,10 +67,13 @@ private static List<Object> getFields(
return fields.stream()
.map(
o -> {
if (o == null) {
return "null";
}
if (o instanceof byte[]) {
return BaseEncoding.base64().encode((byte[]) o);
} else {
return o;
return o.toString();
}
})
.collect(Collectors.toList());
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* 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.udf.examples.java.precision;

import org.apache.flink.cdc.common.types.DataType;
import org.apache.flink.cdc.common.types.DataTypes;
import org.apache.flink.cdc.common.udf.UserDefinedFunction;

/** This is an example UDF class for testing purposes only. */
public class BinaryTypeReturningClass implements UserDefinedFunction {
@Override
public DataType getReturnType() {
return DataTypes.BINARY(17);
}

public byte[] eval() {
return "xyzzy".getBytes();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
/*
* 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.udf.examples.java.precision;

import org.apache.flink.cdc.common.types.DataType;
import org.apache.flink.cdc.common.types.DataTypes;
import org.apache.flink.cdc.common.udf.UserDefinedFunction;

/** This is an example UDF class for testing purposes only. */
public class CharTypeReturningClass implements UserDefinedFunction {

@Override
public DataType getReturnType() {
return DataTypes.CHAR(17);
}

public String eval() {
return "This is a string.";
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
/*
* 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.udf.examples.java.precision;

import org.apache.flink.cdc.common.data.DecimalData;
import org.apache.flink.cdc.common.types.DataType;
import org.apache.flink.cdc.common.types.DataTypes;
import org.apache.flink.cdc.common.udf.UserDefinedFunction;

import java.math.BigDecimal;

/** This is an example UDF class for testing purposes only. */
public class DecimalTypeNonNullReturningClass implements UserDefinedFunction {
@Override
public DataType getReturnType() {
return DataTypes.DECIMAL(10, 3).notNull();
}

public DecimalData eval() {
return DecimalData.fromBigDecimal(new BigDecimal("12.315"), 10, 3);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
/*
* 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.udf.examples.java.precision;

import org.apache.flink.cdc.common.data.DecimalData;
import org.apache.flink.cdc.common.types.DataType;
import org.apache.flink.cdc.common.types.DataTypes;
import org.apache.flink.cdc.common.udf.UserDefinedFunction;

import java.math.BigDecimal;

/** This is an example UDF class for testing purposes only. */
public class DecimalTypeReturningClass implements UserDefinedFunction {
@Override
public DataType getReturnType() {
return DataTypes.DECIMAL(10, 3);
}

public DecimalData eval() {
return DecimalData.fromBigDecimal(new BigDecimal("12.315"), 10, 3);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
/*
* 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.udf.examples.java.precision;

import org.apache.flink.cdc.common.data.LocalZonedTimestampData;
import org.apache.flink.cdc.common.types.DataType;
import org.apache.flink.cdc.common.types.DataTypes;
import org.apache.flink.cdc.common.udf.UserDefinedFunction;

/** This is an example UDF class for testing purposes only. */
public class LocalZonedTimestampTypeReturningClass implements UserDefinedFunction {
@Override
public DataType getReturnType() {
return DataTypes.TIMESTAMP_LTZ(2);
}

public LocalZonedTimestampData eval() {
return LocalZonedTimestampData.fromEpochMillis(24 * 60 * 60 * 1000);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
/*
* 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.udf.examples.java.precision;

import org.apache.flink.cdc.common.data.TimestampData;
import org.apache.flink.cdc.common.types.DataType;
import org.apache.flink.cdc.common.types.DataTypes;
import org.apache.flink.cdc.common.udf.UserDefinedFunction;

/** This is an example UDF class for testing purposes only. */
public class TimestampTypeReturningClass implements UserDefinedFunction {
@Override
public DataType getReturnType() {
return DataTypes.TIMESTAMP(2);
}

public TimestampData eval() {
return TimestampData.fromMillis(24 * 60 * 60 * 1000);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* 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.udf.examples.java.precision;

import org.apache.flink.cdc.common.types.DataType;
import org.apache.flink.cdc.common.types.DataTypes;
import org.apache.flink.cdc.common.udf.UserDefinedFunction;

/** This is an example UDF class for testing purposes only. */
public class VarBinaryTypeReturningClass implements UserDefinedFunction {
@Override
public DataType getReturnType() {
return DataTypes.VARBINARY(17);
}

public byte[] eval() {
return "xyzzy".getBytes();
}
}
Loading
Loading