Skip to content

[FLINK-38216][checkpoint][refactor] Split EndOfChannelStateEvent into EndOfInputChannelStateEvent and EndOfOutputChannelStateEvent #26900

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

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,8 @@
import org.apache.flink.runtime.io.network.buffer.BufferConsumer;
import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
import org.apache.flink.runtime.io.network.partition.consumer.EndOfChannelStateEvent;
import org.apache.flink.runtime.io.network.partition.consumer.EndOfInputChannelStateEvent;
import org.apache.flink.runtime.io.network.partition.consumer.EndOfOutputChannelStateEvent;
import org.apache.flink.runtime.state.CheckpointStorageLocationReference;
import org.apache.flink.util.InstantiationUtil;

Expand All @@ -70,7 +71,7 @@ public class EventSerializer {

private static final int CANCEL_CHECKPOINT_MARKER_EVENT = 4;

private static final int END_OF_CHANNEL_STATE_EVENT = 5;
private static final int END_OF_OUTPUT_CHANNEL_STATE_EVENT = 5;

private static final int ANNOUNCEMENT_EVENT = 6;

Expand All @@ -84,6 +85,8 @@ public class EventSerializer {

private static final int GENERALIZED_WATERMARK_EVENT = 11;

private static final int END_OF_INPUT_CHANNEL_STATE_EVENT = 12;

private static final byte CHECKPOINT_TYPE_CHECKPOINT = 0;

private static final byte CHECKPOINT_TYPE_SAVEPOINT = 1;
Expand All @@ -109,8 +112,10 @@ public static ByteBuffer toSerializedEvent(AbstractEvent event) throws IOExcepti
return serializeCheckpointBarrier((CheckpointBarrier) event);
} else if (eventClass == EndOfSuperstepEvent.class) {
return ByteBuffer.wrap(new byte[] {0, 0, 0, END_OF_SUPERSTEP_EVENT});
} else if (eventClass == EndOfChannelStateEvent.class) {
return ByteBuffer.wrap(new byte[] {0, 0, 0, END_OF_CHANNEL_STATE_EVENT});
} else if (eventClass == EndOfOutputChannelStateEvent.class) {
return ByteBuffer.wrap(new byte[] {0, 0, 0, END_OF_OUTPUT_CHANNEL_STATE_EVENT});
} else if (eventClass == EndOfInputChannelStateEvent.class) {
return ByteBuffer.wrap(new byte[] {0, 0, 0, END_OF_INPUT_CHANNEL_STATE_EVENT});
} else if (eventClass == EndOfData.class) {
return ByteBuffer.wrap(
new byte[] {
Expand Down Expand Up @@ -197,8 +202,10 @@ public static AbstractEvent fromSerializedEvent(ByteBuffer buffer, ClassLoader c
return deserializeCheckpointBarrier(buffer);
} else if (type == END_OF_SUPERSTEP_EVENT) {
return EndOfSuperstepEvent.INSTANCE;
} else if (type == END_OF_CHANNEL_STATE_EVENT) {
return EndOfChannelStateEvent.INSTANCE;
} else if (type == END_OF_OUTPUT_CHANNEL_STATE_EVENT) {
return EndOfOutputChannelStateEvent.INSTANCE;
} else if (type == END_OF_INPUT_CHANNEL_STATE_EVENT) {
return EndOfInputChannelStateEvent.INSTANCE;
} else if (type == END_OF_USER_RECORDS_EVENT) {
return new EndOfData(StopMode.values()[buffer.get()]);
} else if (type == CANCEL_CHECKPOINT_MARKER_EVENT) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
import org.apache.flink.runtime.io.network.api.EndOfData;
import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
import org.apache.flink.runtime.io.network.partition.consumer.EndOfChannelStateEvent;
import org.apache.flink.runtime.io.network.partition.consumer.EndOfOutputChannelStateEvent;

import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator;
Expand Down Expand Up @@ -408,7 +408,7 @@ public boolean isPartialRecord() {
public static DataType getDataType(AbstractEvent event, boolean hasPriority) {
if (hasPriority) {
return PRIORITIZED_EVENT_BUFFER;
} else if (event instanceof EndOfChannelStateEvent) {
} else if (event instanceof EndOfOutputChannelStateEvent) {
return RECOVERY_COMPLETION;
} else if (event instanceof EndOfData) {
return END_OF_DATA;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
import org.apache.flink.runtime.io.network.buffer.BufferConsumer;
import org.apache.flink.runtime.io.network.buffer.BufferPool;
import org.apache.flink.runtime.io.network.logger.NetworkActionsLogger;
import org.apache.flink.runtime.io.network.partition.consumer.EndOfChannelStateEvent;
import org.apache.flink.runtime.io.network.partition.consumer.EndOfOutputChannelStateEvent;
import org.apache.flink.util.function.SupplierWithException;

import javax.annotation.Nullable;
Expand Down Expand Up @@ -262,7 +262,7 @@ public void finishReadRecoveredState(boolean notifyAndBlockOnCompletion) throws
return;
}
try (BufferConsumer eventBufferConsumer =
EventSerializer.toBufferConsumer(EndOfChannelStateEvent.INSTANCE, false)) {
EventSerializer.toBufferConsumer(EndOfOutputChannelStateEvent.INSTANCE, false)) {
for (int i = 0; i < subpartitions.length; i++) {
if (((PipelinedSubpartition) subpartitions[i]).isSupportChannelStateRecover()) {
addToSubpartition(i, eventBufferConsumer.copy(), 0);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
/*
* 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.runtime.io.network.partition.consumer;

import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
import org.apache.flink.runtime.event.RuntimeEvent;

/** Marks the end of recovered state of {@link RecoveredInputChannel} of this subtask. */
public class EndOfInputChannelStateEvent extends RuntimeEvent {

/** The singleton instance of this event. */
public static final EndOfInputChannelStateEvent INSTANCE = new EndOfInputChannelStateEvent();

// ------------------------------------------------------------------------

// not instantiable
private EndOfInputChannelStateEvent() {}

// ------------------------------------------------------------------------

@Override
public void read(DataInputView in) {
// Nothing to do here
}

@Override
public void write(DataOutputView out) {
// Nothing to do here
}

// ------------------------------------------------------------------------

@Override
public int hashCode() {
return 20250813;
}

@Override
public boolean equals(Object obj) {
return obj != null && obj.getClass() == EndOfInputChannelStateEvent.class;
}

@Override
public String toString() {
return getClass().getSimpleName();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,19 +23,19 @@
import org.apache.flink.runtime.event.RuntimeEvent;

/**
* Marks the end of recovered state of {@link RecoveredInputChannel} of this subtask or {@link
* Marks the end of recovered state of {@link
* org.apache.flink.runtime.io.network.partition.ResultSubpartition ResultSubpartition} on the
* upstream.
*/
public class EndOfChannelStateEvent extends RuntimeEvent {
public class EndOfOutputChannelStateEvent extends RuntimeEvent {

/** The singleton instance of this event. */
public static final EndOfChannelStateEvent INSTANCE = new EndOfChannelStateEvent();
public static final EndOfOutputChannelStateEvent INSTANCE = new EndOfOutputChannelStateEvent();

// ------------------------------------------------------------------------

// not instantiable
private EndOfChannelStateEvent() {}
private EndOfOutputChannelStateEvent() {}

// ------------------------------------------------------------------------

Expand All @@ -58,7 +58,7 @@ public int hashCode() {

@Override
public boolean equals(Object obj) {
return obj != null && obj.getClass() == EndOfChannelStateEvent.class;
return obj != null && obj.getClass() == EndOfOutputChannelStateEvent.class;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -154,7 +154,8 @@ public void onRecoveredStateBuffer(Buffer buffer) {
}

public void finishReadRecoveredState() throws IOException {
onRecoveredStateBuffer(EventSerializer.toBuffer(EndOfChannelStateEvent.INSTANCE, false));
onRecoveredStateBuffer(
EventSerializer.toBuffer(EndOfInputChannelStateEvent.INSTANCE, false));
bufferManager.releaseFloatingBuffers();
LOG.debug("{}/{} finished recovering input.", inputGate.getOwningTaskName(), channelInfo);
}
Expand All @@ -172,22 +173,22 @@ private BufferAndAvailability getNextRecoveredStateBuffer() throws IOException {

if (next == null) {
return null;
} else if (isEndOfChannelStateEvent(next)) {
} else if (isEndOfInputChannelStateEvent(next)) {
stateConsumedFuture.complete(null);
return null;
} else {
return new BufferAndAvailability(next, nextDataType, 0, sequenceNumber++);
}
}

private boolean isEndOfChannelStateEvent(Buffer buffer) throws IOException {
private boolean isEndOfInputChannelStateEvent(Buffer buffer) throws IOException {
if (buffer.isBuffer()) {
return false;
}

AbstractEvent event = EventSerializer.fromBuffer(buffer, getClass().getClassLoader());
buffer.setReaderIndex(0);
return event.getClass() == EndOfChannelStateEvent.class;
return event.getClass() == EndOfInputChannelStateEvent.class;
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@
import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
import org.apache.flink.runtime.io.network.partition.consumer.EndOfChannelStateEvent;
import org.apache.flink.runtime.io.network.partition.consumer.EndOfOutputChannelStateEvent;
import org.apache.flink.runtime.plugable.DeserializationDelegate;
import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate;
import org.apache.flink.streaming.runtime.io.checkpointing.CheckpointedInputGate;
Expand Down Expand Up @@ -270,7 +270,7 @@ protected DataInputStatus processEvent(BufferOrEvent bufferOrEvent, DataOutput<T
if (checkpointedInputGate.isFinished()) {
return DataInputStatus.END_OF_INPUT;
}
} else if (event.getClass() == EndOfChannelStateEvent.class) {
} else if (event.getClass() == EndOfOutputChannelStateEvent.class) {
if (checkpointedInputGate.allChannelsRecovered()) {
return DataInputStatus.END_OF_RECOVERY;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@
import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
import org.apache.flink.runtime.io.network.api.EventAnnouncement;
import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
import org.apache.flink.runtime.io.network.partition.consumer.EndOfChannelStateEvent;
import org.apache.flink.runtime.io.network.partition.consumer.EndOfOutputChannelStateEvent;
import org.apache.flink.runtime.io.network.partition.consumer.InputChannel;
import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
import org.apache.flink.streaming.runtime.io.StreamTaskNetworkInput;
Expand Down Expand Up @@ -200,7 +200,7 @@ private Optional<BufferOrEvent> handleEvent(BufferOrEvent bufferOrEvent) throws
announcedBarrier,
eventAnnouncement.getSequenceNumber(),
bufferOrEvent.getChannelInfo());
} else if (bufferOrEvent.getEvent().getClass() == EndOfChannelStateEvent.class) {
} else if (bufferOrEvent.getEvent().getClass() == EndOfOutputChannelStateEvent.class) {
upstreamRecoveryTracker.handleEndOfRecovery(bufferOrEvent.getChannelInfo());
}
return Optional.of(bufferOrEvent);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,8 @@
import org.apache.flink.runtime.io.network.api.SubtaskConnectionDescriptor;
import org.apache.flink.runtime.io.network.buffer.Buffer;
import org.apache.flink.runtime.io.network.buffer.BufferConsumer;
import org.apache.flink.runtime.io.network.partition.consumer.EndOfInputChannelStateEvent;
import org.apache.flink.runtime.io.network.partition.consumer.EndOfOutputChannelStateEvent;
import org.apache.flink.runtime.io.network.util.TestTaskEvent;
import org.apache.flink.runtime.state.CheckpointStorageLocationReference;

Expand Down Expand Up @@ -120,6 +122,9 @@ class EventSerializerTest {
new RecoveryMetadata(3),
new WatermarkEvent(new LongWatermark(42L, "test"), false),
new WatermarkEvent(new BoolWatermark(true, "test"), true),
new WatermarkEvent(new BoolWatermark(true, "test"), true),
EndOfInputChannelStateEvent.INSTANCE,
EndOfOutputChannelStateEvent.INSTANCE,
};

@Test
Expand Down Expand Up @@ -161,6 +166,9 @@ void testToBufferConsumer() throws IOException {
assertThat(bufferConsumer.build().getDataType())
.isEqualTo(Buffer.DataType.UNALIGNED_WATERMARK_EVENT);
}
} else if (evt instanceof EndOfOutputChannelStateEvent) {
assertThat(bufferConsumer.build().getDataType())
.isEqualTo(Buffer.DataType.RECOVERY_COMPLETION);
} else {
assertThat(bufferConsumer.build().getDataType())
.isEqualTo(Buffer.DataType.EVENT_BUFFER);
Expand Down Expand Up @@ -191,6 +199,8 @@ void testToBuffer() throws IOException {
assertThat(buffer.getDataType())
.isEqualTo(Buffer.DataType.UNALIGNED_WATERMARK_EVENT);
}
} else if (evt instanceof EndOfOutputChannelStateEvent) {
assertThat(buffer.getDataType()).isEqualTo(Buffer.DataType.RECOVERY_COMPLETION);
} else {
assertThat(buffer.getDataType()).isEqualTo(Buffer.DataType.EVENT_BUFFER);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@
import org.apache.flink.runtime.io.network.buffer.Buffer;
import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
import org.apache.flink.runtime.io.network.partition.consumer.EndOfChannelStateEvent;
import org.apache.flink.runtime.io.network.partition.consumer.EndOfOutputChannelStateEvent;
import org.apache.flink.runtime.io.network.partition.consumer.InputChannelBuilder;
import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel;
import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
Expand Down Expand Up @@ -86,7 +86,7 @@ void testUpstreamResumedUponEndOfRecovery() throws Exception {
enqueueEndOfState(gate, channelIndex);
Optional<BufferOrEvent> bufferOrEvent = gate.pollNext();
while (bufferOrEvent.isPresent()
&& bufferOrEvent.get().getEvent() instanceof EndOfChannelStateEvent
&& bufferOrEvent.get().getEvent() instanceof EndOfOutputChannelStateEvent
&& !gate.allChannelsRecovered()) {
bufferOrEvent = gate.pollNext();
}
Expand All @@ -97,7 +97,7 @@ void testUpstreamResumedUponEndOfRecovery() throws Exception {
Optional<BufferOrEvent> polled = gate.pollNext();
assertThat(polled).isPresent();
assertThat(polled.get().isEvent()).isTrue();
assertThat(polled.get().getEvent()).isEqualTo(EndOfChannelStateEvent.INSTANCE);
assertThat(polled.get().getEvent()).isEqualTo(EndOfOutputChannelStateEvent.INSTANCE);
assertThat(resumeCounter.getNumResumed()).isEqualTo(numberOfChannels);
assertThat(gate.pollNext())
.as("should only be a single event no matter of what is the number of channels")
Expand Down Expand Up @@ -282,7 +282,7 @@ private void assertAddedInputSize(

private void enqueueEndOfState(CheckpointedInputGate checkpointedInputGate, int channelIndex)
throws IOException {
enqueue(checkpointedInputGate, channelIndex, EndOfChannelStateEvent.INSTANCE);
enqueue(checkpointedInputGate, channelIndex, EndOfOutputChannelStateEvent.INSTANCE);
}

private void enqueueEndOfPartition(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@
import org.apache.flink.runtime.io.network.buffer.BufferConsumer;
import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
import org.apache.flink.runtime.io.network.partition.consumer.CheckpointableInput;
import org.apache.flink.runtime.io.network.partition.consumer.EndOfChannelStateEvent;
import org.apache.flink.runtime.io.network.partition.consumer.EndOfOutputChannelStateEvent;
import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
import org.apache.flink.runtime.io.network.partition.consumer.StreamTestSingleInputGate;
import org.apache.flink.runtime.mailbox.SyncMailboxExecutor;
Expand Down Expand Up @@ -229,9 +229,9 @@ void testInputStatusAfterEndOfRecovery() throws Exception {

inputGate.sendElement(new StreamRecord<>(42L), 0);
assertThat(input.emitNext(output)).isEqualTo(DataInputStatus.MORE_AVAILABLE);
inputGate.sendEvent(EndOfChannelStateEvent.INSTANCE, 0);
inputGate.sendEvent(EndOfOutputChannelStateEvent.INSTANCE, 0);
assertThat(input.emitNext(output)).isEqualTo(DataInputStatus.MORE_AVAILABLE);
inputGate.sendEvent(EndOfChannelStateEvent.INSTANCE, 1);
inputGate.sendEvent(EndOfOutputChannelStateEvent.INSTANCE, 1);
assertThat(input.emitNext(output)).isEqualTo(DataInputStatus.END_OF_RECOVERY);
}

Expand Down