Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

[flink][cdc] Add operator and transformation name for cdc pipeline #2135

Merged
merged 2 commits into from
Oct 17, 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
Original file line number Diff line number Diff line change
Expand Up @@ -173,7 +173,8 @@ public void build(StreamExecutionEnvironment env) throws Exception {
new FlinkCdcSyncDatabaseSinkBuilder<RichCdcMultiplexRecord>()
.withInput(
env.fromSource(source, WatermarkStrategy.noWatermarks(), "Kafka Source")
.flatMap(recordParser))
.flatMap(recordParser)
.name("Parse"))
.withParserFactory(parserFactory)
.withCatalogLoader(catalogLoader())
.withDatabase(database)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,10 @@ public void processElement(T raw, Context context, Collector<Void> collector) th
try {
catalog.createTable(identifier, schema, true);
} catch (Exception e) {
LOG.error("create newly added paimon table error.", e);
LOG.error(
"Cannot create newly added Paimon table {}",
identifier.getFullName(),
e);
}
});

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import org.apache.paimon.catalog.Identifier;
import org.apache.paimon.flink.FlinkConnectorOptions;
import org.apache.paimon.flink.action.MultiTablesSinkMode;
import org.apache.paimon.flink.sink.FlinkStreamPartitioner;
import org.apache.paimon.flink.utils.SingleOutputStreamOperatorUtils;
import org.apache.paimon.options.MemorySize;
import org.apache.paimon.options.Options;
Expand All @@ -33,7 +32,6 @@

import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
import org.apache.flink.streaming.api.transformations.PartitionTransformation;

import javax.annotation.Nullable;

Expand Down Expand Up @@ -140,6 +138,7 @@ private void buildCombinedCdcSink() {
.process(
new CdcDynamicTableParsingProcessFunction<>(
database, catalogLoader, parserFactory))
.name("Side Output")
.setParallelism(input.getParallelism());

// for newly-added tables, create a multiplexing operator that handles all their records
Expand All @@ -151,23 +150,18 @@ private void buildCombinedCdcSink() {
SingleOutputStreamOperatorUtils.getSideOutput(
parsed,
CdcDynamicTableParsingProcessFunction.DYNAMIC_SCHEMA_CHANGE_OUTPUT_TAG)
.process(new MultiTableUpdatedDataFieldsProcessFunction(catalogLoader));
.process(new MultiTableUpdatedDataFieldsProcessFunction(catalogLoader))
.name("Schema Evolution");

FlinkStreamPartitioner<CdcMultiplexRecord> partitioner =
new FlinkStreamPartitioner<>(
new CdcMultiplexRecordChannelComputer(catalogLoader, dynamicOptions));
PartitionTransformation<CdcMultiplexRecord> partitioned =
new PartitionTransformation<>(
newlyAddedTableStream.getTransformation(), partitioner);

if (parallelism != null) {
partitioned.setParallelism(parallelism);
}
DataStream<CdcMultiplexRecord> partitioned =
partition(
newlyAddedTableStream,
new CdcMultiplexRecordChannelComputer(catalogLoader, dynamicOptions),
parallelism);

FlinkCdcMultiTableSink sink =
new FlinkCdcMultiTableSink(catalogLoader, committerCpu, committerMemory);
sink.sinkFrom(
new DataStream<>(input.getExecutionEnvironment(), partitioned), dynamicOptions);
sink.sinkFrom(partitioned, dynamicOptions);
}

private void buildForFixedBucket(FileStoreTable table, DataStream<CdcRecord> parsed) {
Expand Down
Loading