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 @@ -26,12 +26,14 @@
import java.util.NavigableMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import javax.annotation.Nullable;
import org.apache.flink.api.connector.sink2.Committer;
import org.apache.flink.core.io.SimpleVersionedSerialization;
import org.apache.iceberg.AppendFiles;
import org.apache.iceberg.ManifestFile;
import org.apache.iceberg.ReplacePartitions;
import org.apache.iceberg.RowDelta;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.SnapshotUpdate;
import org.apache.iceberg.Table;
import org.apache.iceberg.flink.TableLoader;
Expand Down Expand Up @@ -79,6 +81,7 @@ class IcebergCommitter implements Committer<IcebergCommittable> {
private ExecutorService workerPool;
private int continuousEmptyCheckpoints = 0;
private final boolean tableMaintenanceEnabled;
@Nullable private final PostCommitHook postCommitHook;

IcebergCommitter(
TableLoader tableLoader,
Expand All @@ -89,6 +92,28 @@ class IcebergCommitter implements Committer<IcebergCommittable> {
String sinkId,
IcebergFilesCommitterMetrics committerMetrics,
boolean tableMaintenanceEnabled) {
this(
tableLoader,
branch,
snapshotProperties,
replacePartitions,
workerPoolSize,
sinkId,
committerMetrics,
tableMaintenanceEnabled,
null);
}

IcebergCommitter(
TableLoader tableLoader,
String branch,
Map<String, String> snapshotProperties,
boolean replacePartitions,
int workerPoolSize,
String sinkId,
IcebergFilesCommitterMetrics committerMetrics,
boolean tableMaintenanceEnabled,
@Nullable PostCommitHook postCommitHook) {
this.branch = branch;
this.snapshotProperties = snapshotProperties;
this.replacePartitions = replacePartitions;
Expand All @@ -108,6 +133,7 @@ class IcebergCommitter implements Committer<IcebergCommittable> {
"iceberg-committer-pool-" + table.name() + "-" + sinkId, workerPoolSize);
this.continuousEmptyCheckpoints = 0;
this.tableMaintenanceEnabled = tableMaintenanceEnabled;
this.postCommitHook = postCommitHook;
}

@Override
Expand Down Expand Up @@ -307,6 +333,14 @@ private void commitOperation(
if (committerMetrics != null) {
committerMetrics.commitDuration(durationMs);
}

if (postCommitHook != null) {
table.refresh();
Snapshot snapshot = table.currentSnapshot();
if (snapshot != null) {
postCommitHook.afterCommit(snapshot.snapshotId(), snapshot.summary());
}
}
}

@Override
Expand Down
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 PostCommitHook postCommitHook;

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 PostCommitHook postCommitHook) {
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.postCommitHook = postCommitHook;
}

@Override
Expand Down Expand Up @@ -247,7 +252,8 @@ public Committer<IcebergCommittable> createCommitter(CommitterInitContext contex
workerPoolSize,
sinkId,
metrics,
maintenanceEnabled);
maintenanceEnabled,
postCommitHook);
}

@Override
Expand Down Expand Up @@ -349,6 +355,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 PostCommitHook postCommitHook;

private Builder() {}

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

public Builder postCommitHook(PostCommitHook hook) {
this.postCommitHook = hook;
return this;
}

IcebergSink build() {

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

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

/**
* Callback invoked after each successful Iceberg snapshot commit.
*
* <p>Implementations can react to committed snapshots for use cases such as:
*
* <ul>
* <li>Audit logging of committed snapshots
* <li>Synchronizing metadata to external catalogs (e.g., Hive Metastore, Nessie)
* <li>Triggering compaction or maintenance workflows
* <li>Tracking custom metadata across commits
* </ul>
*
* <p>The hook is called after the Iceberg commit succeeds but before the Flink checkpoint
* completes. Exceptions thrown by the hook will propagate to the committer.
*/
@FunctionalInterface
public interface PostCommitHook extends Serializable {
void afterCommit(long snapshotId, Map<String, String> snapshotSummary);
}
Loading