Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

[Feature][Flink] Support Decimal Type with configurable precision and scale #5419

Merged
merged 1 commit into from
Sep 25, 2023
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
1 change: 1 addition & 0 deletions release-note.md
Original file line number Diff line number Diff line change
Expand Up @@ -147,6 +147,7 @@
- [Core] [API] Add copy method to Catalog codes (#4414)
- [Core] [API] Add options check before create source and sink and transform in FactoryUtil (#4424)
- [Core] [Shade] Add guava shade module (#4358)
- [Core] [Flink] Support Decimal Type with configurable precision and scale (#5419)

### Connector-V2

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.apache.seatunnel.shade.com.typesafe.config.Config;

import org.apache.seatunnel.api.env.EnvCommonOptions;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
import org.apache.seatunnel.common.Constants;
import org.apache.seatunnel.common.config.CheckResult;
import org.apache.seatunnel.common.constants.JobMode;
Expand Down Expand Up @@ -51,8 +52,11 @@

import java.net.URL;
import java.util.ArrayList;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.stream.Collectors;

@Slf4j
Expand All @@ -64,7 +68,8 @@ public class FlinkRuntimeEnvironment implements RuntimeEnvironment {
private StreamExecutionEnvironment environment;

private StreamTableEnvironment tableEnvironment;

private Map<String, SeaTunnelRowType> stagedTypes = new LinkedHashMap<>();
private Optional<SeaTunnelRowType> defaultType = Optional.empty();
private JobMode jobMode;

private String jobName = Constants.LOGO;
Expand Down Expand Up @@ -334,6 +339,24 @@ public void registerResultTable(
name, tableEnvironment.fromChangelogStream(dataStream));
}

public void stageType(String tblName, SeaTunnelRowType type) {
stagedTypes.put(tblName, type);
}

public void stageDefaultType(SeaTunnelRowType type) {
this.defaultType = Optional.of(type);
}

public Optional<SeaTunnelRowType> type(String tblName) {
return stagedTypes.containsKey(tblName)
? Optional.of(stagedTypes.get(tblName))
: Optional.empty();
}

public Optional<SeaTunnelRowType> defaultType() {
return this.defaultType;
}

public static FlinkRuntimeEnvironment getInstance(Config config) {
if (INSTANCE == null) {
synchronized (FlinkRuntimeEnvironment.class) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@
import org.apache.seatunnel.plugin.discovery.PluginIdentifier;
import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSinkPluginDiscovery;
import org.apache.seatunnel.translation.flink.sink.FlinkSink;
import org.apache.seatunnel.translation.flink.utils.TypeConverterUtils;

import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSink;
Expand Down Expand Up @@ -101,8 +100,8 @@ public List<DataStream<Row>> execute(List<DataStream<Row>> upstreamDataStreams)
SeaTunnelSink<SeaTunnelRow, Serializable, Serializable, Serializable> seaTunnelSink =
plugins.get(i);
DataStream<Row> stream = fromSourceTable(sinkConfig).orElse(input);
seaTunnelSink.setTypeInfo(
(SeaTunnelRowType) TypeConverterUtils.convert(stream.getType()));
SeaTunnelRowType sourceType = initSourceType(sinkConfig, stream);
seaTunnelSink.setTypeInfo(sourceType);
if (SupportDataSaveMode.class.isAssignableFrom(seaTunnelSink.getClass())) {
SupportDataSaveMode saveModeSink = (SupportDataSaveMode) seaTunnelSink;
DataSaveMode dataSaveMode = saveModeSink.getUserConfigSaveMode();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,11 @@
import org.apache.seatunnel.shade.com.typesafe.config.Config;

import org.apache.seatunnel.api.common.JobContext;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
import org.apache.seatunnel.common.utils.ReflectionUtils;
import org.apache.seatunnel.core.starter.execution.PluginExecuteProcessor;
import org.apache.seatunnel.core.starter.flink.utils.TableUtil;
import org.apache.seatunnel.translation.flink.utils.TypeConverterUtils;

import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.table.api.Table;
Expand Down Expand Up @@ -117,6 +119,36 @@ protected void registerAppendStream(Config pluginConfig) {
}
}

protected void stageType(Config pluginConfig, SeaTunnelRowType type) {
if (!flinkRuntimeEnvironment.defaultType().isPresent()) {
flinkRuntimeEnvironment.stageDefaultType(type);
}

if (pluginConfig.hasPath("result_table_name")) {
String tblName = pluginConfig.getString("result_table_name");
flinkRuntimeEnvironment.stageType(tblName, type);
}
}

protected Optional<SeaTunnelRowType> sourceType(Config pluginConfig) {
if (pluginConfig.hasPath(SOURCE_TABLE_NAME)) {
String tblName = pluginConfig.getString(SOURCE_TABLE_NAME);
return flinkRuntimeEnvironment.type(tblName);
} else {
return flinkRuntimeEnvironment.defaultType();
}
}

protected SeaTunnelRowType initSourceType(Config sinkConfig, DataStream<Row> stream) {
SeaTunnelRowType sourceType =
sourceType(sinkConfig)
.orElseGet(
() ->
(SeaTunnelRowType)
TypeConverterUtils.convert(stream.getType()));
return sourceType;
}

protected abstract List<T> initializePlugins(
List<URL> jarPaths, List<? extends Config> pluginConfigs);
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.apache.seatunnel.shade.com.typesafe.config.Config;

import org.apache.seatunnel.api.env.EnvCommonOptions;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
import org.apache.seatunnel.common.Constants;
import org.apache.seatunnel.common.config.CheckResult;
import org.apache.seatunnel.common.constants.JobMode;
Expand Down Expand Up @@ -51,8 +52,11 @@

import java.net.URL;
import java.util.ArrayList;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.stream.Collectors;

@Slf4j
Expand All @@ -65,6 +69,9 @@ public class FlinkRuntimeEnvironment implements RuntimeEnvironment {

private StreamTableEnvironment tableEnvironment;

private Map<String, SeaTunnelRowType> stagedTypes = new LinkedHashMap<>();
private Optional<SeaTunnelRowType> defaultType = Optional.empty();

private JobMode jobMode;

private String jobName = Constants.LOGO;
Expand Down Expand Up @@ -334,6 +341,24 @@ public void registerResultTable(
name, tableEnvironment.fromChangelogStream(dataStream));
}

public void stageType(String tblName, SeaTunnelRowType type) {
stagedTypes.put(tblName, type);
}

public void stageDefaultType(SeaTunnelRowType type) {
this.defaultType = Optional.of(type);
}

public Optional<SeaTunnelRowType> type(String tblName) {
return stagedTypes.containsKey(tblName)
? Optional.of(stagedTypes.get(tblName))
: Optional.empty();
}

public Optional<SeaTunnelRowType> defaultType() {
return this.defaultType;
}

public static FlinkRuntimeEnvironment getInstance(Config config) {
if (INSTANCE == null) {
synchronized (FlinkRuntimeEnvironment.class) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,6 @@
import org.apache.seatunnel.plugin.discovery.PluginIdentifier;
import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSinkPluginDiscovery;
import org.apache.seatunnel.translation.flink.sink.FlinkSink;
import org.apache.seatunnel.translation.flink.utils.TypeConverterUtils;

import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSink;
Expand Down Expand Up @@ -102,8 +101,8 @@ public List<DataStream<Row>> execute(List<DataStream<Row>> upstreamDataStreams)
SeaTunnelSink<SeaTunnelRow, Serializable, Serializable, Serializable> seaTunnelSink =
plugins.get(i);
DataStream<Row> stream = fromSourceTable(sinkConfig).orElse(input);
seaTunnelSink.setTypeInfo(
(SeaTunnelRowType) TypeConverterUtils.convert(stream.getType()));
SeaTunnelRowType sourceType = initSourceType(sinkConfig, stream);
seaTunnelSink.setTypeInfo(sourceType);
if (SupportDataSaveMode.class.isAssignableFrom(seaTunnelSink.getClass())) {
SupportDataSaveMode saveModeSink = (SupportDataSaveMode) seaTunnelSink;
DataSaveMode dataSaveMode = saveModeSink.getUserConfigSaveMode();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.seatunnel.api.common.JobContext;
import org.apache.seatunnel.api.source.SeaTunnelSource;
import org.apache.seatunnel.api.source.SupportCoordinate;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
import org.apache.seatunnel.common.constants.JobMode;
import org.apache.seatunnel.core.starter.enums.PluginType;
import org.apache.seatunnel.plugin.discovery.PluginIdentifier;
Expand Down Expand Up @@ -76,12 +77,15 @@ public List<DataStream<Row>> execute(List<DataStream<Row>> upstreamDataStreams)
boolean bounded =
internalSource.getBoundedness()
== org.apache.seatunnel.api.source.Boundedness.BOUNDED;

DataStreamSource<Row> sourceStream =
addSource(
executionEnvironment,
sourceFunction,
"SeaTunnel " + internalSource.getClass().getSimpleName(),
bounded);
stageType(pluginConfig, (SeaTunnelRowType) internalSource.getProducedType());

if (pluginConfig.hasPath(CommonOptions.PARALLELISM.key())) {
int parallelism = pluginConfig.getInt(CommonOptions.PARALLELISM.key());
sourceStream.setParallelism(parallelism);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,8 @@
import org.apache.seatunnel.shade.com.typesafe.config.Config;

import org.apache.seatunnel.api.common.JobContext;
import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
import org.apache.seatunnel.api.transform.SeaTunnelTransform;
import org.apache.seatunnel.core.starter.exception.TaskExecuteException;
import org.apache.seatunnel.plugin.discovery.PluginIdentifier;
Expand Down Expand Up @@ -97,7 +97,10 @@ public List<DataStream<Row>> execute(List<DataStream<Row>> upstreamDataStreams)
SeaTunnelTransform<SeaTunnelRow> transform = plugins.get(i);
Config pluginConfig = pluginConfigs.get(i);
DataStream<Row> stream = fromSourceTable(pluginConfig).orElse(input);
input = flinkTransform(transform, stream);
SeaTunnelRowType sourceType = initSourceType(pluginConfig, stream);
transform.setTypeInfo(sourceType);
input = flinkTransform(sourceType, transform, stream);
stageType(pluginConfig, (SeaTunnelRowType) transform.getProducedType());
registerResultTable(pluginConfig, input);
result.add(input);
} catch (Exception e) {
Expand All @@ -111,11 +114,10 @@ public List<DataStream<Row>> execute(List<DataStream<Row>> upstreamDataStreams)
return result;
}

protected DataStream<Row> flinkTransform(SeaTunnelTransform transform, DataStream<Row> stream) {
SeaTunnelDataType seaTunnelDataType = TypeConverterUtils.convert(stream.getType());
transform.setTypeInfo(seaTunnelDataType);
protected DataStream<Row> flinkTransform(
SeaTunnelRowType sourceType, SeaTunnelTransform transform, DataStream<Row> stream) {
TypeInformation rowTypeInfo = TypeConverterUtils.convert(transform.getProducedType());
FlinkRowConverter transformInputRowConverter = new FlinkRowConverter(seaTunnelDataType);
FlinkRowConverter transformInputRowConverter = new FlinkRowConverter(sourceType);
FlinkRowConverter transformOutputRowConverter =
new FlinkRowConverter(transform.getProducedType());
DataStream<Row> output =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ env {

source {
FakeSource {
row.num = 100000
schema = {
fields {
c_map = "map<string, string>"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ sink {
row_rules = [
{
rule_type = MAX_ROW
rule_value = 5
rule_value = 100000
}
],
field_rules = [
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,9 +83,14 @@ public void convertShortType() {

@Test
public void convertBigDecimalType() {
Assertions.assertEquals(
BasicTypeInfo.BIG_DEC_TYPE_INFO,
TypeConverterUtils.convert(new DecimalType(30, 2)));
/**
* To solve lost precision and scale of {@link
* org.apache.seatunnel.api.table.type.DecimalType}, use {@link
* org.apache.flink.api.common.typeinfo.BasicTypeInfo#STRING_TYPE_INFO} as the convert
* result of {@link org.apache.seatunnel.api.table.type.DecimalType} instance.
*/
Assertions.assertEquals(
BasicTypeInfo.STRING_TYPE_INFO, TypeConverterUtils.convert(new DecimalType(30, 2)));
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.seatunnel.translation.flink.serialization;

import org.apache.seatunnel.api.table.type.DecimalType;
import org.apache.seatunnel.api.table.type.MapType;
import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
Expand All @@ -28,6 +29,8 @@
import org.apache.flink.types.RowKind;

import java.io.IOException;
import java.math.BigDecimal;
import java.math.RoundingMode;
import java.util.HashMap;
import java.util.Map;
import java.util.function.BiFunction;
Expand Down Expand Up @@ -68,6 +71,15 @@ private static Object convert(Object field, SeaTunnelDataType<?> dataType) {
case MAP:
return convertMap(
(Map<?, ?>) field, (MapType<?, ?>) dataType, FlinkRowConverter::convert);

/**
* To solve lost precision and scale of {@link
* org.apache.seatunnel.api.table.type.DecimalType}, use {@link java.lang.String} as
* the convert result of {@link java.math.BigDecimal} instance.
*/
case DECIMAL:
BigDecimal decimal = (BigDecimal) field;
return decimal.toString();
default:
return field;
}
Expand Down Expand Up @@ -122,6 +134,18 @@ private static Object reconvert(Object field, SeaTunnelDataType<?> dataType) {
case MAP:
return convertMap(
(Map<?, ?>) field, (MapType<?, ?>) dataType, FlinkRowConverter::reconvert);

/**
* To solve lost precision and scale of {@link
* org.apache.seatunnel.api.table.type.DecimalType}, create {@link
* java.math.BigDecimal} instance from {@link java.lang.String} type field.
*/
case DECIMAL:
DecimalType decimalType = (DecimalType) dataType;
String decimalData = (String) field;
BigDecimal decimal = new BigDecimal(decimalData);
decimal.setScale(decimalType.getScale(), RoundingMode.HALF_UP);
return decimal;
default:
return field;
}
Expand Down
Loading
Loading