Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-25920] Ignore duplicate EOI in SinkWriter [1.20] #25619

Merged
merged 8 commits into from
Nov 13, 2024

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -46,8 +46,6 @@
import org.apache.flink.streaming.api.operators.util.SimpleVersionedListState;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;

import javax.annotation.Nullable;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
Expand Down Expand Up @@ -138,15 +136,15 @@ public void processElement(StreamRecord<CompactorRequest> element) throws Except
@Override
public void endInput() throws Exception {
// add collecting requests into the final snapshot
checkpointRequests.put(Long.MAX_VALUE, collectingRequests);
checkpointRequests.put(CommittableMessage.EOI, collectingRequests);
collectingRequests = new ArrayList<>();

// submit all requests and wait until they are done
submitUntil(Long.MAX_VALUE);
submitUntil(CommittableMessage.EOI);
assert checkpointRequests.isEmpty();

getAllTasksFuture().join();
emitCompacted(null);
emitCompacted(CommittableMessage.EOI);
assert compactingRequests.isEmpty();
}

Expand Down Expand Up @@ -223,7 +221,7 @@ private void submitUntil(long checkpointId) {
canSubmit.clear();
}

private void emitCompacted(@Nullable Long checkpointId) throws Exception {
private void emitCompacted(long checkpointId) throws Exception {
List<FileSinkCommittable> compacted = new ArrayList<>();
Iterator<Tuple2<CompactorRequest, CompletableFuture<Iterable<FileSinkCommittable>>>> iter =
compactingRequests.iterator();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -171,7 +171,7 @@ private void appendCompactingResultsToSummary(CommittableSummary<FileSinkCommitt
new CommittableSummary<>(
summary.getSubtaskId(),
summary.getNumberOfSubtasks(),
getCheckpointId(summary),
summary.getCheckpointIdOrEOI(),
summary.getNumberOfCommittables() + results.size(),
summary.getNumberOfPendingCommittables() + results.size(),
summary.getNumberOfFailedCommittables())));
Expand All @@ -180,7 +180,7 @@ private void appendCompactingResultsToSummary(CommittableSummary<FileSinkCommitt
new StreamRecord<>(
new CommittableWithLineage<>(
committable,
getCheckpointId(summary),
summary.getCheckpointIdOrEOI(),
summary.getSubtaskId())));
}
}
Expand All @@ -204,7 +204,7 @@ private void handleHiddenCommittable(CommittableWithLineage<FileSinkCommittable>
// cleanup request to the next summary, since the count of pending committable
// for this checkpoint is immutable now
Iterable<FileSinkCommittable> result = submit(request).get();
Long checkpointId = getCheckpointId(message);
Long checkpointId = message.getCheckpointIdOrEOI();
boolean pendingFileSent = false;
for (FileSinkCommittable c : result) {
if (c.hasPendingFile()) {
Expand Down Expand Up @@ -275,10 +275,6 @@ public void snapshotState(StateSnapshotContext context) throws Exception {
remainingRequestsState.update(Collections.singletonList(requestsMap));
}

private Long getCheckpointId(CommittableMessage<FileSinkCommittable> message) {
return message.getCheckpointId().isPresent() ? message.getCheckpointId().getAsLong() : null;
}

private CompletableFuture<Iterable<FileSinkCommittable>> submit(CompactorRequest request) {
CompletableFuture<Iterable<FileSinkCommittable>> resultFuture = new CompletableFuture<>();
compactService.submit(request, resultFuture);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,9 @@

package org.apache.flink.connector.file.table.stream;

import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.typeutils.base.BooleanSerializer;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.state.StateInitializationContext;
import org.apache.flink.runtime.state.StateSnapshotContext;
Expand All @@ -33,6 +36,12 @@
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;

import org.apache.flink.shaded.guava31.com.google.common.collect.Lists;

import java.util.List;

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

/**
* Operator for file system sink. It is a operator version of {@link StreamingFileSink}. It can send
* file and bucket information to downstream.
Expand All @@ -58,6 +67,16 @@ public abstract class AbstractStreamingWriter<IN, OUT> extends AbstractStreamOpe

protected transient long currentWatermark;

/**
* Used to remember that EOI has already happened so that we don't emit the last committables of
* the final checkpoints twice.
*/
private static final ListStateDescriptor<Boolean> END_OF_INPUT_STATE_DESC =
new ListStateDescriptor<>("end_of_input_state", BooleanSerializer.INSTANCE);

private boolean endOfInput;
private ListState<Boolean> endOfInputState;

public AbstractStreamingWriter(
long bucketCheckInterval,
StreamingFileSink.BucketsBuilder<
Expand Down Expand Up @@ -123,6 +142,27 @@ public void bucketInactive(Bucket<IN, String> bucket) {
bucketCheckInterval);

currentWatermark = Long.MIN_VALUE;

// Figure out if we have seen end of input before and if we should anything downstream. We
// have the following
// cases:
// 1. state is empty:
// - First time initialization
// - Restoring from a previous version of Flink that didn't handle EOI
// - Upscaled from a final or regular checkpoint
// In all cases, we regularly handle EOI, potentially resulting in unnecessary .
// 2. state is not empty:
// - This implies Flink restores from a version that handles EOI.
// - If there is one entry, no rescaling happened (for this subtask), so if it's true,
// we recover from a final checkpoint (for this subtask) and can ignore another EOI
// else we have a regular checkpoint.
// - If there are multiple entries, Flink downscaled, and we need to check if all are
// true and do the same as above. As soon as one entry is false, we regularly start
// the writer and potentially emit duplicate summaries if we indeed recovered from a
// final checkpoint.
endOfInputState = context.getOperatorStateStore().getListState(END_OF_INPUT_STATE_DESC);
List<Boolean> previousState = Lists.newArrayList(endOfInputState.get());
endOfInput = !previousState.isEmpty() && !previousState.contains(false);
}

@Override
Expand All @@ -139,6 +179,7 @@ public void processWatermark(Watermark mark) throws Exception {

@Override
public void processElement(StreamRecord<IN> element) throws Exception {
checkState(!endOfInput, "Received element after endOfInput: %s", element);
helper.onElement(
element.getValue(),
getProcessingTimeService().getCurrentProcessingTime(),
Expand All @@ -149,15 +190,20 @@ public void processElement(StreamRecord<IN> element) throws Exception {
@Override
public void notifyCheckpointComplete(long checkpointId) throws Exception {
super.notifyCheckpointComplete(checkpointId);
commitUpToCheckpoint(checkpointId);
if (!this.endOfInput) {
commitUpToCheckpoint(checkpointId);
}
}

@Override
public void endInput() throws Exception {
buckets.onProcessingTime(Long.MAX_VALUE);
helper.snapshotState(Long.MAX_VALUE);
output.emitWatermark(new Watermark(Long.MAX_VALUE));
commitUpToCheckpoint(Long.MAX_VALUE);
if (!this.endOfInput) {
this.endOfInput = true;
buckets.onProcessingTime(Long.MAX_VALUE);
helper.snapshotState(Long.MAX_VALUE);
output.emitWatermark(new Watermark(Long.MAX_VALUE));
commitUpToCheckpoint(Long.MAX_VALUE);
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/*
* 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.core.io;

import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataInputDeserializer;
import org.apache.flink.core.memory.DataOutputSerializer;

import java.io.IOException;
import java.io.Serializable;

/**
* Adapter for {@link TypeSerializer} to {@link SimpleVersionedSerializer}. The implementation is
* naive and should only be used for non-critical paths and tests.
*/
@Internal
public class SimpleVersionedSerializerAdapter<T>
implements SimpleVersionedSerializer<T>, Serializable {
private final TypeSerializer<T> serializer;

public SimpleVersionedSerializerAdapter(TypeSerializer<T> serializer) {
this.serializer = serializer;
}

public int getVersion() {
return serializer.snapshotConfiguration().getCurrentVersion();
}

public byte[] serialize(T value) throws IOException {
DataOutputSerializer dataOutputSerializer = new DataOutputSerializer(10);
serializer.serialize(value, dataOutputSerializer);
return dataOutputSerializer.getCopyOfBuffer();
}

public T deserialize(int version, byte[] serialized) throws IOException {
DataInputDeserializer dataInputDeserializer = new DataInputDeserializer(serialized);
T value = serializer.deserialize(dataInputDeserializer);
dataInputDeserializer.releaseArrays();
return value;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,15 +22,33 @@

import java.util.OptionalLong;

/** The message send from {@link SinkWriter} to {@link Committer}. */
/** The message send from {@code SinkWriter} to {@code Committer}. */
@Experimental
public interface CommittableMessage<CommT> {
/**
* Special value for checkpointId for the end of input in case of batch commit or final
* checkpoint.
*/
long EOI = Long.MAX_VALUE;

/** The subtask that created this committable. */
int getSubtaskId();

/**
* Returns the checkpoint id or empty if the message does not belong to a checkpoint. In that
* case, the committable was created at the end of input (e.g., in batch mode).
*
* @see #getCheckpointIdOrEOI()
*/
@Deprecated
default OptionalLong getCheckpointId() {
long checkpointIdOrEOI = getCheckpointIdOrEOI();
return checkpointIdOrEOI == EOI ? OptionalLong.empty() : OptionalLong.of(checkpointIdOrEOI);
}

/**
* Returns the checkpoint id or EOI if this message belong to the final checkpoint or the batch
* commit.
*/
OptionalLong getCheckpointId();
long getCheckpointIdOrEOI();
}
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,6 @@ public class CommittableMessageSerializer<CommT>
@VisibleForTesting static final int VERSION = 1;
private static final int COMMITTABLE = 1;
private static final int SUMMARY = 2;
private static final long EOI = Long.MAX_VALUE;

private final SimpleVersionedSerializer<CommT> committableSerializer;

Expand All @@ -64,14 +63,14 @@ public byte[] serialize(CommittableMessage<CommT> obj) throws IOException {
committableSerializer,
((CommittableWithLineage<CommT>) obj).getCommittable(),
out);
writeCheckpointId(out, obj);
out.writeLong(obj.getCheckpointIdOrEOI());
out.writeInt(obj.getSubtaskId());
} else if (obj instanceof CommittableSummary) {
out.writeByte(SUMMARY);
out.writeInt(obj.getSubtaskId());
CommittableSummary<?> committableSummary = (CommittableSummary<?>) obj;
out.writeInt(committableSummary.getNumberOfSubtasks());
writeCheckpointId(out, obj);
out.writeLong(obj.getCheckpointIdOrEOI());
out.writeInt(committableSummary.getNumberOfCommittables());
out.writeInt(committableSummary.getNumberOfPendingCommittables());
out.writeInt(committableSummary.getNumberOfFailedCommittables());
Expand All @@ -91,13 +90,13 @@ public CommittableMessage<CommT> deserialize(int version, byte[] serialized)
return new CommittableWithLineage<>(
SimpleVersionedSerialization.readVersionAndDeSerialize(
committableSerializer, in),
readCheckpointId(in),
in.readLong(),
in.readInt());
case SUMMARY:
return new CommittableSummary<>(
in.readInt(),
in.readInt(),
readCheckpointId(in),
in.readLong(),
in.readInt(),
in.readInt(),
in.readInt());
Expand All @@ -109,14 +108,4 @@ public CommittableMessage<CommT> deserialize(int version, byte[] serialized)
+ StringUtils.byteToHexString(serialized));
}
}

private void writeCheckpointId(DataOutputSerializer out, CommittableMessage<CommT> obj)
throws IOException {
out.writeLong(obj.getCheckpointId().orElse(EOI));
}

private Long readCheckpointId(DataInputDeserializer in) throws IOException {
long checkpointId = in.readLong();
return checkpointId == EOI ? null : checkpointId;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,6 @@ private CommittableMessageTypeInfo(
* @param committableSerializerFactory factory to create the serializer for a {@link
* CommittableMessage}
* @param <CommT> type of the committable
* @return
*/
public static <CommT> TypeInformation<CommittableMessage<CommT>> of(
SerializableSupplier<SimpleVersionedSerializer<CommT>> committableSerializerFactory) {
Expand Down Expand Up @@ -86,6 +85,7 @@ public int getTotalFields() {
return 1;
}

@SuppressWarnings({"unchecked", "rawtypes"})
@Override
public Class<CommittableMessage<CommT>> getTypeClass() {
return (Class) CommittableMessage.class;
Expand Down
Loading