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 2020/10/19 15:02:42 UTC

[GitHub] [flink] kl0u commented on a change in pull request #13678: [FLINK-19586] Add stream committer operators for new Sink API

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



##########
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 =

Review comment:
       This can be `private`.

##########
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;

Review comment:
       What about putting a `checkNotNull(committer)` to guarantee the invariants?

##########
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)));

Review comment:
       With the change that @gaoyunhaii proposed, here we will also be able to clear the `currentInputs`. Something like:
   
   ```
   committablesPerCheckpoint.put(context.getCheckpointId(), preCommit(currentInputs));
   streamingCommitterState.update(
   				Collections.singletonList(new StreamingCommitterState<>(committablesPerCheckpoint)));
   currentInputs = new ArrayList<>();
   ```

##########
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;

Review comment:
       Again maybe add a `checkNotNull` for safety.

##########
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);
+	}
+
+	@Override
+	List<GlobalCommT> preCommit() {
+		final List<GlobalCommT> result = computeCurrentGlobalCommittables();
+		currentInputs = new ArrayList<>();
+		currentGlobalCommittables = new ArrayList<>();
+		return result;
+	}
+
+	@Override
+	public void endInput() {
+		endOfInput = true;
+	}
+
+	@Override
+	public void notifyCheckpointComplete(long checkpointId) throws Exception {
+		super.notifyCheckpointComplete(checkpointId);
+		if (endOfInput) {
+			globalCommitter.endOfInput();
+		}
+	}
+
+	private List<GlobalCommT> computeCurrentGlobalCommittables() {
+		final List<GlobalCommT> result = new ArrayList<>();

Review comment:
       This could become: 
   
   ```
   final List<GlobalCommT> result = new ArrayList<>(currentGlobalCommittables);
   		if (!pendingCommittables.isEmpty()) {
   			final GlobalCommT globalCommittable = globalCommitter.combine(currentInputs); // this can change from previous comments
   			result.add(globalCommittable);
   		}
   		currentGlobalCommittables = new ArrayList<>();
   ```

##########
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:
       Big +1 on the comment from @gaoyunhaii to make the `currentInputs` `private` and pass it on as argument in the `preCommit`.  

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/StreamingCommitterState.java
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.api.connector.sink.Committer;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Objects;
+
+/**
+ * The state fo the {@link AbstractStreamingCommitterOperator}.
+ *
+ * @param <CommT> The committable type of the {@link Committer}.
+ */
+final class StreamingCommitterState<CommT> {
+
+	private final List<CommT> committables;
+
+	StreamingCommitterState(List<CommT> committables) {
+		this.committables = committables;

Review comment:
       `checkNotNull`?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/StreamingCommitterStateSerializer.java
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.core.io.SimpleVersionedSerializer;
+import org.apache.flink.core.memory.DataInputDeserializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputSerializer;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * The serializer for the {@link StreamingCommitterState}.
+ */
+
+final class StreamingCommitterStateSerializer<CommT> implements SimpleVersionedSerializer<StreamingCommitterState<CommT>> {
+
+	private static final int MAGIC_NUMBER = 0xb91f252c;
+
+	private final SimpleVersionedSerializer<CommT> committableSerializer;
+
+	StreamingCommitterStateSerializer(SimpleVersionedSerializer<CommT> committableSerializer) {
+		this.committableSerializer = committableSerializer;

Review comment:
       `checkNotNull`?




----------------------------------------------------------------
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:
users@infra.apache.org