zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints URL: https://github.com/apache/flink/pull/11515#discussion_r405634642
########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImplTest.java ########## @@ -0,0 +1,320 @@ +/* + * 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.checkpoint.channel; + +import org.apache.flink.core.memory.HeapMemorySegment; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter.ChannelStateWriteResult; +import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; +import org.apache.flink.runtime.io.network.buffer.NetworkBuffer; +import org.apache.flink.util.function.BiConsumerWithException; +import org.apache.flink.util.function.RunnableWithException; + +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.Deque; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Consumer; + +import static org.apache.flink.runtime.state.ChannelPersistenceITCase.getStreamFactoryFactory; +import static org.apache.flink.util.ExceptionUtils.findThrowable; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * {@link ChannelStateWriterImpl} lifecycle tests. + */ +public class ChannelStateWriterImplTest { + private static final long CHECKPOINT_ID = 42L; + + @Test(expected = IllegalArgumentException.class) + public void testAddEventBuffer() { + NetworkBuffer dataBuf = getBuffer(); + NetworkBuffer eventBuf = getBuffer(); + eventBuf.tagAsEvent(); + ChannelStateWriterImpl writer = openWriter(); + callStart(writer); + try { + writer.addInputData(CHECKPOINT_ID, new InputChannelInfo(1, 1), 1, eventBuf, dataBuf); + } finally { + assertTrue(dataBuf.isRecycled()); + } + } + + @Test + public void testResultCompletion() throws IOException { + ChannelStateWriteResult result; + try (ChannelStateWriterImpl writer = openWriter()) { + callStart(writer); + result = writer.getWriteResult(CHECKPOINT_ID); + assertFalse(result.resultSubpartitionStateHandles.isDone()); + assertFalse(result.inputChannelStateHandles.isDone()); + } + assertTrue(result.inputChannelStateHandles.isDone()); + assertTrue(result.resultSubpartitionStateHandles.isDone()); + } + + @Test + public void testAbort() throws Exception { + NetworkBuffer buffer = getBuffer(); + runWithSyncWorker((writer, worker) -> { + callStart(writer); + callAddInputData(writer, buffer); + callAbort(writer); + worker.processAllRequests(); + assertTrue(writer.getWriteResult(CHECKPOINT_ID).isDone()); + assertTrue(buffer.isRecycled()); + }); + } + + @Test + public void testAbortIgnoresMissing() throws Exception { + runWithSyncWorker(this::callAbort); + } + + @Test(expected = TestException.class) + public void testBuffersRecycledOnError() throws Exception { + unwrappingError(TestException.class, () -> { + NetworkBuffer buffer = getBuffer(); + try (ChannelStateWriterImpl writer = new ChannelStateWriterImpl(new ConcurrentHashMap<>(), failingWorker(), 5)) { + writer.open(); + callAddInputData(writer, buffer); + } finally { + assertTrue(buffer.isRecycled()); + } + }); + } + + @Test + public void testBuffersRecycledOnClose() throws IOException { + NetworkBuffer buffer = getBuffer(); + try (ChannelStateWriterImpl writer = openWriter()) { + callStart(writer); + callAddInputData(writer, buffer); + assertFalse(buffer.isRecycled()); Review comment: seems unstable results? After `callAddInputData`, if the enqueued buffer is already dispatched to be executed by internal thread, then the buffer should be recycled when this assert calls. Although this probability is very small, but in theory it is not stable and easily fragile. ---------------------------------------------------------------- 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: us...@infra.apache.org With regards, Apache Git Services