2828
2929import org .junit .jupiter .api .Test ;
3030
31+ import java .io .IOException ;
3132import java .util .ArrayDeque ;
3233
3334import static org .apache .flink .runtime .checkpoint .CheckpointOptions .unaligned ;
3435import static org .apache .flink .runtime .state .CheckpointStorageLocationReference .getDefault ;
36+ import static org .assertj .core .api .Assertions .assertThat ;
3537import static org .assertj .core .api .Assertions .assertThatThrownBy ;
3638
3739/** Tests for {@link RecoveredInputChannel}. */
3840class RecoveredInputChannelTest {
3941
4042 @ Test
41- void testConversionOnlyPossibleAfterConsumed () {
42- assertThatThrownBy (() -> buildChannel ().toInputChannel ())
43- .isInstanceOf (IllegalStateException .class );
43+ void testConversionOnlyPossibleAfterConsumedWhenConfigDisabled () {
44+ // When config is disabled, toInputChannel() checks stateConsumedFuture
45+ assertThatThrownBy (() -> buildChannel (false ).toInputChannel ())
46+ .isInstanceOf (IllegalStateException .class )
47+ .hasMessageContaining ("recovered state is not fully consumed" );
48+ }
49+
50+ @ Test
51+ void testConversionOnlyPossibleAfterFilteringWhenConfigEnabled () {
52+ // When config is enabled, toInputChannel() checks bufferFilteringCompleteFuture
53+ assertThatThrownBy (() -> buildChannel (true ).toInputChannel ())
54+ .isInstanceOf (IllegalStateException .class )
55+ .hasMessageContaining ("buffer filtering is not complete" );
4456 }
4557
4658 @ Test
4759 void testRequestPartitionsImpossible () {
48- assertThatThrownBy (() -> buildChannel ().requestSubpartitions ())
60+ assertThatThrownBy (() -> buildChannel (false ).requestSubpartitions ())
4961 .isInstanceOf (UnsupportedOperationException .class );
5062 }
5163
5264 @ Test
5365 void testCheckpointStartImpossible () {
5466 assertThatThrownBy (
5567 () ->
56- buildChannel ()
68+ buildChannel (false )
5769 .checkpointStarted (
5870 new CheckpointBarrier (
5971 0L ,
@@ -64,10 +76,101 @@ void testCheckpointStartImpossible() {
6476 .isInstanceOf (CheckpointException .class );
6577 }
6678
67- private RecoveredInputChannel buildChannel () {
79+ @ Test
80+ void testToInputChannelAllowedWhenBufferFilteringCompleteAndConfigEnabled () throws IOException {
81+ // When config is enabled, conversion is allowed when bufferFilteringCompleteFuture is done
82+ TestableRecoveredInputChannel channel = buildTestableChannel (true );
83+
84+ // Initially, conversion should fail
85+ assertThatThrownBy (() -> channel .toInputChannel ())
86+ .isInstanceOf (IllegalStateException .class )
87+ .hasMessageContaining ("buffer filtering is not complete" );
88+
89+ // After finishReadRecoveredState(), bufferFilteringCompleteFuture should be done
90+ channel .finishReadRecoveredState ();
91+ assertThat (channel .getBufferFilteringCompleteFuture ()).isDone ();
92+ assertThat (channel .getStateConsumedFuture ()).isNotDone ();
93+
94+ // Conversion should now succeed (no exception)
95+ InputChannel converted = channel .toInputChannel ();
96+ assertThat (converted ).isNotNull ();
97+ }
98+
99+ @ Test
100+ void testToInputChannelAllowedWhenStateConsumedAndConfigDisabled () throws IOException {
101+ // When config is disabled, conversion is allowed when stateConsumedFuture is done
102+ TestableRecoveredInputChannel channel = buildTestableChannel (false );
103+
104+ // Initially, conversion should fail
105+ assertThatThrownBy (() -> channel .toInputChannel ())
106+ .isInstanceOf (IllegalStateException .class )
107+ .hasMessageContaining ("recovered state is not fully consumed" );
108+
109+ // After finishReadRecoveredState(), bufferFilteringCompleteFuture should NOT be done
110+ // because config is disabled
111+ channel .finishReadRecoveredState ();
112+ assertThat (channel .getBufferFilteringCompleteFuture ()).isNotDone ();
113+ assertThat (channel .getStateConsumedFuture ()).isNotDone ();
114+
115+ // Conversion should still fail because stateConsumedFuture is not done
116+ assertThatThrownBy (() -> channel .toInputChannel ())
117+ .isInstanceOf (IllegalStateException .class )
118+ .hasMessageContaining ("recovered state is not fully consumed" );
119+
120+ // Consume the EndOfInputChannelStateEvent to complete stateConsumedFuture.
121+ // getNextBuffer() returns empty when it encounters the event internally.
122+ assertThat (channel .getNextBuffer ()).isNotPresent ();
123+ assertThat (channel .getStateConsumedFuture ()).isDone ();
124+
125+ // Now conversion should succeed
126+ InputChannel converted = channel .toInputChannel ();
127+ assertThat (converted ).isNotNull ();
128+ }
129+
130+ @ Test
131+ void testBufferFilteringCompleteFutureOnlyCompletesWhenConfigEnabled () throws IOException {
132+ // Config enabled: finishReadRecoveredState() completes bufferFilteringCompleteFuture
133+ RecoveredInputChannel channelEnabled = buildChannel (true );
134+ assertThat (channelEnabled .getBufferFilteringCompleteFuture ()).isNotDone ();
135+ channelEnabled .finishReadRecoveredState ();
136+ assertThat (channelEnabled .getBufferFilteringCompleteFuture ()).isDone ();
137+
138+ // Config disabled: finishReadRecoveredState() does NOT complete
139+ // bufferFilteringCompleteFuture
140+ RecoveredInputChannel channelDisabled = buildChannel (false );
141+ assertThat (channelDisabled .getBufferFilteringCompleteFuture ()).isNotDone ();
142+ channelDisabled .finishReadRecoveredState ();
143+ assertThat (channelDisabled .getBufferFilteringCompleteFuture ()).isNotDone ();
144+ }
145+
146+ @ Test
147+ void testStateConsumedFutureCompletesAfterConsumingAllBuffers () throws IOException {
148+ // This test verifies that stateConsumedFuture completes after consuming
149+ // EndOfInputChannelStateEvent regardless of the config setting
150+ for (boolean configEnabled : new boolean [] {true , false }) {
151+ RecoveredInputChannel channel = buildChannel (configEnabled );
152+
153+ assertThat (channel .getStateConsumedFuture ()).isNotDone ();
154+
155+ channel .finishReadRecoveredState ();
156+ assertThat (channel .getStateConsumedFuture ()).isNotDone ();
157+
158+ // Consuming the EndOfInputChannelStateEvent should complete the future.
159+ // getNextBuffer() returns empty when it encounters the event internally.
160+ assertThat (channel .getNextBuffer ()).isNotPresent ();
161+ assertThat (channel .getStateConsumedFuture ()).isDone ();
162+ }
163+ }
164+
165+ private RecoveredInputChannel buildChannel (boolean checkpointingDuringRecoveryEnabled ) {
68166 try {
167+ SingleInputGate inputGate =
168+ new SingleInputGateBuilder ()
169+ .setCheckpointingDuringRecoveryEnabled (
170+ checkpointingDuringRecoveryEnabled )
171+ .build ();
69172 return new RecoveredInputChannel (
70- new SingleInputGateBuilder (). build () ,
173+ inputGate ,
71174 0 ,
72175 new ResultPartitionID (),
73176 new ResultSubpartitionIndexSet (0 ),
@@ -85,4 +188,41 @@ protected InputChannel toInputChannelInternal(ArrayDeque<Buffer> remainingBuffer
85188 throw new AssertionError ("channel creation failed" , e );
86189 }
87190 }
191+
192+ private TestableRecoveredInputChannel buildTestableChannel (
193+ boolean checkpointingDuringRecoveryEnabled ) {
194+ try {
195+ SingleInputGate inputGate =
196+ new SingleInputGateBuilder ()
197+ .setCheckpointingDuringRecoveryEnabled (
198+ checkpointingDuringRecoveryEnabled )
199+ .build ();
200+ return new TestableRecoveredInputChannel (inputGate );
201+ } catch (Exception e ) {
202+ throw new AssertionError ("channel creation failed" , e );
203+ }
204+ }
205+
206+ /**
207+ * A RecoveredInputChannel that returns a TestInputChannel when converted, for testing purposes.
208+ */
209+ private static class TestableRecoveredInputChannel extends RecoveredInputChannel {
210+ TestableRecoveredInputChannel (SingleInputGate inputGate ) {
211+ super (
212+ inputGate ,
213+ 0 ,
214+ new ResultPartitionID (),
215+ new ResultSubpartitionIndexSet (0 ),
216+ 0 ,
217+ 0 ,
218+ new SimpleCounter (),
219+ new SimpleCounter (),
220+ 10 );
221+ }
222+
223+ @ Override
224+ protected InputChannel toInputChannelInternal (ArrayDeque <Buffer > remainingBuffers ) {
225+ return new TestInputChannel (inputGate , 0 );
226+ }
227+ }
88228}
0 commit comments