Skip to content
Open
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 docs/docs/flink-writes.md
Original file line number Diff line number Diff line change
Expand Up @@ -550,6 +550,7 @@ The Dynamic Iceberg Flink Sink is configured using the Builder pattern. Here are
| `setAll(Map<String, String> properties)` | Set multiple properties at once |
| `tableCreator(TableCreator creator)` | When DynamicIcebergSink creates new Iceberg tables, allows overriding how tables are created - setting custom table properties and location based on the table name. |
| `dropUnusedColumns(boolean enabled)` | When enabled, drops all columns from the current table schema which are not contained in the input schema (see the caveats above on dropping columns). |
| `taskWriterFactoryProvider(DynamicTaskWriterFactoryProvider provider)` | Supply a custom `TaskWriterFactory<RowData>` per write target in place of the default `RowDataTaskWriterFactory`. Useful for throughput-oriented writers that exploit known data pattern, audit/metric layering, custom file naming/layout/properties. |

### Distribution Modes

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,7 @@ public class DynamicIcebergSink
private final Map<String, String> writeProperties;
private final Configuration flinkConfig;
private final int cacheMaximumSize;
private final DynamicTaskWriterFactoryProvider taskWriterFactoryProvider;

// Set by the builder before sinkTo() — forward writer results to union into pre-commit topology
private final transient DataStream<CommittableMessage<DynamicWriteResult>> forwardWriteResults;
Expand All @@ -90,13 +91,15 @@ public class DynamicIcebergSink
Map<String, String> writeProperties,
Configuration flinkConfig,
int cacheMaximumSize,
DynamicTaskWriterFactoryProvider taskWriterFactoryProvider,
DataStream<CommittableMessage<DynamicWriteResult>> forwardWriteResults) {
this.catalogLoader = catalogLoader;
this.snapshotProperties = snapshotProperties;
this.uidPrefix = uidPrefix;
this.writeProperties = writeProperties;
this.flinkConfig = flinkConfig;
this.cacheMaximumSize = cacheMaximumSize;
this.taskWriterFactoryProvider = taskWriterFactoryProvider;
// We generate a random UUID every time when a sink is created.
// This is used to separate files generated by different sinks writing the same table.
// Also used to generate the aggregator operator name
Expand All @@ -113,7 +116,8 @@ public SinkWriter<DynamicRecordInternal> createWriter(WriterInitContext context)
cacheMaximumSize,
new DynamicWriterMetrics(context.metricGroup()),
context.getTaskInfo().getIndexOfThisSubtask(),
context.getTaskInfo().getAttemptNumber());
context.getTaskInfo().getAttemptNumber(),
taskWriterFactoryProvider);
}

@Override
Expand Down Expand Up @@ -190,16 +194,19 @@ static class ForwardWriterSink
private final Map<String, String> writeProperties;
private final Configuration flinkConfig;
private final int cacheMaximumSize;
private final DynamicTaskWriterFactoryProvider taskWriterFactoryProvider;

ForwardWriterSink(
CatalogLoader catalogLoader,
Map<String, String> writeProperties,
Configuration flinkConfig,
int cacheMaximumSize) {
int cacheMaximumSize,
DynamicTaskWriterFactoryProvider taskWriterFactoryProvider) {
this.catalogLoader = catalogLoader;
this.writeProperties = writeProperties;
this.flinkConfig = flinkConfig;
this.cacheMaximumSize = cacheMaximumSize;
this.taskWriterFactoryProvider = taskWriterFactoryProvider;
}

@Override
Expand All @@ -211,7 +218,8 @@ public SinkWriter<DynamicRecordInternal> createWriter(WriterInitContext context)
cacheMaximumSize,
new DynamicWriterMetrics(context.metricGroup()),
context.getTaskInfo().getIndexOfThisSubtask(),
context.getTaskInfo().getAttemptNumber());
context.getTaskInfo().getAttemptNumber(),
taskWriterFactoryProvider);
}

@Override
Expand All @@ -235,6 +243,8 @@ public static class Builder<T> {
private final Map<String, String> snapshotSummary = Maps.newHashMap();
private ReadableConfig readableConfig = new Configuration();
private TableCreator tableCreator = TableCreator.DEFAULT;
private DynamicTaskWriterFactoryProvider taskWriterFactoryProvider =
DynamicTaskWriterFactoryProvider.DEFAULT;

Builder() {}

Expand Down Expand Up @@ -413,6 +423,18 @@ public Builder<T> caseSensitive(boolean newCaseSensitive) {
return this;
}

/**
* Provide a custom {@link DynamicTaskWriterFactoryProvider} used to build the underlying {@link
* org.apache.iceberg.flink.sink.TaskWriterFactory} for each write target. When omitted, {@link
* DynamicTaskWriterFactoryProvider#DEFAULT} is used, which produces the standard {@link
* org.apache.iceberg.flink.sink.RowDataTaskWriterFactory}.
*/
public Builder<T> taskWriterFactoryProvider(DynamicTaskWriterFactoryProvider provider) {
Preconditions.checkNotNull(provider, "Task writer factory provider shouldn't be null");
this.taskWriterFactoryProvider = provider;
return this;
}

private String operatorName(String suffix) {
return uidPrefix != null ? uidPrefix + "-" + suffix : suffix;
}
Expand All @@ -432,7 +454,11 @@ private DynamicIcebergSink build(
// Forward writer: chained with generator via forward edge, no data shuffle
ForwardWriterSink forwardWriterSink =
new ForwardWriterSink(
catalogLoader, writeOptions, flinkConfig, flinkDynamicSinkConf.cacheMaxSize());
catalogLoader,
writeOptions,
flinkConfig,
flinkDynamicSinkConf.cacheMaxSize(),
taskWriterFactoryProvider);
TypeInformation<CommittableMessage<DynamicWriteResult>> writeResultTypeInfo =
CommittableMessageTypeInfo.of(DynamicWriteResultSerializer::new);

Expand Down Expand Up @@ -465,6 +491,7 @@ DynamicIcebergSink instantiateSink(
writeProperties,
flinkWriteConf,
flinkDynamicSinkConf.cacheMaxSize(),
taskWriterFactoryProvider,
forwardWriteResults);
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
/*
* 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.iceberg.flink.sink.dynamic;

import java.io.Serializable;
import java.util.List;
import java.util.Map;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.logical.RowType;
import org.apache.iceberg.FileFormat;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.Table;
import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory;
import org.apache.iceberg.flink.sink.TaskWriterFactory;

/**
* Pluggable provider that creates a {@link TaskWriterFactory} for a given write target. {@link
* DynamicWriter} resolves the table, schema, partition spec, write properties and equality fields
* before delegating final factory construction here, allowing callers to swap in a custom
* implementation while keeping the surrounding validation and configuration logic intact.
*/
@FunctionalInterface
public interface DynamicTaskWriterFactoryProvider extends Serializable {

TaskWriterFactory<RowData> create(
Table table,
RowType flinkSchema,
long targetFileSizeBytes,
FileFormat format,
Map<String, String> writeProperties,
List<Integer> equalityFieldIds,
boolean upsertMode,
Schema schema,
PartitionSpec spec);

DynamicTaskWriterFactoryProvider DEFAULT =
(table,
flinkSchema,
targetFileSizeBytes,
format,
writeProperties,
equalityFieldIds,
upsertMode,
schema,
spec) ->
new RowDataTaskWriterFactory(
() -> table,
flinkSchema,
targetFileSizeBytes,
format,
writeProperties,
equalityFieldIds,
upsertMode,
schema,
spec);
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,8 @@
import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.flink.FlinkSchemaUtil;
import org.apache.iceberg.flink.FlinkWriteConf;
import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory;
import org.apache.iceberg.flink.sink.SinkUtil;
import org.apache.iceberg.flink.sink.TaskWriterFactory;
import org.apache.iceberg.io.TaskWriter;
import org.apache.iceberg.io.WriteResult;
import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
Expand All @@ -56,14 +56,15 @@ class DynamicWriter implements CommittingSinkWriter<DynamicRecordInternal, Dynam

private static final Logger LOG = LoggerFactory.getLogger(DynamicWriter.class);

private final Map<WriteTarget, RowDataTaskWriterFactory> taskWriterFactories;
private final Map<WriteTarget, TaskWriterFactory<RowData>> taskWriterFactories;
private final Map<WriteTarget, TaskWriter<RowData>> writers;
private final Configuration flinkConfig;
private final Map<String, String> commonWriteProperties;
private final DynamicWriterMetrics metrics;
private final int subTaskId;
private final int attemptId;
private final Catalog catalog;
private final DynamicTaskWriterFactoryProvider taskWriterFactoryProvider;

DynamicWriter(
Catalog catalog,
Expand All @@ -72,13 +73,15 @@ class DynamicWriter implements CommittingSinkWriter<DynamicRecordInternal, Dynam
int cacheMaximumSize,
DynamicWriterMetrics metrics,
int subTaskId,
int attemptId) {
int attemptId,
DynamicTaskWriterFactoryProvider taskWriterFactoryProvider) {
this.catalog = catalog;
this.commonWriteProperties = commonWriteProperties;
this.flinkConfig = flinkConfig;
this.metrics = metrics;
this.subTaskId = subTaskId;
this.attemptId = attemptId;
this.taskWriterFactoryProvider = taskWriterFactoryProvider;
this.taskWriterFactories = new LRUCache<>(cacheMaximumSize);
this.writers = Maps.newHashMap();

Expand All @@ -98,7 +101,7 @@ public void write(DynamicRecordInternal element, Context context)
element.upsertMode(),
element.equalityFields()),
writerKey -> {
RowDataTaskWriterFactory taskWriterFactory =
TaskWriterFactory<RowData> taskWriterFactory =
taskWriterFactories.computeIfAbsent(
writerKey,
factoryKey -> {
Expand Down Expand Up @@ -130,8 +133,8 @@ public void write(DynamicRecordInternal element, Context context)
flinkWriteConf.dataFileFormat(), flinkWriteConf, table);

LOG.debug("Creating new writer factory for table '{}'", table.name());
return new RowDataTaskWriterFactory(
() -> table,
return taskWriterFactoryProvider.create(
table,
FlinkSchemaUtil.convert(element.schema()),
flinkWriteConf.targetDataFileSize(),
flinkWriteConf.dataFileFormat(),
Expand Down Expand Up @@ -217,7 +220,7 @@ DynamicWriterMetrics getMetrics() {
}

@VisibleForTesting
Map<WriteTarget, RowDataTaskWriterFactory> getTaskWriterFactories() {
Map<WriteTarget, TaskWriterFactory<RowData>> getTaskWriterFactories() {
return taskWriterFactories;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1644,6 +1644,7 @@ static class CommitHookDynamicIcebergSink extends DynamicIcebergSink {
writeProperties,
flinkConfig,
100,
DynamicTaskWriterFactoryProvider.DEFAULT,
forwardWritten);
this.commitHook = commitHook;
this.overwriteMode = new FlinkWriteConf(writeProperties, flinkConfig).overwriteMode();
Expand Down
Loading