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
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import java.util.Set;
import java.util.UUID;
import java.util.function.Function;
import javax.annotation.Nullable;
import org.apache.commons.lang3.StringUtils;
import org.apache.flink.annotation.Experimental;
import org.apache.flink.api.common.SupportsConcurrentExecutionAttempts;
Expand Down Expand Up @@ -171,6 +172,8 @@ public class IcebergSink
// equalityFieldIds instead.
private final Set<String> equalityFieldColumns;

@Nullable private final OutputFileFactoryProvider outputFileFactoryProvider;

private final transient List<MaintenanceTaskBuilder<?>> maintenanceTasks;
private final transient FlinkMaintenanceConfig flinkMaintenanceConfig;

Expand All @@ -188,7 +191,8 @@ private IcebergSink(
boolean overwriteMode,
List<MaintenanceTaskBuilder<?>> maintenanceTasks,
FlinkMaintenanceConfig flinkMaintenanceConfig,
Set<String> equalityFieldColumns) {
Set<String> equalityFieldColumns,
@Nullable OutputFileFactoryProvider outputFileFactoryProvider) {
this.tableLoader = tableLoader;
this.snapshotProperties = snapshotProperties;
this.uidSuffix = uidSuffix;
Expand All @@ -212,6 +216,7 @@ private IcebergSink(
this.maintenanceTasks = maintenanceTasks;
this.flinkMaintenanceConfig = flinkMaintenanceConfig;
this.equalityFieldColumns = equalityFieldColumns;
this.outputFileFactoryProvider = outputFileFactoryProvider;
}

@Override
Expand All @@ -224,7 +229,10 @@ public SinkWriter<RowData> createWriter(WriterInitContext context) {
dataFileFormat,
writeProperties,
equalityFieldIds,
upsertMode);
upsertMode,
table.schema(),
table.spec(),
outputFileFactoryProvider);
IcebergStreamWriterMetrics metrics =
new IcebergStreamWriterMetrics(context.metricGroup(), table.name());
return new IcebergSinkWriter(
Expand Down Expand Up @@ -349,6 +357,7 @@ public static class Builder implements IcebergSinkBuilder<Builder> {
private ReadableConfig readableConfig = new Configuration();
private List<String> equalityFieldColumns = null;
private final List<MaintenanceTaskBuilder<?>> maintenanceTasks = Lists.newArrayList();
@Nullable private OutputFileFactoryProvider outputFileFactoryProvider;

private Builder() {}

Expand Down Expand Up @@ -716,6 +725,11 @@ public Builder toBranch(String branch) {
return this;
}

public Builder outputFileFactoryProvider(OutputFileFactoryProvider provider) {
this.outputFileFactoryProvider = provider;
return this;
}

IcebergSink build() {

Preconditions.checkArgument(
Expand Down Expand Up @@ -806,7 +820,8 @@ IcebergSink build() {
overwriteMode,
maintenanceTasks,
flinkMaintenanceConfig,
equalityFieldColumnsSet);
equalityFieldColumnsSet,
outputFileFactoryProvider);
}

/**
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
/*
* 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;

import java.io.Serializable;
import org.apache.iceberg.FileFormat;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Table;
import org.apache.iceberg.io.OutputFileFactory;

/**
* Plugin interface for providing a custom {@link OutputFileFactory}.
*
* <p>Implementations can customize how output files are created, enabling use cases such as:
*
* <ul>
* <li>Custom file naming or path rewriting
* <li>Routing writes to non-standard storage endpoints
* <li>Region-specific or environment-specific file placement
* </ul>
*
* <p>When set on {@link IcebergSink.Builder#outputFileFactoryProvider}, the provided factory
* replaces the default {@link OutputFileFactory} created by {@link RowDataTaskWriterFactory}.
*/
@FunctionalInterface
public interface OutputFileFactoryProvider extends Serializable {
OutputFileFactory create(
Table table, int taskId, int attemptId, FileFormat format, PartitionSpec spec);
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import java.util.Map;
import java.util.Set;
import java.util.function.Supplier;
import javax.annotation.Nullable;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.logical.RowType;
import org.apache.iceberg.FileFormat;
Expand Down Expand Up @@ -53,6 +54,7 @@ public class RowDataTaskWriterFactory implements TaskWriterFactory<RowData> {
private final Set<Integer> equalityFieldIds;
private final boolean upsert;
private final FileWriterFactory<RowData> fileWriterFactory;
@Nullable private final OutputFileFactoryProvider outputFileFactoryProvider;
private boolean useDv;

private transient OutputFileFactory outputFileFactory;
Expand Down Expand Up @@ -105,7 +107,32 @@ public RowDataTaskWriterFactory(
boolean upsert,
Schema schema,
PartitionSpec spec) {
this(
tableSupplier,
flinkSchema,
targetFileSizeBytes,
format,
writeProperties,
equalityFieldIds,
upsert,
schema,
spec,
null);
}

public RowDataTaskWriterFactory(
SerializableSupplier<Table> tableSupplier,
RowType flinkSchema,
long targetFileSizeBytes,
FileFormat format,
Map<String, String> writeProperties,
Collection<Integer> equalityFieldIds,
boolean upsert,
Schema schema,
PartitionSpec spec,
@Nullable OutputFileFactoryProvider outputFileFactoryProvider) {
this.tableSupplier = tableSupplier;
this.outputFileFactoryProvider = outputFileFactoryProvider;

Table table;
if (tableSupplier instanceof CachingTableSupplier) {
Expand Down Expand Up @@ -174,12 +201,17 @@ public void initialize(int taskId, int attemptId) {
refreshTable();
this.useDv = TableUtil.formatVersion(table) > 2;

this.outputFileFactory =
OutputFileFactory.builderFor(table, taskId, attemptId)
.format(format)
.ioSupplier(() -> tableSupplier.get().io())
.defaultSpec(spec)
.build();
if (outputFileFactoryProvider != null) {
this.outputFileFactory =
outputFileFactoryProvider.create(table, taskId, attemptId, format, spec);
} else {
this.outputFileFactory =
OutputFileFactory.builderFor(table, taskId, attemptId)
.format(format)
.ioSupplier(() -> tableSupplier.get().io())
.defaultSpec(spec)
.build();
}
}

@Override
Expand Down
Loading