gaoyunhaii commented on a change in pull request #13678:
URL: https://github.com/apache/flink/pull/13678#discussion_r507569024



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/AbstractStreamingCommitterOperator.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import 
org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer;
+import org.apache.flink.api.connector.sink.Committer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.util.SimpleVersionedListState;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+
+/**
+ * Abstract base class for operators that work with a {@link Committer} or a 
{@link org.apache.flink.api.connector.sink.GlobalCommitter}.
+ *
+ * <p>Sub-classes are responsible for implementing {@link 
#recoveredCommittables(List)} and {@link #preCommit()}.
+ *
+ * @param <InputT> The input type of the {@link Committer}.
+ * @param <CommT> The committable type of the {@link Committer}.
+ */
+abstract class AbstractStreamingCommitterOperator<InputT, CommT> extends 
AbstractStreamOperator<CommT>
+               implements OneInputStreamOperator<InputT, CommT> {
+
+       private static final long serialVersionUID = 1L;
+
+       /** The operator's state descriptor. */
+       static final ListStateDescriptor<byte[]> 
STREAMING_COMMITTER_RAW_STATES_DESC =
+                       new ListStateDescriptor<>(
+                                       "streaming_committer_raw_states",
+                                       BytePrimitiveArraySerializer.INSTANCE);
+
+       /** Group the committable by the checkpoint id. */
+       private final NavigableMap<Long, List<CommT>> committablesPerCheckpoint;
+
+       /** The committable's serializer. */
+       private final StreamingCommitterStateSerializer<CommT> 
streamingCommitterStateSerializer;
+
+       /** Responsible for committing the committable to the external system. 
**/
+       protected final Committer<CommT> committer;
+
+       /** The operator's state. */
+       private ListState<StreamingCommitterState<CommT>> 
streamingCommitterState;
+
+       /** Inputs collected between every pre-commit. */
+       protected List<InputT> currentInputs;
+
+       /**
+        * Notify a list of committables that might need to be committed again 
after recovering from a failover.

Review comment:
       One small format problem is that it seems we are always using "Third 
person singular" for method/class comments starts with verbs, namely we should 
use `Notifies` instead of `Notify` here and similarly for the other comments in 
the PR. 

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/GlobalStreamingCommitterOperator.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Runtime {@link org.apache.flink.streaming.api.operators.StreamOperator} for 
executing {@link GlobalCommitter}.
+ *
+ * @param <CommT> The committable type of the {@link GlobalCommitter}.
+ * @param <GlobalCommT> The global committable type of the {@link 
GlobalCommitter}.
+ */
+@Internal
+public final class GlobalStreamingCommitterOperator<CommT, GlobalCommT> 
extends AbstractStreamingCommitterOperator<CommT, GlobalCommT>
+               implements BoundedOneInput {
+
+       /** Aggregate committables to global committables and commit the global 
committables to the external system. */
+       private GlobalCommitter<CommT, GlobalCommT> globalCommitter;

Review comment:
       could be `final`

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/AbstractStreamingCommitterOperator.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import 
org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer;
+import org.apache.flink.api.connector.sink.Committer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.util.SimpleVersionedListState;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+
+/**
+ * Abstract base class for operators that work with a {@link Committer} or a 
{@link org.apache.flink.api.connector.sink.GlobalCommitter}.
+ *
+ * <p>Sub-classes are responsible for implementing {@link 
#recoveredCommittables(List)} and {@link #preCommit()}.
+ *
+ * @param <InputT> The input type of the {@link Committer}.
+ * @param <CommT> The committable type of the {@link Committer}.
+ */
+abstract class AbstractStreamingCommitterOperator<InputT, CommT> extends 
AbstractStreamOperator<CommT>
+               implements OneInputStreamOperator<InputT, CommT> {
+
+       private static final long serialVersionUID = 1L;
+
+       /** The operator's state descriptor. */
+       static final ListStateDescriptor<byte[]> 
STREAMING_COMMITTER_RAW_STATES_DESC =
+                       new ListStateDescriptor<>(
+                                       "streaming_committer_raw_states",
+                                       BytePrimitiveArraySerializer.INSTANCE);
+
+       /** Group the committable by the checkpoint id. */
+       private final NavigableMap<Long, List<CommT>> committablesPerCheckpoint;
+
+       /** The committable's serializer. */
+       private final StreamingCommitterStateSerializer<CommT> 
streamingCommitterStateSerializer;
+
+       /** Responsible for committing the committable to the external system. 
**/
+       protected final Committer<CommT> committer;
+
+       /** The operator's state. */
+       private ListState<StreamingCommitterState<CommT>> 
streamingCommitterState;
+
+       /** Inputs collected between every pre-commit. */
+       protected List<InputT> currentInputs;

Review comment:
       The relationship between `currentInput` and `currentGlobalCommittables` 
seems to be a little complex to understand. As a whole, I would tend to 
   
   1. Make `currentInputs` only available in the abstract class, pass it to 
`precommit` as a parameter and clear it in the abstract class.
   2. In `StreamingCommitterOperator` the result of `recoveredCommittables` 
should be saved in a standalone list, as did in the 
`GlobalStreamingCommitterOperator`. The recovered committables are semantically 
not inputs.
   3. In `StreamingCommitterOperator` and `GlobalStreamingCommitterOperator` 
rename the `currentGlobalCommittables` to be `recovered(Global)Committables`.  
   
   
   One pending problem here is that do we need to immediately commit the 
recovered committables ? In the current implementation of `StreamingFileSink` 
we did so.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/GlobalStreamingCommitterOperator.java
##########
@@ -0,0 +1,99 @@
+/*
+ * 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;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.connector.sink.GlobalCommitter;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Runtime {@link org.apache.flink.streaming.api.operators.StreamOperator} for 
executing {@link GlobalCommitter}.
+ *
+ * @param <CommT> The committable type of the {@link GlobalCommitter}.
+ * @param <GlobalCommT> The global committable type of the {@link 
GlobalCommitter}.
+ */
+@Internal
+public final class GlobalStreamingCommitterOperator<CommT, GlobalCommT> 
extends AbstractStreamingCommitterOperator<CommT, GlobalCommT>
+               implements BoundedOneInput {
+
+       /** Aggregate committables to global committables and commit the global 
committables to the external system. */
+       private GlobalCommitter<CommT, GlobalCommT> globalCommitter;
+
+       /** The current pending global committables. */
+       private List<GlobalCommT> currentGlobalCommittables;
+
+       private boolean endOfInput;
+
+       GlobalStreamingCommitterOperator(
+                       GlobalCommitter<CommT, GlobalCommT> globalCommitter,
+                       SimpleVersionedSerializer<GlobalCommT> 
committableSerializer) {
+               super(globalCommitter, committableSerializer);
+               this.globalCommitter = globalCommitter;
+               this.endOfInput = false;
+       }
+
+       @Override
+       void recoveredCommittables(List<GlobalCommT> committables) {
+               this.currentGlobalCommittables = 
globalCommitter.filterRecoveredCommittables(committables);

Review comment:
       Remove `this.` if not needed

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/AbstractStreamingCommitterOperator.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import 
org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer;
+import org.apache.flink.api.connector.sink.Committer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.util.SimpleVersionedListState;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+
+/**
+ * Abstract base class for operators that work with a {@link Committer} or a 
{@link org.apache.flink.api.connector.sink.GlobalCommitter}.
+ *
+ * <p>Sub-classes are responsible for implementing {@link 
#recoveredCommittables(List)} and {@link #preCommit()}.
+ *
+ * @param <InputT> The input type of the {@link Committer}.
+ * @param <CommT> The committable type of the {@link Committer}.
+ */
+abstract class AbstractStreamingCommitterOperator<InputT, CommT> extends 
AbstractStreamOperator<CommT>
+               implements OneInputStreamOperator<InputT, CommT> {
+
+       private static final long serialVersionUID = 1L;
+
+       /** The operator's state descriptor. */
+       static final ListStateDescriptor<byte[]> 
STREAMING_COMMITTER_RAW_STATES_DESC =
+                       new ListStateDescriptor<>(
+                                       "streaming_committer_raw_states",
+                                       BytePrimitiveArraySerializer.INSTANCE);
+
+       /** Group the committable by the checkpoint id. */
+       private final NavigableMap<Long, List<CommT>> committablesPerCheckpoint;
+
+       /** The committable's serializer. */
+       private final StreamingCommitterStateSerializer<CommT> 
streamingCommitterStateSerializer;
+
+       /** Responsible for committing the committable to the external system. 
**/
+       protected final Committer<CommT> committer;
+
+       /** The operator's state. */
+       private ListState<StreamingCommitterState<CommT>> 
streamingCommitterState;
+
+       /** Inputs collected between every pre-commit. */
+       protected List<InputT> currentInputs;
+
+       /**
+        * Notify a list of committables that might need to be committed again 
after recovering from a failover.
+        *
+        * @param committables A list of committables
+        */
+       abstract void recoveredCommittables(List<CommT> committables);
+
+       /**
+        * Prepare a commit.
+        *
+        * @return A list of committables that could be committed in the 
following checkpoint complete.
+        */
+       abstract List<CommT> preCommit();
+
+       AbstractStreamingCommitterOperator(
+                       Committer<CommT> committer,
+                       SimpleVersionedSerializer<CommT> committableSerializer) 
{
+               this.committer = committer;
+               this.streamingCommitterStateSerializer = new 
StreamingCommitterStateSerializer<>(
+                               committableSerializer);
+               this.committablesPerCheckpoint = new TreeMap<>();
+               this.currentInputs = new ArrayList<>();
+       }
+
+       @Override
+       public void initializeState(StateInitializationContext context) throws 
Exception {
+               super.initializeState(context);
+               streamingCommitterState = new SimpleVersionedListState<>(
+                               
context.getOperatorStateStore().getListState(STREAMING_COMMITTER_RAW_STATES_DESC),
+                               streamingCommitterStateSerializer);
+               final List<CommT> restored = new ArrayList<>();
+               streamingCommitterState.get().forEach(s -> 
restored.addAll(s.getCommittables()));
+               recoveredCommittables(restored);
+       }
+
+       @Override
+       public void processElement(StreamRecord<InputT> element) throws 
Exception {
+               currentInputs.add(element.getValue());
+       }
+
+       @Override
+       public void close() throws Exception {
+               super.close();
+               committer.close();
+       }
+
+       @Override
+       public void snapshotState(StateSnapshotContext context) throws 
Exception {
+               super.snapshotState(context);
+               committablesPerCheckpoint.put(context.getCheckpointId(), 
preCommit());
+               streamingCommitterState.update(
+                               Collections.singletonList(new 
StreamingCommitterState<>(committablesPerCheckpoint)));
+       }
+
+       @Override
+       public void notifyCheckpointComplete(long checkpointId) throws 
Exception {
+               super.notifyCheckpointComplete(checkpointId);
+               commitUpTo(checkpointId);
+       }
+
+       private void commitUpTo(long checkpointId) throws Exception {
+               final Iterator<Map.Entry<Long, List<CommT>>>
+                               it = 
committablesPerCheckpoint.headMap(checkpointId, true).entrySet().iterator();
+
+               while (it.hasNext()) {
+

Review comment:
       Remove this empty line?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to