You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/01/21 13:35:35 UTC

[GitHub] [flink] alpreu commented on a change in pull request #18428: [FLINK-25575] Add Sink V2 operators and translation

alpreu commented on a change in pull request #18428:
URL: https://github.com/apache/flink/pull/18428#discussion_r789656921



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/committables/CommittableCollector.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.flink.streaming.runtime.operators.sink.committables;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.connector.sink2.Sink.InitContext;
+import org.apache.flink.streaming.api.connector.sink2.CommittableMessage;
+import org.apache.flink.streaming.api.connector.sink2.CommittableSummary;
+import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage;
+
+import javax.annotation.Nullable;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class is responsible to book-keep the committing progress across checkpoints and subtasks.
+ * It handles the emission of committables and the {@link CommittableSummary}.
+ *
+ * @param <CommT> type of committable
+ */
+@Internal
+public class CommittableCollector<CommT> {
+    private static final long EOI = Long.MAX_VALUE;
+    private final NavigableMap<Long, CheckpointCommittablesImpl<CommT>> checkpointCommittables;
+    private final int subtaskId;
+    private final int numberOfSubtasks;
+
+    CommittableCollector(int subtaskId, int numberOfSubtasks) {
+        this.subtaskId = subtaskId;
+        this.numberOfSubtasks = numberOfSubtasks;
+        this.checkpointCommittables = new TreeMap<>();
+    }
+
+    /** For deserialization. */
+    CommittableCollector(Map<Long, CheckpointCommittablesImpl<CommT>> checkpointCommittables) {
+        this.checkpointCommittables = new TreeMap<>(checkpointCommittables);
+        this.subtaskId = 0;
+        this.numberOfSubtasks = 1;
+    }
+
+    /**
+     * Creates a {@link CommittableCollector} based on the current runtime information. This method
+     * should be used for to instantiate a collector for all Sink V2.
+     *
+     * @param context holding runtime of information
+     * @param <CommT> type of the committable
+     * @return {@link CommittableCollector}
+     */
+    public static <CommT> CommittableCollector<CommT> of(RuntimeContext context) {
+        return new CommittableCollector<>(
+                context.getIndexOfThisSubtask(), context.getNumberOfParallelSubtasks());
+    }
+
+    /**
+     * Creates a {@link CommittableCollector} for a list of committables. This method is mainly used
+     * to create a collector from the state of Sink V1.
+     *
+     * @param r list of committables
+     * @param <CommT> type of committables
+     * @return {@link CommittableCollector}
+     */
+    static <CommT> CommittableCollector<CommT> ofLegacy(List<CommT> r) {
+        CommittableCollector<CommT> committableCollector = new CommittableCollector<>(0, 1);
+        // add a checkpoint with the lowest checkpoint id, this will be merged into the next
+        // checkpoint data, subtask id is arbitrary
+        CommittableSummary<CommT> summary =
+                new CommittableSummary<>(
+                        0, 1, InitContext.INITIAL_CHECKPOINT_ID, r.size(), r.size(), 0);
+        committableCollector.addSummary(summary);
+        SubtaskCommittables<CommT> subtask =
+                committableCollector
+                        .getCheckpointCommittables(summary)
+                        .getSubtaskCommittables(summary.getSubtaskId());
+        r.forEach(subtask::add);
+        return committableCollector;
+    }
+
+    /**
+     * Adds a {@link CommittableMessage} to the collector to hold it until emission.
+     *
+     * @param message either {@link CommittableSummary} or {@link CommittableWithLineage}
+     */
+    public void addMessage(CommittableMessage<CommT> message) {
+        if (message instanceof CommittableSummary) {
+            addSummary((CommittableSummary<CommT>) message);
+        } else if (message instanceof CommittableWithLineage) {
+            addCommittable((CommittableWithLineage<CommT>) message);
+        }
+    }
+
+    /**
+     * Returns all {@link CheckpointCommittables} until the requested checkpoint id.
+     *
+     * @param checkpointId counter
+     * @return collection of {@link CheckpointCommittables}
+     */
+    public Collection<? extends CheckpointCommittables<CommT>> getCheckpointCommittablesUpTo(
+            long checkpointId) {
+        // clean up fully committed previous checkpoints
+        // this wouldn't work with concurrent unaligned checkpoints
+        Collection<CheckpointCommittablesImpl<CommT>> checkpoints =
+                checkpointCommittables.headMap(checkpointId, true).values();
+        checkpoints.removeIf(CheckpointCommittablesImpl::isFinished);
+        return checkpoints;
+    }
+
+    /**
+     * Returns all {@link CheckpointCommittables} that are currently hold by the collector.
+     *
+     * @return collection of {@link CheckpointCommittables}
+     */
+    @Nullable
+    public Collection<? extends CheckpointCommittables<CommT>> getEndOfInputCommittables() {
+        return getCheckpointCommittablesUpTo(EOI);
+    }
+
+    /**
+     * Returns whether all {@link CheckpointCommittables} currently hold by the collector are either

Review comment:
       ```suggestion
        * Returns whether all {@link CheckpointCommittables} currently held by the collector are either
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org