kl0u commented on a change in pull request #13678: URL: https://github.com/apache/flink/pull/13678#discussion_r508475873
########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/AbstractStreamingCommitterOperator.java ########## @@ -0,0 +1,179 @@ +/* + * 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; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Abstract base class for operators that work with a {@link Committer} or a + * {@link org.apache.flink.api.connector.sink.GlobalCommitter} in the streaming execution mode. + * + * <p>Sub-classes are responsible for implementing {@link #recoveredCommittables(List)} and {@link #preCommit(List)}. + * + * @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. */ + private 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. */ + private List<InputT> currentInputs; + + /** + * Notifies 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); + + /** + * Prepares a commit. + * + * @param input A list of input elements received since last pre-commit + * + * @return @return A list of committables that could be committed in the following checkpoint complete. + */ + abstract List<CommT> preCommit(List<InputT> input); + + AbstractStreamingCommitterOperator( + Committer<CommT> committer, + SimpleVersionedSerializer<CommT> committableSerializer) { + this.committer = checkNotNull(committer); + this.streamingCommitterStateSerializer = new StreamingCommitterStateSerializer<>( + checkNotNull(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(currentInputs)); + streamingCommitterState.update( + Collections.singletonList(new StreamingCommitterState<>(committablesPerCheckpoint))); + currentInputs = new ArrayList<>(); + } + + @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()) { + final Map.Entry<Long, List<CommT>> entry = it.next(); + final List<CommT> neededToRetryCommittables = committer.commit(entry.getValue()); Review comment: Here we can get the values (`List<CommT> committables = entry.getValue();`) and reuse them, and not call `entry.getValue()` multiple times. ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/GlobalStreamingCommitterOperator.java ########## @@ -0,0 +1,88 @@ +/* + * 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.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; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Runtime {@link org.apache.flink.streaming.api.operators.StreamOperator} for executing + * {@link GlobalCommitter} in the streaming execution mode. + * + * @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 final GlobalCommitter<CommT, GlobalCommT> globalCommitter; + + /** The global committables that might need to be committed again after recovering from a failover. */ + private List<GlobalCommT> recoveredGlobalCommittables; Review comment: I think the code would be easier if this becomes `final` and we do not set it to `null` but we call `clear()` whenever we want to re-initialize it. This will remove all the checks about `if (mylist == null) ...`. ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/AbstractStreamingCommitterOperator.java ########## @@ -0,0 +1,179 @@ +/* + * 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; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Abstract base class for operators that work with a {@link Committer} or a + * {@link org.apache.flink.api.connector.sink.GlobalCommitter} in the streaming execution mode. + * + * <p>Sub-classes are responsible for implementing {@link #recoveredCommittables(List)} and {@link #preCommit(List)}. + * + * @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. */ + private 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; Review comment: I would propose to make this `private` and add a getter that is visible for testing. In general I am not so much in favour of `@VisibleForTesting` things and I would prefer if we passed a custom `Committer` in the test harness and use that to get whatever internal information we want, but I understand that this may require refactoring the tests which may take some time. But that can be a thing to keep it in the list of future things to do :) ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/StreamingCommitterOperator.java ########## @@ -0,0 +1,60 @@ +/* + * 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 org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.util.ArrayList; +import java.util.List; + +/** + * Runtime {@link org.apache.flink.streaming.api.operators.StreamOperator} for executing + * {@link Committer} in the streaming execution mode. + * + * @param <CommT> The committable type of the {@link Committer}. + */ +final class StreamingCommitterOperator<CommT> extends AbstractStreamingCommitterOperator<CommT, CommT> { + + /** The committables that might need to be committed again after recovering from a failover. */ + private List<CommT> recoveredCommittables; Review comment: As in the global committer, I would make this `final` and simply clear it up whenever we want to re-initialize it. I think that checking for `null` always leaves a possibility for error in the future, when someone else will touch this code. ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/AbstractStreamingCommitterOperator.java ########## @@ -0,0 +1,179 @@ +/* + * 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; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Abstract base class for operators that work with a {@link Committer} or a + * {@link org.apache.flink.api.connector.sink.GlobalCommitter} in the streaming execution mode. + * + * <p>Sub-classes are responsible for implementing {@link #recoveredCommittables(List)} and {@link #preCommit(List)}. + * + * @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. */ + private 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. */ + private List<InputT> currentInputs; + + /** + * Notifies 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); + + /** + * Prepares a commit. + * + * @param input A list of input elements received since last pre-commit + * + * @return @return A list of committables that could be committed in the following checkpoint complete. + */ + abstract List<CommT> preCommit(List<InputT> input); + + AbstractStreamingCommitterOperator( + Committer<CommT> committer, + SimpleVersionedSerializer<CommT> committableSerializer) { + this.committer = checkNotNull(committer); + this.streamingCommitterStateSerializer = new StreamingCommitterStateSerializer<>( + checkNotNull(committableSerializer)); Review comment: We do not need `checkNotNull` here because we check in the `StreamingCommitterStateSerializer` constructor. ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/GlobalStreamingCommitterOperator.java ########## @@ -0,0 +1,88 @@ +/* + * 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.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; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Runtime {@link org.apache.flink.streaming.api.operators.StreamOperator} for executing + * {@link GlobalCommitter} in the streaming execution mode. + * + * @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 final GlobalCommitter<CommT, GlobalCommT> globalCommitter; + + /** The global committables that might need to be committed again after recovering from a failover. */ + private List<GlobalCommT> recoveredGlobalCommittables; + + private boolean endOfInput; + + GlobalStreamingCommitterOperator( + GlobalCommitter<CommT, GlobalCommT> globalCommitter, + SimpleVersionedSerializer<GlobalCommT> committableSerializer) { + super(globalCommitter, committableSerializer); + this.globalCommitter = checkNotNull(globalCommitter); + this.endOfInput = false; + } + + @Override + void recoveredCommittables(List<GlobalCommT> committables) { + recoveredGlobalCommittables = globalCommitter.filterRecoveredCommittables(committables); + } + + @Override + List<GlobalCommT> preCommit(List<CommT> input) { + final List<GlobalCommT> result = new ArrayList<>(); + if (recoveredGlobalCommittables != null && !recoveredGlobalCommittables.isEmpty()) { Review comment: With the above changes, this can become : ``` final List<GlobalCommT> result = new ArrayList<>(recoveredGlobalCommittables); recoveredGlobalCommittables.clear(); if (!input.isEmpty()) { result.add(globalCommitter.combine(input)); } ``` ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/AbstractStreamingCommitterOperator.java ########## @@ -0,0 +1,179 @@ +/* + * 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; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Abstract base class for operators that work with a {@link Committer} or a + * {@link org.apache.flink.api.connector.sink.GlobalCommitter} in the streaming execution mode. + * + * <p>Sub-classes are responsible for implementing {@link #recoveredCommittables(List)} and {@link #preCommit(List)}. + * + * @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. */ + private 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. */ + private List<InputT> currentInputs; + + /** + * Notifies 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); + + /** + * Prepares a commit. + * + * @param input A list of input elements received since last pre-commit + * + * @return @return A list of committables that could be committed in the following checkpoint complete. + */ + abstract List<CommT> preCommit(List<InputT> input); + + AbstractStreamingCommitterOperator( + Committer<CommT> committer, + SimpleVersionedSerializer<CommT> committableSerializer) { + this.committer = checkNotNull(committer); + this.streamingCommitterStateSerializer = new StreamingCommitterStateSerializer<>( + checkNotNull(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(currentInputs)); + streamingCommitterState.update( + Collections.singletonList(new StreamingCommitterState<>(committablesPerCheckpoint))); + currentInputs = new ArrayList<>(); + } + + @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()) { + final Map.Entry<Long, List<CommT>> entry = it.next(); + final List<CommT> neededToRetryCommittables = committer.commit(entry.getValue()); + + if (!neededToRetryCommittables.isEmpty()) { + throw new UnsupportedOperationException("Currently does not support the re-commit!"); + } + Review comment: What about logging here that we are `committing state for checkpoint XXXX`. This may help if something goes wrong. ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/sink/GlobalStreamingCommitterOperator.java ########## @@ -0,0 +1,88 @@ +/* + * 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.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; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Runtime {@link org.apache.flink.streaming.api.operators.StreamOperator} for executing + * {@link GlobalCommitter} in the streaming execution mode. + * + * @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 final GlobalCommitter<CommT, GlobalCommT> globalCommitter; + + /** The global committables that might need to be committed again after recovering from a failover. */ + private List<GlobalCommT> recoveredGlobalCommittables; + + private boolean endOfInput; + + GlobalStreamingCommitterOperator( + GlobalCommitter<CommT, GlobalCommT> globalCommitter, + SimpleVersionedSerializer<GlobalCommT> committableSerializer) { + super(globalCommitter, committableSerializer); + this.globalCommitter = checkNotNull(globalCommitter); + this.endOfInput = false; + } + + @Override + void recoveredCommittables(List<GlobalCommT> committables) { + recoveredGlobalCommittables = globalCommitter.filterRecoveredCommittables(committables); Review comment: Here we should check that committbles are not `null` to be safe. In addition, I would propose to not reuse the same list that we expose to the users but we copy the result of the `filterRecoveredCommittables()` in our own list. ---------------------------------------------------------------- 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]
