Skip to content

Commit

Permalink
[flink] PartitionMarkDone enables the use of various partition trigge…
Browse files Browse the repository at this point in the history
…r strategies.
  • Loading branch information
Aitozi committed Oct 28, 2024
1 parent 92550da commit ed485ea
Show file tree
Hide file tree
Showing 10 changed files with 269 additions and 208 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@
import java.io.IOException;
import java.util.List;

import static org.apache.paimon.flink.sink.partition.PartitionMarkDone.markDone;
import static org.apache.paimon.flink.sink.partition.PartitionMarkDoneTrigger.markDone;
import static org.apache.paimon.utils.ParameterUtils.getPartitions;
import static org.apache.paimon.utils.Preconditions.checkArgument;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@
import java.util.List;
import java.util.Map;

import static org.apache.paimon.flink.sink.partition.PartitionMarkDone.markDone;
import static org.apache.paimon.flink.sink.partition.PartitionMarkDoneTrigger.markDone;

/** Table partition mark done action for Flink. */
public class MarkPartitionDoneAction extends TableActionBase {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@
import java.io.IOException;
import java.util.List;

import static org.apache.paimon.flink.sink.partition.PartitionMarkDone.markDone;
import static org.apache.paimon.flink.sink.partition.PartitionMarkDoneTrigger.markDone;
import static org.apache.paimon.utils.ParameterUtils.getPartitions;
import static org.apache.paimon.utils.Preconditions.checkArgument;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@

import org.apache.paimon.annotation.VisibleForTesting;
import org.apache.paimon.flink.metrics.FlinkMetricRegistry;
import org.apache.paimon.flink.sink.partition.PartitionMarkDone;
import org.apache.paimon.flink.sink.partition.PartitionCollector;
import org.apache.paimon.io.DataFileMeta;
import org.apache.paimon.manifest.ManifestCommittable;
import org.apache.paimon.table.FileStoreTable;
Expand All @@ -43,7 +43,7 @@ public class StoreCommitter implements Committer<Committable, ManifestCommittabl

private final TableCommitImpl commit;
@Nullable private final CommitterMetrics committerMetrics;
@Nullable private final PartitionMarkDone partitionMarkDone;
private final PartitionCollector partitionCollector;

public StoreCommitter(FileStoreTable table, TableCommit commit, Context context) {
this.commit = (TableCommitImpl) commit;
Expand All @@ -56,8 +56,8 @@ public StoreCommitter(FileStoreTable table, TableCommit commit, Context context)
}

try {
this.partitionMarkDone =
PartitionMarkDone.create(
this.partitionCollector =
PartitionCollector.create(
context.streamingCheckpointEnabled(),
context.isRestored(),
context.stateStore(),
Expand Down Expand Up @@ -111,29 +111,23 @@ public void commit(List<ManifestCommittable> committables)
throws IOException, InterruptedException {
commit.commitMultiple(committables, false);
calcNumBytesAndRecordsOut(committables);
if (partitionMarkDone != null) {
partitionMarkDone.notifyCommittable(committables);
}
partitionCollector.notifyCommittable(committables);
}

@Override
public int filterAndCommit(
List<ManifestCommittable> globalCommittables, boolean checkAppendFiles) {
int committed = commit.filterAndCommitMultiple(globalCommittables, checkAppendFiles);
if (partitionMarkDone != null) {
partitionMarkDone.notifyCommittable(globalCommittables);
}
partitionCollector.notifyCommittable(globalCommittables);
return committed;
}

@Override
public Map<Long, List<Committable>> groupByCheckpoint(Collection<Committable> committables) {
if (partitionMarkDone != null) {
try {
partitionMarkDone.snapshotState();
} catch (Exception e) {
throw new RuntimeException(e);
}
try {
partitionCollector.snapshotState();
} catch (Exception e) {
throw new RuntimeException(e);
}

Map<Long, List<Committable>> grouped = new HashMap<>();
Expand All @@ -146,9 +140,7 @@ public Map<Long, List<Committable>> groupByCheckpoint(Collection<Committable> co
@Override
public void close() throws Exception {
commit.close();
if (partitionMarkDone != null) {
partitionMarkDone.close();
}
partitionCollector.close();
}

private void calcNumBytesAndRecordsOut(List<ManifestCommittable> committables) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
/*
* 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.paimon.flink.sink.partition;

import org.apache.paimon.manifest.ManifestCommittable;
import org.apache.paimon.table.FileStoreTable;
import org.apache.paimon.utils.IOUtils;

import org.apache.flink.api.common.state.OperatorStateStore;

import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;

/** Partition collector. */
public class PartitionCollector implements Closeable {

private final List<PartitionTrigger> triggers;

private PartitionCollector(List<PartitionTrigger> triggers) {
this.triggers = triggers;
}

public void notifyCommittable(List<ManifestCommittable> committables) {
for (PartitionTrigger trigger : triggers) {
trigger.notifyCommittable(committables);
}
}

public void snapshotState() throws Exception {
for (PartitionTrigger trigger : triggers) {
trigger.snapshotState();
}
}

@Override
public void close() throws IOException {
IOUtils.closeAllQuietly(triggers);
}

public static PartitionCollector create(
boolean isStreaming,
boolean isRestored,
OperatorStateStore stateStore,
FileStoreTable table)
throws Exception {
List<PartitionTrigger> triggers = new ArrayList<>();
PartitionMarkDoneTrigger.create(
table.coreOptions(), isStreaming, isRestored, stateStore, table)
.ifPresent(triggers::add);

return new PartitionCollector(triggers);
}
}

This file was deleted.

Loading

0 comments on commit ed485ea

Please sign in to comment.