LadyForest commented on a change in pull request #28:
URL: https://github.com/apache/flink-table-store/pull/28#discussion_r820487023



##########
File path: 
flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/sink/global/LocalCommitterOperator.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.table.store.connector.sink.global;
+
+import org.apache.flink.api.connector.sink2.Committer;
+import org.apache.flink.api.connector.sink2.Committer.CommitRequest;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import 
org.apache.flink.streaming.runtime.operators.sink.committables.CommitRequestImpl;
+import 
org.apache.flink.streaming.runtime.operators.sink.committables.CommitRequestState;
+import org.apache.flink.util.function.SerializableSupplier;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A {@link AbstractCommitterOperator} to process local committer. */
+public class LocalCommitterOperator<CommT> extends 
AbstractCommitterOperator<CommT, CommT> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final SerializableSupplier<Committer<CommT>> committerFactory;
+
+    private Committer<CommT> committer;
+
+    public LocalCommitterOperator(
+            SerializableSupplier<Committer<CommT>> committerFactory,
+            SerializableSupplier<SimpleVersionedSerializer<CommT>> 
committableSerializer) {
+        super(committableSerializer);
+        this.committerFactory = checkNotNull(committerFactory);
+    }
+
+    @Override
+    public void initializeState(StateInitializationContext context) throws 
Exception {
+        committer = committerFactory.get();
+        super.initializeState(context);
+    }
+
+    @Override
+    public void commit(boolean isRecover, List<CommT> committables)
+            throws IOException, InterruptedException {
+        if (committables.isEmpty()) {
+            return;
+        }
+
+        List<CommitRequestImpl> requests = new 
ArrayList<>(committables.size());
+        for (CommT comm : committables) {
+            requests.add(new CommitRequestImpl(comm));
+        }
+
+        long sleep = 1000;
+        while (true) {
+            // commit
+            requests.forEach(CommitRequestImpl::setSelected);
+            committer.commit(new ArrayList<>(requests));
+            requests.forEach(CommitRequestImpl::setCommittedIfNoError);
+
+            // drain finished
+            requests.removeIf(CommitRequestImpl::isFinished);
+            if (requests.isEmpty()) {
+                return;
+            }
+
+            //noinspection BusyWait
+            Thread.sleep(sleep);
+            sleep *= 2;
+        }
+    }
+
+    @Override
+    public List<CommT> toCommittables(long checkpoint, List<CommT> inputs) {
+        return inputs;
+    }
+
+    @Override
+    public void close() throws Exception {
+        committer.close();
+        super.close();
+    }
+
+    /** {@link CommitRequest} implementation. */
+    public class CommitRequestImpl implements CommitRequest<CommT> {
+
+        private CommT committable;
+        private int numRetries;
+        private CommitRequestState state;
+
+        private CommitRequestImpl(CommT committable) {
+            this.committable = committable;
+            this.state = CommitRequestState.RECEIVED;
+        }
+
+        private boolean isFinished() {
+            return state.isFinalState();
+        }
+
+        @Override
+        public CommT getCommittable() {
+            return this.committable;
+        }
+
+        @Override
+        public int getNumberOfRetries() {
+            return this.numRetries;
+        }
+
+        @Override
+        public void signalFailedWithKnownReason(Throwable t) {
+            this.state = CommitRequestState.FAILED;

Review comment:
       Should we throw an exception here? O.w. the exception will be swallowed 
without any trace.




-- 
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-unsubscr...@flink.apache.org

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


Reply via email to