Skip to content

Commit

Permalink
[FLINK-26516][streaming] Recover GlobalCommittables with Sink V1 Glob…
Browse files Browse the repository at this point in the history
…alCommittable serializer

With SinkV2 the committer and global committer work very similar and
they only write committables into state. SinkV1's GlobalCommitter on the
other hand used to write GlobalCommittables into state so this commits
adds an migration path.

This closes apache#18805.
  • Loading branch information
fapaul authored and gaoyunhaii committed Mar 8, 2022
1 parent 948f571 commit 955e5ff
Show file tree
Hide file tree
Showing 15 changed files with 1,096 additions and 63 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/*
* 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.api.connector.sink2;

import org.apache.flink.annotation.Internal;
import org.apache.flink.streaming.runtime.operators.sink.committables.CommittableCollector;

import java.util.Collection;

@Internal
class GlobalCommittableWrapper<CommT, GlobalCommT> {

private final CommittableCollector<CommT> committableCollector;
private final Collection<GlobalCommT> globalCommittables;

public GlobalCommittableWrapper(
CommittableCollector<CommT> committableCollector,
Collection<GlobalCommT> globalCommittables) {
this.committableCollector = committableCollector;
this.globalCommittables = globalCommittables;
}

public Collection<GlobalCommT> getGlobalCommittables() {
return globalCommittables;
}

public CommittableCollector<CommT> getCommittableCollector() {
return committableCollector;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
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.GlobalCommitter;
import org.apache.flink.api.connector.sink2.Committer;
import org.apache.flink.core.io.SimpleVersionedSerializer;
import org.apache.flink.runtime.state.StateInitializationContext;
Expand All @@ -31,6 +32,7 @@
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.streaming.api.operators.util.SimpleVersionedListState;
import org.apache.flink.streaming.api.transformations.SinkV1Adapter;
import org.apache.flink.streaming.runtime.operators.sink.committables.CheckpointCommittableManager;
import org.apache.flink.streaming.runtime.operators.sink.committables.CommittableCollector;
import org.apache.flink.streaming.runtime.operators.sink.committables.CommittableCollectorSerializer;
Expand All @@ -39,30 +41,39 @@
import org.apache.flink.streaming.runtime.tasks.StreamTask;
import org.apache.flink.util.function.SerializableSupplier;

import javax.annotation.Nullable;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;

import static org.apache.flink.util.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkState;

class GlobalCommitterOperator<CommT> extends AbstractStreamOperator<Void>
class GlobalCommitterOperator<CommT, GlobalCommT> extends AbstractStreamOperator<Void>
implements OneInputStreamOperator<CommittableMessage<CommT>, Void>, BoundedOneInput {

/** The operator's state descriptor. */
private static final ListStateDescriptor<byte[]> GLOBAL_COMMITTER_OPERATOR_RAW_STATES_DESC =
new ListStateDescriptor<>(
"global_committer_raw_states", BytePrimitiveArraySerializer.INSTANCE);
"streaming_committer_raw_states", BytePrimitiveArraySerializer.INSTANCE);

private final SerializableSupplier<Committer<CommT>> committerFactory;
private final SerializableSupplier<SimpleVersionedSerializer<CommT>>
committableSerializerFactory;

private ListState<CommittableCollector<CommT>> committableCollectorState;
private ListState<GlobalCommittableWrapper<CommT, GlobalCommT>> globalCommitterState;
private Committer<CommT> committer;
private CommittableCollector<CommT> committableCollector;
private long lastCompletedCheckpointId = -1;
private SimpleVersionedSerializer<CommT> committableSerializer;

@Nullable private GlobalCommitter<CommT, GlobalCommT> globalCommitter;
@Nullable private SimpleVersionedSerializer<GlobalCommT> globalCommittableSerializer;
private List<GlobalCommT> sinkV1State = new ArrayList<>();

GlobalCommitterOperator(
SerializableSupplier<Committer<CommT>> committerFactory,
SerializableSupplier<SimpleVersionedSerializer<CommT>> committableSerializerFactory) {
Expand All @@ -79,29 +90,55 @@ public void setup(
committer = committerFactory.get();
committableCollector = CommittableCollector.of(getRuntimeContext());
committableSerializer = committableSerializerFactory.get();
if (committer instanceof SinkV1Adapter.GlobalCommitterAdapter) {
final SinkV1Adapter<?, CommT, ?, GlobalCommT>.GlobalCommitterAdapter gc =
((SinkV1Adapter<?, CommT, ?, GlobalCommT>.GlobalCommitterAdapter) committer);
globalCommitter = gc.getGlobalCommitter();
globalCommittableSerializer = gc.getGlobalCommittableSerializer();
}
}

@Override
public void snapshotState(StateSnapshotContext context) throws Exception {
super.snapshotState(context);
// It is important to copy the collector to not mutate the state.
committableCollectorState.update(Collections.singletonList(committableCollector.copy()));
globalCommitterState.update(
Collections.singletonList(
new GlobalCommittableWrapper<>(
committableCollector.copy(), new ArrayList<>(sinkV1State))));
}

@Override
public void initializeState(StateInitializationContext context) throws Exception {
super.initializeState(context);
committableCollectorState =
final CommittableCollectorSerializer<CommT> committableCollectorSerializer =
new CommittableCollectorSerializer<>(
committableSerializer,
getRuntimeContext().getIndexOfThisSubtask(),
getRuntimeContext().getMaxNumberOfParallelSubtasks());
final SimpleVersionedSerializer<GlobalCommittableWrapper<CommT, GlobalCommT>> serializer =
new GlobalCommitterSerializer<>(
committableCollectorSerializer,
globalCommittableSerializer,
getRuntimeContext().getIndexOfThisSubtask(),
getRuntimeContext().getMaxNumberOfParallelSubtasks());
globalCommitterState =
new SimpleVersionedListState<>(
context.getOperatorStateStore()
.getListState(GLOBAL_COMMITTER_OPERATOR_RAW_STATES_DESC),
new CommittableCollectorSerializer<>(
committableSerializer,
getRuntimeContext().getIndexOfThisSubtask(),
getRuntimeContext().getMaxNumberOfParallelSubtasks()));
serializer);
if (context.isRestored()) {
committableCollectorState.get().forEach(cc -> committableCollector.merge(cc));
globalCommitterState
.get()
.forEach(
cc -> {
sinkV1State.addAll(cc.getGlobalCommittables());
committableCollector.merge(cc.getCommittableCollector());
});
lastCompletedCheckpointId = context.getRestoredCheckpointId().getAsLong();
if (globalCommitter != null) {
sinkV1State = globalCommitter.filterRecoveredCommittables(sinkV1State);
}
// try to re-commit recovered transactions as quickly as possible
commit(lastCompletedCheckpointId);
}
Expand All @@ -110,6 +147,9 @@ public void initializeState(StateInitializationContext context) throws Exception
@Override
public void notifyCheckpointComplete(long checkpointId) throws Exception {
super.notifyCheckpointComplete(checkpointId);
checkState(
globalCommitter != null || sinkV1State.isEmpty(),
"GlobalCommitter is required to commit SinkV1 state.");
lastCompletedCheckpointId = Math.max(lastCompletedCheckpointId, checkpointId);
commit(lastCompletedCheckpointId);
}
Expand All @@ -125,6 +165,9 @@ private Collection<? extends CheckpointCommittableManager<CommT>> getCommittable
}

private void commit(long checkpointId) throws IOException, InterruptedException {
if (globalCommitter != null && !sinkV1State.isEmpty()) {
sinkV1State = globalCommitter.commit(sinkV1State);
}
for (CheckpointCommittableManager<CommT> committable : getCommittables(checkpointId)) {
boolean fullyReceived = committable.getCheckpointId() == lastCompletedCheckpointId;
committable.commit(fullyReceived, committer);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,140 @@
/*
* 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.api.connector.sink2;

import org.apache.flink.annotation.Internal;
import org.apache.flink.core.io.SimpleVersionedSerialization;
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.streaming.runtime.operators.sink.committables.CommittableCollector;
import org.apache.flink.streaming.runtime.operators.sink.committables.CommittableCollectorSerializer;
import org.apache.flink.streaming.runtime.operators.sink.committables.SinkV1CommittableDeserializer;

import javax.annotation.Nullable;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;

import static org.apache.flink.util.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkState;

@Internal
class GlobalCommitterSerializer<CommT, GlobalCommT>
implements SimpleVersionedSerializer<GlobalCommittableWrapper<CommT, GlobalCommT>> {

private static final int MAGIC_NUMBER = 0xb91f252b;

private final CommittableCollectorSerializer<CommT> committableCollectorSerializer;
@Nullable private final SimpleVersionedSerializer<GlobalCommT> globalCommittableSerializer;
private final int subtaskId;
private final int numberOfSubtasks;

GlobalCommitterSerializer(
CommittableCollectorSerializer<CommT> committableCollectorSerializer,
@Nullable SimpleVersionedSerializer<GlobalCommT> globalCommittableSerializer,
int subtaskId,
int numberOfSubtasks) {
this.committableCollectorSerializer = checkNotNull(committableCollectorSerializer);
this.globalCommittableSerializer = globalCommittableSerializer;
this.subtaskId = subtaskId;
this.numberOfSubtasks = numberOfSubtasks;
}

@Override
public int getVersion() {
return 2;
}

@Override
public byte[] serialize(GlobalCommittableWrapper<CommT, GlobalCommT> obj) throws IOException {
final DataOutputSerializer out = new DataOutputSerializer(256);
out.writeInt(MAGIC_NUMBER);
if (globalCommittableSerializer != null) {
out.writeBoolean(true);
final Collection<GlobalCommT> globalCommittables = obj.getGlobalCommittables();
SimpleVersionedSerialization.writeVersionAndSerializeList(
globalCommittableSerializer, new ArrayList<>(globalCommittables), out);
} else {
out.writeBoolean(false);
}
SimpleVersionedSerialization.writeVersionAndSerialize(
committableCollectorSerializer, obj.getCommittableCollector(), out);
return out.getCopyOfBuffer();
}

@Override
public GlobalCommittableWrapper<CommT, GlobalCommT> deserialize(int version, byte[] serialized)
throws IOException {
final DataInputDeserializer in = new DataInputDeserializer(serialized);
if (version == 1) {
if (globalCommittableSerializer == null) {
throw new IllegalStateException(
"Tried to deserialize Sink V1 state without a GlobalCommittable serializer.");
}
return deserializeV1(in);
}
if (version == 2) {
validateMagicNumber(in);
return deserializeV2(in);
}
throw new IllegalStateException("Unrecognized version or corrupt state: " + version);
}

private GlobalCommittableWrapper<CommT, GlobalCommT> deserializeV1(DataInputView in)
throws IOException {
final List<GlobalCommT> globalCommittables =
SinkV1CommittableDeserializer.readVersionAndDeserializeList(
globalCommittableSerializer, in);
return new GlobalCommittableWrapper<>(
new CommittableCollector<>(subtaskId, numberOfSubtasks), globalCommittables);
}

private GlobalCommittableWrapper<CommT, GlobalCommT> deserializeV2(DataInputView in)
throws IOException {
final boolean withGlobalCommittableSerializer = in.readBoolean();
List<GlobalCommT> globalCommittables;
if (globalCommittableSerializer == null) {
checkState(
!withGlobalCommittableSerializer,
"Trying to recover state from a GlobalCommittable serializer without specifying one.");
globalCommittables = Collections.emptyList();
} else {
globalCommittables =
SimpleVersionedSerialization.readVersionAndDeserializeList(
globalCommittableSerializer, in);
}
return new GlobalCommittableWrapper<>(
SimpleVersionedSerialization.readVersionAndDeSerialize(
committableCollectorSerializer, in),
globalCommittables);
}

private static void validateMagicNumber(DataInputView in) throws IOException {
final int magicNumber = in.readInt();
if (magicNumber != MAGIC_NUMBER) {
throw new IllegalStateException(
String.format("Corrupt data: Unexpected magic number %08X", magicNumber));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -417,12 +417,16 @@ public List<CommT> commit(List<CommT> committables) {
public void close() throws Exception {}
}

private class GlobalCommitterAdapter implements Committer<CommT> {
GlobalCommitter<CommT, GlobalCommT> globalCommitter;
/** Simulate the global committer behaviour with a committer. */
@Internal
public class GlobalCommitterAdapter implements Committer<CommT> {
final GlobalCommitter<CommT, GlobalCommT> globalCommitter;
final SimpleVersionedSerializer<GlobalCommT> globalCommittableSerializer;

GlobalCommitterAdapter() {
try {
globalCommitter = sink.createGlobalCommitter().get();
globalCommittableSerializer = sink.getGlobalCommittableSerializer().get();
} catch (IOException e) {
throw new UncheckedIOException("Cannot create global committer", e);
}
Expand All @@ -436,16 +440,32 @@ public void close() throws Exception {
@Override
public void commit(Collection<CommitRequest<CommT>> committables)
throws IOException, InterruptedException {
if (committables.isEmpty()) {
return;
}

List<CommT> rawCommittables =
committables.stream()
.map(CommitRequest::getCommittable)
.collect(Collectors.toList());
List<GlobalCommT> globalCommittables =
Collections.singletonList(globalCommitter.combine(rawCommittables));
List<GlobalCommT> failures = globalCommitter.commit(globalCommittables);
// Only committables are retriable so the complete batch of committables is retried
// because we cannot trace back the committable to which global committable it belongs.
// This might lead to committing the same global committable twice, but we assume that
// the GlobalCommitter commit call is idempotent.
if (!failures.isEmpty()) {
committables.forEach(CommitRequest::retryLater);
}
}

public GlobalCommitter<CommT, GlobalCommT> getGlobalCommitter() {
return globalCommitter;
}

public SimpleVersionedSerializer<GlobalCommT> getGlobalCommittableSerializer() {
return globalCommittableSerializer;
}
}
}
Loading

0 comments on commit 955e5ff

Please sign in to comment.