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] PartitionMarkDone enables the use of various partition trigge… #4386

Closed
wants to merge 3 commits into from
Closed
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 @@ -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.PartitionMarkDoneListener.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.PartitionMarkDoneListener.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.PartitionMarkDoneListener.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.PartitionListeners;
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 PartitionListeners partitionListeners;

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.partitionListeners =
PartitionListeners.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);
}
partitionListeners.notifyCommittable(committables);
}

@Override
public int filterAndCommit(
List<ManifestCommittable> globalCommittables, boolean checkAppendFiles) {
int committed = commit.filterAndCommitMultiple(globalCommittables, checkAppendFiles);
if (partitionMarkDone != null) {
partitionMarkDone.notifyCommittable(globalCommittables);
}
partitionListeners.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 {
partitionListeners.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();
}
partitionListeners.close();
}

private void calcNumBytesAndRecordsOut(List<ManifestCommittable> committables) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
/*
* 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 java.io.Closeable;
import java.util.List;

/** The partition listener. */
public interface PartitionListener extends Closeable {

void notifyCommittable(List<ManifestCommittable> committables);

void snapshotState() throws Exception;
}
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 PartitionListeners implements Closeable {

private final List<PartitionListener> listeners;

private PartitionListeners(List<PartitionListener> listeners) {
this.listeners = listeners;
}

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

public void snapshotState() throws Exception {
for (PartitionListener trigger : listeners) {
trigger.snapshotState();
}
}

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

public static PartitionListeners create(
boolean isStreaming,
boolean isRestored,
OperatorStateStore stateStore,
FileStoreTable table)
throws Exception {
List<PartitionListener> listeners = new ArrayList<>();
PartitionMarkDoneListener.create(
table.coreOptions(), isStreaming, isRestored, stateStore, table)
.ifPresent(listeners::add);

return new PartitionListeners(listeners);
}
}

This file was deleted.

Loading
Loading