[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405676332
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java
 ##
 @@ -130,95 +130,108 @@ public void checkpointState(
 
// Step (1): Prepare the checkpoint, allow operators to do some 
pre-barrier work.
//   The pre-barrier work should be nothing or minimal 
in the common case.
-   operatorChain.prepareSnapshotPreBarrier(checkpointId);
+   
operatorChain.prepareSnapshotPreBarrier(metadata.getCheckpointId());
 
// Step (2): Send the checkpoint barrier downstream
-   operatorChain.broadcastCheckpointBarrier(
-   checkpointId,
-   checkpointMetaData.getTimestamp(),
-   checkpointOptions);
+   
operatorChain.broadcastCheckpointBarrier(metadata.getCheckpointId(), 
metadata.getTimestamp(), options);
 
-   // Step (3): Take the state snapshot. This should be largely 
asynchronous, to not
-   //   impact progress of the streaming topology
+   // Step (3): Take the state snapshot. This should be largely 
asynchronous, to not impact progress of the streaming topology
 
-   long startSyncPartNano = System.nanoTime();
-
-   HashMap 
operatorSnapshotsInProgress = new 
HashMap<>(operatorChain.getNumberOfOperators());
-   ChannelStateWriteResult channelStateWriteResult =
-   checkpointOptions.getCheckpointType() == CHECKPOINT ? 
channelStateWriter.getWriteResult(checkpointMetaData.getCheckpointId()) :
-   ChannelStateWriteResult.EMPTY;
+   Map snapshotFutures = new 
HashMap<>(operatorChain.getNumberOfOperators());
try {
-   for (StreamOperatorWrapper operatorWrapper : 
operatorChain.getAllOperators(true)) {
-   operatorSnapshotsInProgress.put(
-   
operatorWrapper.getStreamOperator().getOperatorID(),
-   buildOperatorSnapshotFutures(
-   checkpointMetaData,
-   checkpointOptions,
-   operatorChain,
-   
operatorWrapper.getStreamOperator(),
-   isCanceled,
-   channelStateWriteResult)
-   );
-   }
-   
checkpointStorage.clearCacheFor(checkpointMetaData.getCheckpointId());
+   takeSnapshotSync(snapshotFutures, metadata, metrics, 
options, operatorChain, isCanceled);
+   finishAndReportAsync(snapshotFutures, metadata, 
metrics);
+   } catch (Exception ex) {
+   cleanup(snapshotFutures, metadata, metrics, options, 
ex);
+   }
+   }
 
-   if (LOG.isDebugEnabled()) {
-   LOG.debug("Finished synchronous checkpoints for 
checkpoint {} on task {}",
-   checkpointMetaData.getCheckpointId(), 
taskName);
-   }
+   private void cleanup(
+   Map 
operatorSnapshotsInProgress,
+   CheckpointMetaData metadata,
+   CheckpointMetrics metrics, CheckpointOptions options,
+   Exception ex) throws Exception {
 
-   long startAsyncPartNano = System.nanoTime();
-
-   
checkpointMetrics.setSyncDurationMillis((startAsyncPartNano - 
startSyncPartNano) / 1_000_000);
-
-   // we are transferring ownership over 
snapshotInProgressList for cleanup to the thread, active on submit
-   executorService.execute(new AsyncCheckpointRunnable(
-   operatorSnapshotsInProgress,
-   checkpointMetaData,
-   checkpointMetrics,
-   startAsyncPartNano,
-   taskName,
-   closeableRegistry,
-   env,
-   asyncExceptionHandler));
-
-   if (LOG.isDebugEnabled()) {
-   LOG.debug(
-   "{} - finished synchronous part of 
checkpoint {}. Alignment duration: {} ms, snapshot duration {} ms",
-   taskName, 
checkpointMetaData.getCheckpointId()

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405675039
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequestExecutorImplTest.java
 ##
 @@ -0,0 +1,203 @@
+/*
+ * 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.util.ExceptionUtils;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import org.junit.Test;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.concurrent.LinkedBlockingDeque;
+
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequestDispatcher.NO_OP;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * {@link ChannelStateWriteRequestExecutorImpl} test.
+ */
+public class ChannelStateWriteRequestExecutorImplTest {
+
+   @Test(expected = IllegalStateException.class)
+   public void testCloseAfterSubmit() throws Exception {
+   testCloseAfterSubmit(ChannelStateWriteRequestExecutor::submit);
+   }
+
+   @Test(expected = IllegalStateException.class)
+   public void testCloseAfterSubmitPriority() throws Exception {
+   
testCloseAfterSubmit(ChannelStateWriteRequestExecutor::submitPriority);
+   }
+
+   @Test
+   public void testSubmitFailure() throws Exception {
+   testSubmitFailure(ChannelStateWriteRequestExecutor::submit);
+   }
+
+   @Test
+   public void testSubmitPriorityFailure() throws Exception {
+   
testSubmitFailure(ChannelStateWriteRequestExecutor::submitPriority);
+   }
+
+   private void 
testCloseAfterSubmit(BiConsumerWithException requestFun) throws Exception {
+   WorkerClosingDeque closingDeque = new WorkerClosingDeque();
+   ChannelStateWriteRequestExecutorImpl worker = new 
ChannelStateWriteRequestExecutorImpl(NO_OP, closingDeque);
+   closingDeque.setWorker(worker);
 
 Review comment:
   I mean it is better to add `worker.start()`, then it is easy to distinguish 
the conditions with below `testSubmitFailure`. `testSubmitFailure` is to test 
the impact without starting, and `testCloseAfterSubmit` is to test the impact 
withe explicit close after starting.


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405654821
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/state/ChannelPersistenceITCase.java
 ##
 @@ -0,0 +1,179 @@
+/*
+ * 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.state;
+
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
+import org.apache.flink.runtime.checkpoint.StateObjectCollection;
+import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
+import org.apache.flink.runtime.checkpoint.channel.ChannelStateReader;
+import 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult;
+import org.apache.flink.runtime.checkpoint.channel.ChannelStateReaderImpl;
+import 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter.ChannelStateWriteResult;
+import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriterImpl;
+import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo;
+import org.apache.flink.runtime.checkpoint.channel.ResultSubpartitionInfo;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import 
org.apache.flink.runtime.state.memory.NonPersistentMetadataCheckpointStorageLocation;
+import org.apache.flink.util.function.BiFunctionWithException;
+
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.stream.Collectors;
+
+import static java.util.Collections.singletonMap;
+import static org.apache.flink.runtime.checkpoint.CheckpointType.CHECKPOINT;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.NO_MORE_DATA;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter.SEQUENCE_NUMBER_UNKNOWN;
+import static org.apache.flink.util.Preconditions.checkState;
+import static org.junit.Assert.assertArrayEquals;
+
+/**
+ * ChannelPersistenceITCase.
+ */
+public class ChannelPersistenceITCase {
+   private static final Random RANDOM = new 
Random(System.currentTimeMillis());
+
+   @Test
+   public void testReadWritten() throws Exception {
+   long checkpointId = 1L;
+
+   InputChannelInfo inputChannelInfo = new InputChannelInfo(2, 3);
+   byte[] inputChannelInfoData = randomBytes(1024);
+
+   ResultSubpartitionInfo resultSubpartitionInfo = new 
ResultSubpartitionInfo(4, 5);
+   byte[] resultSubpartitionInfoData = randomBytes(1024);
+
+   ChannelStateWriteResult handles = write(
+   checkpointId,
+   singletonMap(inputChannelInfo, inputChannelInfoData),
+   singletonMap(resultSubpartitionInfo, 
resultSubpartitionInfoData)
+   );
+
+   assertArrayEquals(inputChannelInfoData, read(
+   toTaskStateSnapshot(handles),
+   inputChannelInfoData.length,
+   (reader, mem) -> reader.readInputData(inputChannelInfo, 
new NetworkBuffer(mem, FreeingBufferRecycler.INSTANCE))
+   ));
+
+   assertArrayEquals(resultSubpartitionInfoData, read(
+   toTaskStateSnapshot(handles),
+   resultSubpartitionInfoData.length,
+   (reader, mem) -> 
reader.readOutputData(resultSubpartitionInfo, new BufferBuilder(mem, 
FreeingBufferRecycler.INSTANCE))
+   ));
+   }
+
+   private byte[] randomBytes(int size) {
+   byte[] bytes = new byte[size];
+   RANDOM.nextBytes(bytes);
+   return bytes;
+   }
+
+   pri

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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 IOExcept

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405620878
 
 

 ##
 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 IOExcept

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405619864
 
 

 ##
 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());
 
 Review comment:
   I guess it is not determined results? When the writer#close, then it relies 
on `ChannelStateWriteRequestExecutorImpl#cleanupRequests` to cancel the start 
request to complete the result. But if the start request was already taken away 
from the queue by internal thread before, then we can not take it from queue to 
cancel. Or I missed something else?


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405604653
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequestExecutorImplTest.java
 ##
 @@ -0,0 +1,203 @@
+/*
+ * 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.util.ExceptionUtils;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import org.junit.Test;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.concurrent.LinkedBlockingDeque;
+
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequestDispatcher.NO_OP;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * {@link ChannelStateWriteRequestExecutorImpl} test.
+ */
+public class ChannelStateWriteRequestExecutorImplTest {
+
+   @Test(expected = IllegalStateException.class)
+   public void testCloseAfterSubmit() throws Exception {
+   testCloseAfterSubmit(ChannelStateWriteRequestExecutor::submit);
+   }
+
+   @Test(expected = IllegalStateException.class)
+   public void testCloseAfterSubmitPriority() throws Exception {
+   
testCloseAfterSubmit(ChannelStateWriteRequestExecutor::submitPriority);
+   }
+
+   @Test
+   public void testSubmitFailure() throws Exception {
+   testSubmitFailure(ChannelStateWriteRequestExecutor::submit);
+   }
+
+   @Test
+   public void testSubmitPriorityFailure() throws Exception {
+   
testSubmitFailure(ChannelStateWriteRequestExecutor::submitPriority);
+   }
+
+   private void 
testCloseAfterSubmit(BiConsumerWithException requestFun) throws Exception {
+   WorkerClosingDeque closingDeque = new WorkerClosingDeque();
+   ChannelStateWriteRequestExecutorImpl worker = new 
ChannelStateWriteRequestExecutorImpl(NO_OP, closingDeque);
+   closingDeque.setWorker(worker);
+   TestWriteRequest request = new TestWriteRequest();
+   requestFun.accept(worker, request);
+   assertTrue(closingDeque.isEmpty());
+   assertFalse(request.isCancelled());
+   }
+
+   private void 
testSubmitFailure(BiConsumerWithException submitAction) throws Exception {
+   TestWriteRequest request = new TestWriteRequest();
+   LinkedBlockingDeque deque = new 
LinkedBlockingDeque<>();
+   try {
+   submitAction.accept(new 
ChannelStateWriteRequestExecutorImpl(NO_OP, deque), request);
+   } catch (IllegalStateException e) {
+   // expected: executor not started;
+   return;
+   } finally {
+   assertTrue(request.cancelled);
+   assertTrue(deque.isEmpty());
+   }
+   throw new RuntimeException("expected exception not thrown");
+   }
+
+   @Test
+   @SuppressWarnings("CallToThreadRun")
+   public void testCleanup() throws IOException {
+   TestWriteRequest request = new TestWriteRequest();
+   LinkedBlockingDeque deque = new 
LinkedBlockingDeque<>();
+   deque.add(request);
+   TestRequestDispatcher requestProcessor = new 
TestRequestDispatcher();
+   ChannelStateWriteRequestExecutorImpl worker = new 
ChannelStateWriteRequestExecutorImpl(requestProcessor, deque);
+
+   worker.close();
+   worker.run();
+
+   assertTrue(requestProcessor.isStopped());
+   assertTrue(deque.isEmpty());
+   assertTrue(request.isCancelled());
+   }
+
+   @Test
+   public void testIgnoresInterruptsWhileRunning() throws Exception {
+   TestRequestDispatcher requestProcessor = new 
TestRequestDispatcher();
+   LinkedBlockingDeque deque = new 
LinkedBlockingDeque<>();
+   try (ChannelStateW

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405517754
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequestExecutorImplTest.java
 ##
 @@ -0,0 +1,203 @@
+/*
+ * 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.util.ExceptionUtils;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import org.junit.Test;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.concurrent.LinkedBlockingDeque;
+
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequestDispatcher.NO_OP;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * {@link ChannelStateWriteRequestExecutorImpl} test.
+ */
+public class ChannelStateWriteRequestExecutorImplTest {
+
+   @Test(expected = IllegalStateException.class)
+   public void testCloseAfterSubmit() throws Exception {
+   testCloseAfterSubmit(ChannelStateWriteRequestExecutor::submit);
+   }
+
+   @Test(expected = IllegalStateException.class)
+   public void testCloseAfterSubmitPriority() throws Exception {
+   
testCloseAfterSubmit(ChannelStateWriteRequestExecutor::submitPriority);
+   }
+
+   @Test
+   public void testSubmitFailure() throws Exception {
+   testSubmitFailure(ChannelStateWriteRequestExecutor::submit);
+   }
+
+   @Test
+   public void testSubmitPriorityFailure() throws Exception {
+   
testSubmitFailure(ChannelStateWriteRequestExecutor::submitPriority);
+   }
+
+   private void 
testCloseAfterSubmit(BiConsumerWithException requestFun) throws Exception {
+   WorkerClosingDeque closingDeque = new WorkerClosingDeque();
+   ChannelStateWriteRequestExecutorImpl worker = new 
ChannelStateWriteRequestExecutorImpl(NO_OP, closingDeque);
+   closingDeque.setWorker(worker);
 
 Review comment:
   The worker should start after created, otherwise even though we do not call 
close via `WorkerClosingDeque`, it can still encounter exception as did in 
`testSubmitFailure`.


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405492070
 
 

 ##
 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());
 
 Review comment:
   eventBuf.recycleBuffer()


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405428301
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateSerializerTest.java
 ##
 @@ -0,0 +1,89 @@
+/*
+ * 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.MemorySegmentFactory;
+import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+
+import org.junit.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Random;
+
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateByteBuffer.wrap;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * ChannelStateSerializerTest.
+ */
+public class ChannelStateSerializerTest {
 
 Review comment:
   This class can be merged with `ChannelStateSerializerImplTest`, because they 
are all aiming for testing the `ChannelStateSerializerImpl` actually, for 
wrapping `BufferBuilder`, `Buffer`, and `bytes[]` separately in different tests.


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405425100
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReaderImplTest.java
 ##
 @@ -0,0 +1,205 @@
+package org.apache.flink.runtime.checkpoint.channel;
+/*
+ * 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.
+ */
+
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
+import org.apache.flink.runtime.checkpoint.StateObjectCollection;
+import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
+import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.InputChannelStateHandle;
+import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+import static java.util.Collections.singletonList;
+import static java.util.stream.Collectors.toMap;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.HAS_MORE_DATA;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.NO_MORE_DATA;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * {@link ChannelStateReaderImpl} test.
+ */
+public class ChannelStateReaderImplTest {
+
+   private static final InputChannelInfo CHANNEL = new InputChannelInfo(1, 
2);
+   private static final byte[] DATA = generateData(10);
+   private ChannelStateReaderImpl reader;
+
+   @Before
+   public void init() {
+   reader = getReader(CHANNEL, DATA);
+   }
+
+   @After
+   public void tearDown() throws Exception {
+   reader.close();
+   }
+
+   @Test
+   public void testDifferentBufferSizes() throws Exception {
+   for (int bufferSize = 1; bufferSize < 2 * DATA.length; 
bufferSize++) {
+   try (ChannelStateReaderImpl reader = getReader(CHANNEL, 
DATA)) { // re-create reader to re-read the same channel
+   readAndVerify(bufferSize, CHANNEL, DATA, 
reader);
+   }
+   }
+   }
+
+   @Test
+   public void testWithOffsets() throws IOException {
+   Map handlesAndBytes = 
generateHandlesWithBytes(10, 20);
+   ChannelStateReader reader = new 
ChannelStateReaderImpl(taskStateSnapshot(handlesAndBytes.keySet()), new 
ChannelStateSerializerImpl());
+   for (Map.Entry e : 
handlesAndBytes.entrySet()) {
+   readAndVerify(42, e.getKey().getInfo(), e.getValue(), 
reader);
+   }
+   }
+
+   @Test(expected = Exception.class)
+   public void testReadOnlyOnce() throws IOException {
+   reader.readInputData(CHANNEL, getBuffer(DATA.length));
+   reader.readInputData(CHANNEL, getBuffer(DATA.length));
+   }
+
+   @Test(expected = Exception.class)
+   public void testReadClosed() throws Exception {
+   reader.close();
+   reader.readInputData(CHANNEL, getBuffer(DATA.length));
+   }
+
+   @Test(expected = IllegalArgumentException.class)
+   public void testReadWrongChannelState() throws IOException {
+   InputChannelInfo wrongChannel = new 
InputChannelInfo(CHANNEL.getGateIdx() + 1, CHANNEL.getInputChannelIdx() + 1);
+   reader.readInputData(wrongChannel, getBuffer(DATA.length));
+   }
+
+   private TaskStateSnapshot 
taskStateSnapshot(Collection inputChannelStateHandles) 
{
+   return new TaskStateSnapshot(

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405425100
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReaderImplTest.java
 ##
 @@ -0,0 +1,205 @@
+package org.apache.flink.runtime.checkpoint.channel;
+/*
+ * 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.
+ */
+
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
+import org.apache.flink.runtime.checkpoint.StateObjectCollection;
+import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
+import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.InputChannelStateHandle;
+import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+import static java.util.Collections.singletonList;
+import static java.util.stream.Collectors.toMap;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.HAS_MORE_DATA;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.NO_MORE_DATA;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * {@link ChannelStateReaderImpl} test.
+ */
+public class ChannelStateReaderImplTest {
+
+   private static final InputChannelInfo CHANNEL = new InputChannelInfo(1, 
2);
+   private static final byte[] DATA = generateData(10);
+   private ChannelStateReaderImpl reader;
+
+   @Before
+   public void init() {
+   reader = getReader(CHANNEL, DATA);
+   }
+
+   @After
+   public void tearDown() throws Exception {
+   reader.close();
+   }
+
+   @Test
+   public void testDifferentBufferSizes() throws Exception {
+   for (int bufferSize = 1; bufferSize < 2 * DATA.length; 
bufferSize++) {
+   try (ChannelStateReaderImpl reader = getReader(CHANNEL, 
DATA)) { // re-create reader to re-read the same channel
+   readAndVerify(bufferSize, CHANNEL, DATA, 
reader);
+   }
+   }
+   }
+
+   @Test
+   public void testWithOffsets() throws IOException {
+   Map handlesAndBytes = 
generateHandlesWithBytes(10, 20);
+   ChannelStateReader reader = new 
ChannelStateReaderImpl(taskStateSnapshot(handlesAndBytes.keySet()), new 
ChannelStateSerializerImpl());
+   for (Map.Entry e : 
handlesAndBytes.entrySet()) {
+   readAndVerify(42, e.getKey().getInfo(), e.getValue(), 
reader);
+   }
+   }
+
+   @Test(expected = Exception.class)
+   public void testReadOnlyOnce() throws IOException {
+   reader.readInputData(CHANNEL, getBuffer(DATA.length));
+   reader.readInputData(CHANNEL, getBuffer(DATA.length));
+   }
+
+   @Test(expected = Exception.class)
+   public void testReadClosed() throws Exception {
+   reader.close();
+   reader.readInputData(CHANNEL, getBuffer(DATA.length));
+   }
+
+   @Test(expected = IllegalArgumentException.class)
+   public void testReadWrongChannelState() throws IOException {
+   InputChannelInfo wrongChannel = new 
InputChannelInfo(CHANNEL.getGateIdx() + 1, CHANNEL.getInputChannelIdx() + 1);
+   reader.readInputData(wrongChannel, getBuffer(DATA.length));
+   }
+
+   private TaskStateSnapshot 
taskStateSnapshot(Collection inputChannelStateHandles) 
{
+   return new TaskStateSnapshot(

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405420576
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateSerializerImplTest.java
 ##
 @@ -0,0 +1,108 @@
+/*
+ * 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.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+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.junit.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Random;
+
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateByteBuffer.wrap;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * {@link ChannelStateSerializerImpl} test.
+ */
+public class ChannelStateSerializerImplTest {
+
+   private final Random random = new Random();
+
+   @Test
+   public void testWriteRead() throws IOException {
+   int bufSize = 10;
+   int[] numBuffersToWriteAtOnce = {0, 1, 2, 3};
+   byte[] data = getData(bufSize);
+   ChannelStateSerializer s = new ChannelStateSerializerImpl();
+   ByteArrayOutputStream baos = new ByteArrayOutputStream();
+   DataOutputStream out = new DataOutputStream(baos);
 
 Review comment:
   close the stream at the end?


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405408437
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReaderImplTest.java
 ##
 @@ -0,0 +1,205 @@
+package org.apache.flink.runtime.checkpoint.channel;
+/*
+ * 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.
+ */
+
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
+import org.apache.flink.runtime.checkpoint.StateObjectCollection;
+import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
+import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.InputChannelStateHandle;
+import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+import static java.util.Collections.singletonList;
+import static java.util.stream.Collectors.toMap;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.HAS_MORE_DATA;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.NO_MORE_DATA;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * {@link ChannelStateReaderImpl} test.
+ */
+public class ChannelStateReaderImplTest {
+
+   private static final InputChannelInfo CHANNEL = new InputChannelInfo(1, 
2);
+   private static final byte[] DATA = generateData(10);
+   private ChannelStateReaderImpl reader;
+
+   @Before
+   public void init() {
+   reader = getReader(CHANNEL, DATA);
+   }
+
+   @After
+   public void tearDown() throws Exception {
+   reader.close();
+   }
+
+   @Test
+   public void testDifferentBufferSizes() throws Exception {
+   for (int bufferSize = 1; bufferSize < 2 * DATA.length; 
bufferSize++) {
+   try (ChannelStateReaderImpl reader = getReader(CHANNEL, 
DATA)) { // re-create reader to re-read the same channel
+   readAndVerify(bufferSize, CHANNEL, DATA, 
reader);
+   }
+   }
+   }
+
+   @Test
+   public void testWithOffsets() throws IOException {
+   Map handlesAndBytes = 
generateHandlesWithBytes(10, 20);
+   ChannelStateReader reader = new 
ChannelStateReaderImpl(taskStateSnapshot(handlesAndBytes.keySet()), new 
ChannelStateSerializerImpl());
+   for (Map.Entry e : 
handlesAndBytes.entrySet()) {
+   readAndVerify(42, e.getKey().getInfo(), e.getValue(), 
reader);
+   }
+   }
+
+   @Test(expected = Exception.class)
+   public void testReadOnlyOnce() throws IOException {
 
 Review comment:
   I have not seen any tests covering the resource release for the reader. If 
possible, it is better to further verify the internal 
`RefCountingFSDataInputStream` is dereferenced and closed when no more data.  
Then we can confirm no resource leak. 


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405408437
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReaderImplTest.java
 ##
 @@ -0,0 +1,205 @@
+package org.apache.flink.runtime.checkpoint.channel;
+/*
+ * 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.
+ */
+
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
+import org.apache.flink.runtime.checkpoint.StateObjectCollection;
+import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
+import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.InputChannelStateHandle;
+import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+import static java.util.Collections.singletonList;
+import static java.util.stream.Collectors.toMap;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.HAS_MORE_DATA;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.NO_MORE_DATA;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * {@link ChannelStateReaderImpl} test.
+ */
+public class ChannelStateReaderImplTest {
+
+   private static final InputChannelInfo CHANNEL = new InputChannelInfo(1, 
2);
+   private static final byte[] DATA = generateData(10);
+   private ChannelStateReaderImpl reader;
+
+   @Before
+   public void init() {
+   reader = getReader(CHANNEL, DATA);
+   }
+
+   @After
+   public void tearDown() throws Exception {
+   reader.close();
+   }
+
+   @Test
+   public void testDifferentBufferSizes() throws Exception {
+   for (int bufferSize = 1; bufferSize < 2 * DATA.length; 
bufferSize++) {
+   try (ChannelStateReaderImpl reader = getReader(CHANNEL, 
DATA)) { // re-create reader to re-read the same channel
+   readAndVerify(bufferSize, CHANNEL, DATA, 
reader);
+   }
+   }
+   }
+
+   @Test
+   public void testWithOffsets() throws IOException {
+   Map handlesAndBytes = 
generateHandlesWithBytes(10, 20);
+   ChannelStateReader reader = new 
ChannelStateReaderImpl(taskStateSnapshot(handlesAndBytes.keySet()), new 
ChannelStateSerializerImpl());
+   for (Map.Entry e : 
handlesAndBytes.entrySet()) {
+   readAndVerify(42, e.getKey().getInfo(), e.getValue(), 
reader);
+   }
+   }
+
+   @Test(expected = Exception.class)
+   public void testReadOnlyOnce() throws IOException {
 
 Review comment:
   If possible, it is better to further verify the internal 
`RefCountingFSDataInputStream` is dereferenced and closed when no more data.  
Then we can confirm no resource leak. 


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405328387
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReaderImplTest.java
 ##
 @@ -0,0 +1,205 @@
+package org.apache.flink.runtime.checkpoint.channel;
+/*
+ * 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.
+ */
+
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
+import org.apache.flink.runtime.checkpoint.StateObjectCollection;
+import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
+import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
+import org.apache.flink.runtime.io.network.buffer.NetworkBuffer;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.InputChannelStateHandle;
+import org.apache.flink.runtime.state.memory.ByteStreamStateHandle;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+import static java.util.Collections.singletonList;
+import static java.util.stream.Collectors.toMap;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.HAS_MORE_DATA;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.NO_MORE_DATA;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * {@link ChannelStateReaderImpl} test.
+ */
+public class ChannelStateReaderImplTest {
 
 Review comment:
   All the below tests are for the  `ChannelStateReader#readInput`, not sure 
whether we also need to cover the code path for `ChannelStateReader#readOutput`.


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405317909
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateCheckpointWriterTest.java
 ##
 @@ -0,0 +1,133 @@
+/*
+ * 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.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.runtime.state.InputChannelStateHandle;
+import 
org.apache.flink.runtime.state.memory.MemCheckpointStreamFactory.MemoryCheckpointOutputStream;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * {@link ChannelStateCheckpointWriter} test.
+ */
+public class ChannelStateCheckpointWriterTest {
+   private static final RunnableWithException NO_OP_RUNNABLE = () -> {
+   };
+   private final Random random = new Random();
+
+   @Test
+   public void testRecyclingBuffers() throws Exception {
+   ChannelStateCheckpointWriter writer = createWriter(new 
ChannelStateWriteResult());
+   NetworkBuffer buffer = new 
NetworkBuffer(HeapMemorySegment.FACTORY.allocateUnpooledSegment(10, null), 
FreeingBufferRecycler.INSTANCE);
+   writer.writeInput(new InputChannelInfo(1, 2), buffer);
+   assertTrue(buffer.isRecycled());
+   }
+
+   @Test
+   public void testFlush() throws Exception {
+   class FlushRecorder extends DataOutputStream {
+   private boolean flushed = false;
+
+   private FlushRecorder() {
+   super(new ByteArrayOutputStream());
+   }
+
+   @Override
+   public void flush() throws IOException {
+   flushed = true;
+   super.flush();
+   }
+   }
+
+   FlushRecorder dataStream = new FlushRecorder();
+   final ChannelStateCheckpointWriter writer = new 
ChannelStateCheckpointWriter(
+   1L,
+   new ChannelStateWriteResult(),
+   new ChannelStateSerializerImpl(),
+   NO_OP_RUNNABLE,
+   new MemoryCheckpointOutputStream(42),
+   dataStream
+   );
+
+   writer.completeInput();
+   writer.completeOutput();
+
+   assertTrue(dataStream.flushed);
 
 Review comment:
   it is better to also verify the `ChannelStateWriteResult#isDone` when 
complete both input and output.


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405306450
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java
 ##
 @@ -130,95 +130,108 @@ public void checkpointState(
 
// Step (1): Prepare the checkpoint, allow operators to do some 
pre-barrier work.
//   The pre-barrier work should be nothing or minimal 
in the common case.
-   operatorChain.prepareSnapshotPreBarrier(checkpointId);
+   
operatorChain.prepareSnapshotPreBarrier(metadata.getCheckpointId());
 
// Step (2): Send the checkpoint barrier downstream
-   operatorChain.broadcastCheckpointBarrier(
-   checkpointId,
-   checkpointMetaData.getTimestamp(),
-   checkpointOptions);
+   
operatorChain.broadcastCheckpointBarrier(metadata.getCheckpointId(), 
metadata.getTimestamp(), options);
 
-   // Step (3): Take the state snapshot. This should be largely 
asynchronous, to not
-   //   impact progress of the streaming topology
+   // Step (3): Take the state snapshot. This should be largely 
asynchronous, to not impact progress of the streaming topology
 
-   long startSyncPartNano = System.nanoTime();
-
-   HashMap 
operatorSnapshotsInProgress = new 
HashMap<>(operatorChain.getNumberOfOperators());
-   ChannelStateWriteResult channelStateWriteResult =
-   checkpointOptions.getCheckpointType() == CHECKPOINT ? 
channelStateWriter.getWriteResult(checkpointMetaData.getCheckpointId()) :
-   ChannelStateWriteResult.EMPTY;
+   Map snapshotFutures = new 
HashMap<>(operatorChain.getNumberOfOperators());
try {
-   for (StreamOperatorWrapper operatorWrapper : 
operatorChain.getAllOperators(true)) {
-   operatorSnapshotsInProgress.put(
-   
operatorWrapper.getStreamOperator().getOperatorID(),
-   buildOperatorSnapshotFutures(
-   checkpointMetaData,
-   checkpointOptions,
-   operatorChain,
-   
operatorWrapper.getStreamOperator(),
-   isCanceled,
-   channelStateWriteResult)
-   );
-   }
-   
checkpointStorage.clearCacheFor(checkpointMetaData.getCheckpointId());
+   takeSnapshotSync(snapshotFutures, metadata, metrics, 
options, operatorChain, isCanceled);
+   finishAndReportAsync(snapshotFutures, metadata, 
metrics);
+   } catch (Exception ex) {
+   cleanup(snapshotFutures, metadata, metrics, options, 
ex);
+   }
+   }
 
-   if (LOG.isDebugEnabled()) {
-   LOG.debug("Finished synchronous checkpoints for 
checkpoint {} on task {}",
-   checkpointMetaData.getCheckpointId(), 
taskName);
-   }
+   private void cleanup(
+   Map 
operatorSnapshotsInProgress,
+   CheckpointMetaData metadata,
+   CheckpointMetrics metrics, CheckpointOptions options,
+   Exception ex) throws Exception {
 
-   long startAsyncPartNano = System.nanoTime();
-
-   
checkpointMetrics.setSyncDurationMillis((startAsyncPartNano - 
startSyncPartNano) / 1_000_000);
-
-   // we are transferring ownership over 
snapshotInProgressList for cleanup to the thread, active on submit
-   executorService.execute(new AsyncCheckpointRunnable(
-   operatorSnapshotsInProgress,
-   checkpointMetaData,
-   checkpointMetrics,
-   startAsyncPartNano,
-   taskName,
-   closeableRegistry,
-   env,
-   asyncExceptionHandler));
-
-   if (LOG.isDebugEnabled()) {
-   LOG.debug(
-   "{} - finished synchronous part of 
checkpoint {}. Alignment duration: {} ms, snapshot duration {} ms",
-   taskName, 
checkpointMetaData.getCheckpointId()

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405305452
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java
 ##
 @@ -130,95 +130,108 @@ public void checkpointState(
 
// Step (1): Prepare the checkpoint, allow operators to do some 
pre-barrier work.
//   The pre-barrier work should be nothing or minimal 
in the common case.
-   operatorChain.prepareSnapshotPreBarrier(checkpointId);
+   
operatorChain.prepareSnapshotPreBarrier(metadata.getCheckpointId());
 
// Step (2): Send the checkpoint barrier downstream
-   operatorChain.broadcastCheckpointBarrier(
-   checkpointId,
-   checkpointMetaData.getTimestamp(),
-   checkpointOptions);
+   
operatorChain.broadcastCheckpointBarrier(metadata.getCheckpointId(), 
metadata.getTimestamp(), options);
 
-   // Step (3): Take the state snapshot. This should be largely 
asynchronous, to not
-   //   impact progress of the streaming topology
+   // Step (3): Take the state snapshot. This should be largely 
asynchronous, to not impact progress of the streaming topology
 
-   long startSyncPartNano = System.nanoTime();
-
-   HashMap 
operatorSnapshotsInProgress = new 
HashMap<>(operatorChain.getNumberOfOperators());
-   ChannelStateWriteResult channelStateWriteResult =
-   checkpointOptions.getCheckpointType() == CHECKPOINT ? 
channelStateWriter.getWriteResult(checkpointMetaData.getCheckpointId()) :
-   ChannelStateWriteResult.EMPTY;
+   Map snapshotFutures = new 
HashMap<>(operatorChain.getNumberOfOperators());
try {
-   for (StreamOperatorWrapper operatorWrapper : 
operatorChain.getAllOperators(true)) {
-   operatorSnapshotsInProgress.put(
-   
operatorWrapper.getStreamOperator().getOperatorID(),
-   buildOperatorSnapshotFutures(
-   checkpointMetaData,
-   checkpointOptions,
-   operatorChain,
-   
operatorWrapper.getStreamOperator(),
-   isCanceled,
-   channelStateWriteResult)
-   );
-   }
-   
checkpointStorage.clearCacheFor(checkpointMetaData.getCheckpointId());
+   takeSnapshotSync(snapshotFutures, metadata, metrics, 
options, operatorChain, isCanceled);
+   finishAndReportAsync(snapshotFutures, metadata, 
metrics);
+   } catch (Exception ex) {
+   cleanup(snapshotFutures, metadata, metrics, options, 
ex);
+   }
+   }
 
-   if (LOG.isDebugEnabled()) {
-   LOG.debug("Finished synchronous checkpoints for 
checkpoint {} on task {}",
-   checkpointMetaData.getCheckpointId(), 
taskName);
-   }
+   private void cleanup(
+   Map 
operatorSnapshotsInProgress,
+   CheckpointMetaData metadata,
+   CheckpointMetrics metrics, CheckpointOptions options,
+   Exception ex) throws Exception {
 
-   long startAsyncPartNano = System.nanoTime();
-
-   
checkpointMetrics.setSyncDurationMillis((startAsyncPartNano - 
startSyncPartNano) / 1_000_000);
-
-   // we are transferring ownership over 
snapshotInProgressList for cleanup to the thread, active on submit
-   executorService.execute(new AsyncCheckpointRunnable(
-   operatorSnapshotsInProgress,
-   checkpointMetaData,
-   checkpointMetrics,
-   startAsyncPartNano,
-   taskName,
-   closeableRegistry,
-   env,
-   asyncExceptionHandler));
-
-   if (LOG.isDebugEnabled()) {
-   LOG.debug(
-   "{} - finished synchronous part of 
checkpoint {}. Alignment duration: {} ms, snapshot duration {} ms",
-   taskName, 
checkpointMetaData.getCheckpointId()

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405304470
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java
 ##
 @@ -130,95 +130,108 @@ public void checkpointState(
 
// Step (1): Prepare the checkpoint, allow operators to do some 
pre-barrier work.
//   The pre-barrier work should be nothing or minimal 
in the common case.
-   operatorChain.prepareSnapshotPreBarrier(checkpointId);
+   
operatorChain.prepareSnapshotPreBarrier(metadata.getCheckpointId());
 
// Step (2): Send the checkpoint barrier downstream
-   operatorChain.broadcastCheckpointBarrier(
-   checkpointId,
-   checkpointMetaData.getTimestamp(),
-   checkpointOptions);
+   
operatorChain.broadcastCheckpointBarrier(metadata.getCheckpointId(), 
metadata.getTimestamp(), options);
 
-   // Step (3): Take the state snapshot. This should be largely 
asynchronous, to not
-   //   impact progress of the streaming topology
+   // Step (3): Take the state snapshot. This should be largely 
asynchronous, to not impact progress of the streaming topology
 
-   long startSyncPartNano = System.nanoTime();
-
-   HashMap 
operatorSnapshotsInProgress = new 
HashMap<>(operatorChain.getNumberOfOperators());
-   ChannelStateWriteResult channelStateWriteResult =
-   checkpointOptions.getCheckpointType() == CHECKPOINT ? 
channelStateWriter.getWriteResult(checkpointMetaData.getCheckpointId()) :
-   ChannelStateWriteResult.EMPTY;
+   Map snapshotFutures = new 
HashMap<>(operatorChain.getNumberOfOperators());
try {
-   for (StreamOperatorWrapper operatorWrapper : 
operatorChain.getAllOperators(true)) {
-   operatorSnapshotsInProgress.put(
-   
operatorWrapper.getStreamOperator().getOperatorID(),
-   buildOperatorSnapshotFutures(
-   checkpointMetaData,
-   checkpointOptions,
-   operatorChain,
-   
operatorWrapper.getStreamOperator(),
-   isCanceled,
-   channelStateWriteResult)
-   );
-   }
-   
checkpointStorage.clearCacheFor(checkpointMetaData.getCheckpointId());
+   takeSnapshotSync(snapshotFutures, metadata, metrics, 
options, operatorChain, isCanceled);
+   finishAndReportAsync(snapshotFutures, metadata, 
metrics);
+   } catch (Exception ex) {
+   cleanup(snapshotFutures, metadata, metrics, options, 
ex);
+   }
+   }
 
-   if (LOG.isDebugEnabled()) {
-   LOG.debug("Finished synchronous checkpoints for 
checkpoint {} on task {}",
-   checkpointMetaData.getCheckpointId(), 
taskName);
-   }
+   private void cleanup(
+   Map 
operatorSnapshotsInProgress,
+   CheckpointMetaData metadata,
+   CheckpointMetrics metrics, CheckpointOptions options,
+   Exception ex) throws Exception {
 
-   long startAsyncPartNano = System.nanoTime();
-
-   
checkpointMetrics.setSyncDurationMillis((startAsyncPartNano - 
startSyncPartNano) / 1_000_000);
-
-   // we are transferring ownership over 
snapshotInProgressList for cleanup to the thread, active on submit
-   executorService.execute(new AsyncCheckpointRunnable(
-   operatorSnapshotsInProgress,
-   checkpointMetaData,
-   checkpointMetrics,
-   startAsyncPartNano,
-   taskName,
-   closeableRegistry,
-   env,
-   asyncExceptionHandler));
-
-   if (LOG.isDebugEnabled()) {
-   LOG.debug(
-   "{} - finished synchronous part of 
checkpoint {}. Alignment duration: {} ms, snapshot duration {} ms",
-   taskName, 
checkpointMetaData.getCheckpointId()

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-08 Thread GitBox
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_r405303703
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java
 ##
 @@ -130,95 +130,108 @@ public void checkpointState(
 
// Step (1): Prepare the checkpoint, allow operators to do some 
pre-barrier work.
//   The pre-barrier work should be nothing or minimal 
in the common case.
-   operatorChain.prepareSnapshotPreBarrier(checkpointId);
+   
operatorChain.prepareSnapshotPreBarrier(metadata.getCheckpointId());
 
// Step (2): Send the checkpoint barrier downstream
-   operatorChain.broadcastCheckpointBarrier(
-   checkpointId,
-   checkpointMetaData.getTimestamp(),
-   checkpointOptions);
+   
operatorChain.broadcastCheckpointBarrier(metadata.getCheckpointId(), 
metadata.getTimestamp(), options);
 
-   // Step (3): Take the state snapshot. This should be largely 
asynchronous, to not
-   //   impact progress of the streaming topology
+   // Step (3): Take the state snapshot. This should be largely 
asynchronous, to not impact progress of the streaming topology
 
-   long startSyncPartNano = System.nanoTime();
-
-   HashMap 
operatorSnapshotsInProgress = new 
HashMap<>(operatorChain.getNumberOfOperators());
-   ChannelStateWriteResult channelStateWriteResult =
-   checkpointOptions.getCheckpointType() == CHECKPOINT ? 
channelStateWriter.getWriteResult(checkpointMetaData.getCheckpointId()) :
-   ChannelStateWriteResult.EMPTY;
+   Map snapshotFutures = new 
HashMap<>(operatorChain.getNumberOfOperators());
try {
-   for (StreamOperatorWrapper operatorWrapper : 
operatorChain.getAllOperators(true)) {
-   operatorSnapshotsInProgress.put(
-   
operatorWrapper.getStreamOperator().getOperatorID(),
-   buildOperatorSnapshotFutures(
-   checkpointMetaData,
-   checkpointOptions,
-   operatorChain,
-   
operatorWrapper.getStreamOperator(),
-   isCanceled,
-   channelStateWriteResult)
-   );
-   }
-   
checkpointStorage.clearCacheFor(checkpointMetaData.getCheckpointId());
+   takeSnapshotSync(snapshotFutures, metadata, metrics, 
options, operatorChain, isCanceled);
+   finishAndReportAsync(snapshotFutures, metadata, 
metrics);
+   } catch (Exception ex) {
+   cleanup(snapshotFutures, metadata, metrics, options, 
ex);
+   }
+   }
 
-   if (LOG.isDebugEnabled()) {
-   LOG.debug("Finished synchronous checkpoints for 
checkpoint {} on task {}",
-   checkpointMetaData.getCheckpointId(), 
taskName);
-   }
+   private void cleanup(
+   Map 
operatorSnapshotsInProgress,
+   CheckpointMetaData metadata,
+   CheckpointMetrics metrics, CheckpointOptions options,
+   Exception ex) throws Exception {
 
-   long startAsyncPartNano = System.nanoTime();
-
-   
checkpointMetrics.setSyncDurationMillis((startAsyncPartNano - 
startSyncPartNano) / 1_000_000);
-
-   // we are transferring ownership over 
snapshotInProgressList for cleanup to the thread, active on submit
-   executorService.execute(new AsyncCheckpointRunnable(
-   operatorSnapshotsInProgress,
-   checkpointMetaData,
-   checkpointMetrics,
-   startAsyncPartNano,
-   taskName,
-   closeableRegistry,
-   env,
-   asyncExceptionHandler));
-
-   if (LOG.isDebugEnabled()) {
-   LOG.debug(
-   "{} - finished synchronous part of 
checkpoint {}. Alignment duration: {} ms, snapshot duration {} ms",
-   taskName, 
checkpointMetaData.getCheckpointId()

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r405298804
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java
 ##
 @@ -188,6 +221,77 @@ public void checkpointState(
}
}
 
+   private OperatorSnapshotFutures buildOperatorSnapshotFutures(
+   CheckpointMetaData checkpointMetaData,
+   CheckpointOptions checkpointOptions,
+   OperatorChain operatorChain,
+   StreamOperator op,
+   Supplier isCanceled,
+   ChannelStateWriteResult channelStateWriteResult) throws 
Exception {
+   CheckpointStreamFactory storage = 
checkpointStorage.resolveCheckpointStorageLocation(
+   checkpointMetaData.getCheckpointId(),
+   checkpointOptions.getTargetLocation());
+   OperatorSnapshotFutures snapshotInProgress = 
checkpointStreamOperator(
+   op,
+   checkpointMetaData,
+   checkpointOptions,
+   storage,
+   isCanceled);
+   if (op == operatorChain.getHeadOperator()) {
+   snapshotInProgress.setInputChannelStateFuture(
+   channelStateWriteResult
+   .getInputChannelStateHandles()
+   .thenApply(StateObjectCollection::new)
+   .thenApply(SnapshotResult::of));
+   }
+   if (op == operatorChain.getTailOperator()) {
+   snapshotInProgress.setResultSubpartitionStateFuture(
+   channelStateWriteResult
+   .getResultSubpartitionStateHandles()
+   .thenApply(StateObjectCollection::new)
+   .thenApply(SnapshotResult::of));
+   }
+   return snapshotInProgress;
+   }
+
+   @Override
+   public void close() throws IOException {
+   channelStateWriter.close();
+   }
+
+   // Caches checkpoint output stream factories to prevent multiple output 
stream per checkpoint.
+   // This could result from requesting output stream by different 
entities (this and channelStateWriter)
+   // We can't just pass a stream to the channelStateWriter because it can 
receive checkpoint call earlier than this class
+   // in some unaligned checkpoints scenarios
+   private static class CachingCheckpointStorageWorkerView implements 
CheckpointStorageWorkerView {
 
 Review comment:
   Do we have the tests coverage for this new introduced caching function in 
the commit "[FLINK-16744][task] send channel state handles to JM"?


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r405298804
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java
 ##
 @@ -188,6 +221,77 @@ public void checkpointState(
}
}
 
+   private OperatorSnapshotFutures buildOperatorSnapshotFutures(
+   CheckpointMetaData checkpointMetaData,
+   CheckpointOptions checkpointOptions,
+   OperatorChain operatorChain,
+   StreamOperator op,
+   Supplier isCanceled,
+   ChannelStateWriteResult channelStateWriteResult) throws 
Exception {
+   CheckpointStreamFactory storage = 
checkpointStorage.resolveCheckpointStorageLocation(
+   checkpointMetaData.getCheckpointId(),
+   checkpointOptions.getTargetLocation());
+   OperatorSnapshotFutures snapshotInProgress = 
checkpointStreamOperator(
+   op,
+   checkpointMetaData,
+   checkpointOptions,
+   storage,
+   isCanceled);
+   if (op == operatorChain.getHeadOperator()) {
+   snapshotInProgress.setInputChannelStateFuture(
+   channelStateWriteResult
+   .getInputChannelStateHandles()
+   .thenApply(StateObjectCollection::new)
+   .thenApply(SnapshotResult::of));
+   }
+   if (op == operatorChain.getTailOperator()) {
+   snapshotInProgress.setResultSubpartitionStateFuture(
+   channelStateWriteResult
+   .getResultSubpartitionStateHandles()
+   .thenApply(StateObjectCollection::new)
+   .thenApply(SnapshotResult::of));
+   }
+   return snapshotInProgress;
+   }
+
+   @Override
+   public void close() throws IOException {
+   channelStateWriter.close();
+   }
+
+   // Caches checkpoint output stream factories to prevent multiple output 
stream per checkpoint.
+   // This could result from requesting output stream by different 
entities (this and channelStateWriter)
+   // We can't just pass a stream to the channelStateWriter because it can 
receive checkpoint call earlier than this class
+   // in some unaligned checkpoints scenarios
+   private static class CachingCheckpointStorageWorkerView implements 
CheckpointStorageWorkerView {
 
 Review comment:
   Do we have the tests coverage for this new introduced caching function in 
the commit `[FLINK-16744][task] send channel state handles to JM`?


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r405294182
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
 ##
 @@ -1057,7 +1062,7 @@ else if (current == ExecutionState.RUNNING) {
// case the canceling could not 
continue
 
// The canceller calls cancel 
and interrupts the executing thread once
-   Runnable canceler = new 
TaskCanceler(LOG, this :: closeNetworkResources, invokable, executingThread, 
taskNameWithSubtask);
+   Runnable canceler = new 
TaskCanceler(LOG, this::closeNetworkResources, invokable, executingThread, 
taskNameWithSubtask);
 
 Review comment:
   nit: irrelevant change, better to have spaces


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r405284887
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java
 ##
 @@ -0,0 +1,186 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.state.CheckpointStorageWorkerView;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.io.IOException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeInput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeOutput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.write;
+
+/**
+ * {@link ChannelStateWriter} implemented using
+ * {@link CheckpointStreamFactory.CheckpointStateOutputStream 
CheckpointStateOutputStreams}. Internally, it has by default
+ * 
+ * one stream per checkpoint; having multiple streams would mean more 
files written and more connections opened
+ * (and more latency on restore)
+ * one thread; having multiple threads means more connections, couples 
with the implementation and increases complexity
+ * 
+ * Thread-safety: this class is thread-safe when used with a thread-safe 
{@link ChannelStateWriteRequestExecutor executor}
+ * (e.g. default {@link ChannelStateWriteRequestExecutorImpl}.
+ */
+@Internal
+@ThreadSafe
+public class ChannelStateWriterImpl implements ChannelStateWriter {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriterImpl.class);
+   private static final int DEFAULT_MAX_CHECKPOINTS = 5; // currently, 
only single in-flight checkpoint is supported
+
+   private final ChannelStateWriteRequestExecutor executor;
+   private final ConcurrentMap results;
+   private final int maxCheckpoints;
+
+   /**
+* Creates a {@link ChannelStateWriterImpl} with {@link 
#DEFAULT_MAX_CHECKPOINTS} as {@link #maxCheckpoints}.
+*/
+   public ChannelStateWriterImpl(CheckpointStorageWorkerView 
streamFactoryResolver) {
+   this(streamFactoryResolver, DEFAULT_MAX_CHECKPOINTS);
+   }
+
+   /**
+* Creates a {@link ChannelStateWriterImpl} with {@link 
ChannelStateSerializerImpl default} {@link ChannelStateSerializer},
+* and a {@link ChannelStateWriteRequestExecutorImpl}.
+*
+* @param maxCheckpointsmaximum number of checkpoints to be 
written currently or finished but not taken yet.
+* @param streamFactoryResolver a factory to obtain output stream 
factory for a given checkpoint
+*/
+   ChannelStateWriterImpl(CheckpointStorageWorkerView 
streamFactoryResolver, int maxCheckpoints) {
+   this(
+   new ConcurrentHashMap<>(maxCheckpoints),
+   new ChannelStateWriteRequestExecutorImpl(new 
ChannelStateWriteRequestDispatcherImpl(streamFactoryResolver, new 
ChannelStateSerializerImpl())),
+   maxCheckpoints
+   );
+   }
+
+   ChannelStateWriterImpl(ConcurrentMap 
results, ChannelStateWriteRequestExecutor executor, int maxCheckpoints) {
+   this.results = results;
+   this.maxCheckpoints = maxCheckpoints;
+   this.executor = executor;
+   }
+
+   @Override
+   public void start(long checkpointId, CheckpointOptions 
checkpointOptions) {
+   LOG.debug("start checkpoint {} ({})", checkpointId, 
checkpointOptions);
+   ChannelStateWriteResult result = new ChannelStateWriteResult();
+ 

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r405270284
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequestExecutorImpl.java
 ##
 @@ -0,0 +1,161 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.BlockingDeque;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.LinkedBlockingDeque;
+
+/**
+ * Executes {@link ChannelStateWriteRequest}s in a separate thread. Any 
exception occurred during execution causes this
+ * thread to stop and the exception to be re-thrown on any subsequent call.
+ */
+@ThreadSafe
+class ChannelStateWriteRequestExecutorImpl implements 
ChannelStateWriteRequestExecutor {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriteRequestExecutorImpl.class);
+   private static final int DEFAULT_HANDOVER_CAPACITY = 10_000;
+
+   private final ChannelStateWriteRequestDispatcher dispatcher;
+   private final BlockingDeque deque;
+   private final Thread thread;
+   private volatile Exception thrown = null;
+   private volatile boolean wasClosed = false;
+
+   ChannelStateWriteRequestExecutorImpl(ChannelStateWriteRequestDispatcher 
dispatcher) {
+   this(dispatcher, new 
LinkedBlockingDeque<>(DEFAULT_HANDOVER_CAPACITY));
+   }
+
+   ChannelStateWriteRequestExecutorImpl(ChannelStateWriteRequestDispatcher 
dispatcher, BlockingDeque deque) {
+   this.dispatcher = dispatcher;
+   this.deque = deque;
+   this.thread = new Thread(this::run);
+   this.thread.setDaemon(true);
+   }
+
+   @VisibleForTesting
+   void run() {
+   try {
+   loop();
+   } catch (Exception ex) {
+   thrown = ex;
+   } finally {
+   cleanupRequests();
+   dispatcher.close(thrown == null ? new 
CancellationException() : thrown);
+   }
+   LOG.debug("loop terminated");
+   }
+
+   private void loop() throws Exception {
+   while (isActive()) {
+   try {
+   dispatcher.dispatch(deque.take());
+   } catch (InterruptedException e) {
+   if (isActive()) {
+   LOG.debug("interrupted while waiting 
for a request (continue waiting)", e);
+   } else {
+   Thread.currentThread().interrupt();
+   }
+   }
+   }
+   }
+
+   private void cleanupRequests() {
+   Throwable cause = thrown == null ? new CancellationException() 
: thrown;
+   List drained = new ArrayList<>();
+   deque.drainTo(drained);
+   LOG.info("discarding {} drained requests", drained.size());
+   for (ChannelStateWriteRequest request : drained) {
+   request.cancel(cause);
+   }
+   }
+
+   @Override
+   public void start() throws IllegalStateException {
+   this.thread.start();
+   }
+
+   @Override
+   public void submit(ChannelStateWriteRequest request) throws Exception {
+   submitInternal(request, () -> deque.add(request));
+   }
+
+   @Override
+   public void submitPriority(ChannelStateWriteRequest request) throws 
Exception {
+   submitInternal(request, () -> deque.addFirst(request));
+   }
+
+

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r405262725
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequestExecutorImpl.java
 ##
 @@ -0,0 +1,161 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.BlockingDeque;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.LinkedBlockingDeque;
+
+/**
+ * Executes {@link ChannelStateWriteRequest}s in a separate thread. Any 
exception occurred during execution causes this
+ * thread to stop and the exception to be re-thrown on any subsequent call.
+ */
+@ThreadSafe
+class ChannelStateWriteRequestExecutorImpl implements 
ChannelStateWriteRequestExecutor {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriteRequestExecutorImpl.class);
+   private static final int DEFAULT_HANDOVER_CAPACITY = 10_000;
+
+   private final ChannelStateWriteRequestDispatcher dispatcher;
+   private final BlockingDeque deque;
+   private final Thread thread;
+   private volatile Exception thrown = null;
+   private volatile boolean wasClosed = false;
+
+   ChannelStateWriteRequestExecutorImpl(ChannelStateWriteRequestDispatcher 
dispatcher) {
+   this(dispatcher, new 
LinkedBlockingDeque<>(DEFAULT_HANDOVER_CAPACITY));
+   }
+
+   ChannelStateWriteRequestExecutorImpl(ChannelStateWriteRequestDispatcher 
dispatcher, BlockingDeque deque) {
+   this.dispatcher = dispatcher;
+   this.deque = deque;
+   this.thread = new Thread(this::run);
+   this.thread.setDaemon(true);
+   }
+
+   @VisibleForTesting
+   void run() {
+   try {
+   loop();
+   } catch (Exception ex) {
+   thrown = ex;
+   } finally {
+   cleanupRequests();
+   dispatcher.close(thrown == null ? new 
CancellationException() : thrown);
+   }
+   LOG.debug("loop terminated");
+   }
+
+   private void loop() throws Exception {
+   while (isActive()) {
+   try {
+   dispatcher.dispatch(deque.take());
+   } catch (InterruptedException e) {
+   if (isActive()) {
+   LOG.debug("interrupted while waiting 
for a request (continue waiting)", e);
+   } else {
+   Thread.currentThread().interrupt();
+   }
+   }
+   }
+   }
+
+   private void cleanupRequests() {
+   Throwable cause = thrown == null ? new CancellationException() 
: thrown;
+   List drained = new ArrayList<>();
+   deque.drainTo(drained);
+   LOG.info("discarding {} drained requests", drained.size());
+   for (ChannelStateWriteRequest request : drained) {
+   request.cancel(cause);
+   }
+   }
+
+   @Override
+   public void start() throws IllegalStateException {
+   this.thread.start();
+   }
+
+   @Override
+   public void submit(ChannelStateWriteRequest request) throws Exception {
+   submitInternal(request, () -> deque.add(request));
+   }
+
+   @Override
+   public void submitPriority(ChannelStateWriteRequest request) throws 
Exception {
+   submitInternal(request, () -> deque.addFirst(request));
+   }
+
+

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r405262725
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequestExecutorImpl.java
 ##
 @@ -0,0 +1,161 @@
+/*
+ * 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.annotation.VisibleForTesting;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.BlockingDeque;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.LinkedBlockingDeque;
+
+/**
+ * Executes {@link ChannelStateWriteRequest}s in a separate thread. Any 
exception occurred during execution causes this
+ * thread to stop and the exception to be re-thrown on any subsequent call.
+ */
+@ThreadSafe
+class ChannelStateWriteRequestExecutorImpl implements 
ChannelStateWriteRequestExecutor {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriteRequestExecutorImpl.class);
+   private static final int DEFAULT_HANDOVER_CAPACITY = 10_000;
+
+   private final ChannelStateWriteRequestDispatcher dispatcher;
+   private final BlockingDeque deque;
+   private final Thread thread;
+   private volatile Exception thrown = null;
+   private volatile boolean wasClosed = false;
+
+   ChannelStateWriteRequestExecutorImpl(ChannelStateWriteRequestDispatcher 
dispatcher) {
+   this(dispatcher, new 
LinkedBlockingDeque<>(DEFAULT_HANDOVER_CAPACITY));
+   }
+
+   ChannelStateWriteRequestExecutorImpl(ChannelStateWriteRequestDispatcher 
dispatcher, BlockingDeque deque) {
+   this.dispatcher = dispatcher;
+   this.deque = deque;
+   this.thread = new Thread(this::run);
+   this.thread.setDaemon(true);
+   }
+
+   @VisibleForTesting
+   void run() {
+   try {
+   loop();
+   } catch (Exception ex) {
+   thrown = ex;
+   } finally {
+   cleanupRequests();
+   dispatcher.close(thrown == null ? new 
CancellationException() : thrown);
+   }
+   LOG.debug("loop terminated");
+   }
+
+   private void loop() throws Exception {
+   while (isActive()) {
+   try {
+   dispatcher.dispatch(deque.take());
+   } catch (InterruptedException e) {
+   if (isActive()) {
+   LOG.debug("interrupted while waiting 
for a request (continue waiting)", e);
+   } else {
+   Thread.currentThread().interrupt();
+   }
+   }
+   }
+   }
+
+   private void cleanupRequests() {
+   Throwable cause = thrown == null ? new CancellationException() 
: thrown;
+   List drained = new ArrayList<>();
+   deque.drainTo(drained);
+   LOG.info("discarding {} drained requests", drained.size());
+   for (ChannelStateWriteRequest request : drained) {
+   request.cancel(cause);
+   }
+   }
+
+   @Override
+   public void start() throws IllegalStateException {
+   this.thread.start();
+   }
+
+   @Override
+   public void submit(ChannelStateWriteRequest request) throws Exception {
+   submitInternal(request, () -> deque.add(request));
+   }
+
+   @Override
+   public void submitPriority(ChannelStateWriteRequest request) throws 
Exception {
+   submitInternal(request, () -> deque.addFirst(request));
+   }
+
+

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r405256556
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequest.java
 ##
 @@ -0,0 +1,158 @@
+/*
+ * 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.runtime.checkpoint.channel.ChannelStateWriter.ChannelStateWriteResult;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.state.CheckpointStorageLocationReference;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+
+interface ChannelStateWriteRequest {
+   long getCheckpointId();
+
+   void cancel(Throwable cause);
+
+   static CheckpointInProgressRequest completeInput(long checkpointId) {
+   return new CheckpointInProgressRequest("completeInput", 
checkpointId, ChannelStateCheckpointWriter::completeInput, false);
+   }
+
+   static CheckpointInProgressRequest completeOutput(long checkpointId) {
+   return new CheckpointInProgressRequest("completeOutput", 
checkpointId, ChannelStateCheckpointWriter::completeOutput, false);
+   }
+
+   static ChannelStateWriteRequest write(long checkpointId, 
InputChannelInfo info, Buffer... flinkBuffers) {
+   return new CheckpointInProgressRequest("writeInput", 
checkpointId, writer -> writer.writeInput(info, flinkBuffers), 
recycle(flinkBuffers), false);
+   }
+
+   static ChannelStateWriteRequest write(long checkpointId, 
ResultSubpartitionInfo info, Buffer... flinkBuffers) {
+   return new CheckpointInProgressRequest("writeOutput", 
checkpointId, writer -> writer.writeOutput(info, flinkBuffers), 
recycle(flinkBuffers), false);
+   }
+
+   static ChannelStateWriteRequest start(long checkpointId, 
ChannelStateWriteResult targetResult, CheckpointStorageLocationReference 
locationReference) {
+   return new CheckpointStartRequest(checkpointId, targetResult, 
locationReference);
+   }
+
+   static ChannelStateWriteRequest abort(long checkpointId, Throwable 
cause) {
+   return new CheckpointInProgressRequest("abort", checkpointId, 
writer -> writer.fail(cause), true);
+   }
+
+   static Consumer recycle(Buffer[] flinkBuffers) {
+   return unused -> {
+   for (Buffer b : flinkBuffers) {
+   b.recycleBuffer();
+   }
+   };
+   }
+}
+
+final class CheckpointStartRequest implements ChannelStateWriteRequest {
+   private final ChannelStateWriteResult targetResult;
+   private final CheckpointStorageLocationReference locationReference;
+   private final long checkpointId;
+
+   CheckpointStartRequest(long checkpointId, ChannelStateWriteResult 
targetResult, CheckpointStorageLocationReference locationReference) {
+   this.checkpointId = checkpointId;
+   this.targetResult = targetResult;
+   this.locationReference = locationReference;
+   }
+
+   @Override
+   public long getCheckpointId() {
+   return checkpointId;
+   }
+
+   ChannelStateWriteResult getTargetResult() {
+   return targetResult;
+   }
+
+   public CheckpointStorageLocationReference getLocationReference() {
+   return locationReference;
+   }
+
+   @Override
+   public void cancel(Throwable cause) {
+   targetResult.fail(cause);
+   }
+
+   @Override
+   public String toString() {
+   return "start " + checkpointId;
+   }
+}
+
+final class CheckpointInProgressRequest implements ChannelStateWriteRequest {
+   private final ThrowingConsumer 
action;
+   private final Consumer discardAction;
+   private final long checkpointId;
+   private final String name;
+   private final bo

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r405256556
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequest.java
 ##
 @@ -0,0 +1,158 @@
+/*
+ * 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.runtime.checkpoint.channel.ChannelStateWriter.ChannelStateWriteResult;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.state.CheckpointStorageLocationReference;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+
+interface ChannelStateWriteRequest {
+   long getCheckpointId();
+
+   void cancel(Throwable cause);
+
+   static CheckpointInProgressRequest completeInput(long checkpointId) {
+   return new CheckpointInProgressRequest("completeInput", 
checkpointId, ChannelStateCheckpointWriter::completeInput, false);
+   }
+
+   static CheckpointInProgressRequest completeOutput(long checkpointId) {
+   return new CheckpointInProgressRequest("completeOutput", 
checkpointId, ChannelStateCheckpointWriter::completeOutput, false);
+   }
+
+   static ChannelStateWriteRequest write(long checkpointId, 
InputChannelInfo info, Buffer... flinkBuffers) {
+   return new CheckpointInProgressRequest("writeInput", 
checkpointId, writer -> writer.writeInput(info, flinkBuffers), 
recycle(flinkBuffers), false);
+   }
+
+   static ChannelStateWriteRequest write(long checkpointId, 
ResultSubpartitionInfo info, Buffer... flinkBuffers) {
+   return new CheckpointInProgressRequest("writeOutput", 
checkpointId, writer -> writer.writeOutput(info, flinkBuffers), 
recycle(flinkBuffers), false);
+   }
+
+   static ChannelStateWriteRequest start(long checkpointId, 
ChannelStateWriteResult targetResult, CheckpointStorageLocationReference 
locationReference) {
+   return new CheckpointStartRequest(checkpointId, targetResult, 
locationReference);
+   }
+
+   static ChannelStateWriteRequest abort(long checkpointId, Throwable 
cause) {
+   return new CheckpointInProgressRequest("abort", checkpointId, 
writer -> writer.fail(cause), true);
+   }
+
+   static Consumer recycle(Buffer[] flinkBuffers) {
+   return unused -> {
+   for (Buffer b : flinkBuffers) {
+   b.recycleBuffer();
+   }
+   };
+   }
+}
+
+final class CheckpointStartRequest implements ChannelStateWriteRequest {
+   private final ChannelStateWriteResult targetResult;
+   private final CheckpointStorageLocationReference locationReference;
+   private final long checkpointId;
+
+   CheckpointStartRequest(long checkpointId, ChannelStateWriteResult 
targetResult, CheckpointStorageLocationReference locationReference) {
+   this.checkpointId = checkpointId;
+   this.targetResult = targetResult;
+   this.locationReference = locationReference;
+   }
+
+   @Override
+   public long getCheckpointId() {
+   return checkpointId;
+   }
+
+   ChannelStateWriteResult getTargetResult() {
+   return targetResult;
+   }
+
+   public CheckpointStorageLocationReference getLocationReference() {
+   return locationReference;
+   }
+
+   @Override
+   public void cancel(Throwable cause) {
+   targetResult.fail(cause);
+   }
+
+   @Override
+   public String toString() {
+   return "start " + checkpointId;
+   }
+}
+
+final class CheckpointInProgressRequest implements ChannelStateWriteRequest {
+   private final ThrowingConsumer 
action;
+   private final Consumer discardAction;
+   private final long checkpointId;
+   private final String name;
+   private final bo

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r405248014
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequest.java
 ##
 @@ -0,0 +1,158 @@
+/*
+ * 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.runtime.checkpoint.channel.ChannelStateWriter.ChannelStateWriteResult;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.state.CheckpointStorageLocationReference;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Consumer;
+
+interface ChannelStateWriteRequest {
+   long getCheckpointId();
+
+   void cancel(Throwable cause);
+
+   static CheckpointInProgressRequest completeInput(long checkpointId) {
+   return new CheckpointInProgressRequest("completeInput", 
checkpointId, ChannelStateCheckpointWriter::completeInput, false);
+   }
+
+   static CheckpointInProgressRequest completeOutput(long checkpointId) {
+   return new CheckpointInProgressRequest("completeOutput", 
checkpointId, ChannelStateCheckpointWriter::completeOutput, false);
+   }
+
+   static ChannelStateWriteRequest write(long checkpointId, 
InputChannelInfo info, Buffer... flinkBuffers) {
+   return new CheckpointInProgressRequest("writeInput", 
checkpointId, writer -> writer.writeInput(info, flinkBuffers), 
recycle(flinkBuffers), false);
+   }
+
+   static ChannelStateWriteRequest write(long checkpointId, 
ResultSubpartitionInfo info, Buffer... flinkBuffers) {
+   return new CheckpointInProgressRequest("writeOutput", 
checkpointId, writer -> writer.writeOutput(info, flinkBuffers), 
recycle(flinkBuffers), false);
+   }
+
+   static ChannelStateWriteRequest start(long checkpointId, 
ChannelStateWriteResult targetResult, CheckpointStorageLocationReference 
locationReference) {
+   return new CheckpointStartRequest(checkpointId, targetResult, 
locationReference);
+   }
+
+   static ChannelStateWriteRequest abort(long checkpointId, Throwable 
cause) {
+   return new CheckpointInProgressRequest("abort", checkpointId, 
writer -> writer.fail(cause), true);
+   }
+
+   static Consumer recycle(Buffer[] flinkBuffers) {
+   return unused -> {
+   for (Buffer b : flinkBuffers) {
+   b.recycleBuffer();
+   }
+   };
+   }
+}
+
+final class CheckpointStartRequest implements ChannelStateWriteRequest {
+   private final ChannelStateWriteResult targetResult;
+   private final CheckpointStorageLocationReference locationReference;
+   private final long checkpointId;
+
+   CheckpointStartRequest(long checkpointId, ChannelStateWriteResult 
targetResult, CheckpointStorageLocationReference locationReference) {
+   this.checkpointId = checkpointId;
+   this.targetResult = targetResult;
+   this.locationReference = locationReference;
+   }
+
+   @Override
+   public long getCheckpointId() {
+   return checkpointId;
+   }
+
+   ChannelStateWriteResult getTargetResult() {
+   return targetResult;
+   }
+
+   public CheckpointStorageLocationReference getLocationReference() {
+   return locationReference;
+   }
+
+   @Override
+   public void cancel(Throwable cause) {
+   targetResult.fail(cause);
+   }
+
+   @Override
+   public String toString() {
+   return "start " + checkpointId;
+   }
+}
+
+final class CheckpointInProgressRequest implements ChannelStateWriteRequest {
+   private final ThrowingConsumer 
action;
+   private final Consumer discardAction;
+   private final long checkpointId;
+   private final String name;
+   private final bo

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r404967282
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateSerializer.java
 ##
 @@ -0,0 +1,182 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import static java.lang.Math.addExact;
+import static java.lang.Math.min;
+
+interface ChannelStateSerializer {
+
+   void writeHeader(DataOutputStream dataStream) throws IOException;
+
+   void writeData(DataOutputStream stream, Buffer... flinkBuffers) throws 
IOException;
+}
+
+interface ChannelStateDeserializer {
+
+   void readHeader(InputStream stream) throws IOException;
+
+   int readLength(InputStream stream) throws IOException;
+
+   int readData(InputStream stream, ChannelStateByteBuffer buffer, int 
bytes) throws IOException;
+}
+
+/**
+ * Wrapper around various buffers to receive channel state data.
+ */
+@Internal
+@NotThreadSafe
+interface ChannelStateByteBuffer {
+
+   boolean isWritable();
+
+   /**
+* Read up to bytesToRead bytes into this buffer from the 
given {@link InputStream}.
+* @return the total number of bytes read into this buffer.
+*/
+   int writeBytes(InputStream input, int bytesToRead) throws IOException;
+
+   static ChannelStateByteBuffer wrap(Buffer buffer) {
+   return new ChannelStateByteBuffer() {
+
+   private final ByteBuf byteBuf = buffer.asByteBuf();
+
+   @Override
+   public boolean isWritable() {
+   return byteBuf.isWritable();
+   }
+
+   @Override
+   public int writeBytes(InputStream input, int 
bytesToRead) throws IOException {
+   return byteBuf.writeBytes(input, 
Math.min(bytesToRead, byteBuf.writableBytes()));
+   }
+   };
+   }
+
+   static ChannelStateByteBuffer wrap(BufferBuilder bufferBuilder) {
+   final byte[] buf = new byte[1024];
+   return new ChannelStateByteBuffer() {
+   @Override
+   public boolean isWritable() {
+   return !bufferBuilder.isFull();
+   }
+
+   @Override
+   public int writeBytes(InputStream input, int 
bytesToRead) throws IOException {
+   int left = bytesToRead;
+   for (int toRead = getToRead(left); toRead > 0; 
toRead = getToRead(left)) {
+   int read = input.read(buf, 0, toRead);
+   int copied = 
bufferBuilder.append(java.nio.ByteBuffer.wrap(buf, 0, read));
+   Preconditions.checkState(copied == 
read);
+   left -= read;
+   }
+   bufferBuilder.commit();
+   return bytesToRead - left;
+   }
+
+   private int getToRead(int bytesToRead) {
+   return min(bytesToRead, min(buf.length, 
bufferBuilder.getWritableBytes()));
+   }
+   };
+   }
+
+   static ChannelStateByteBuffer wrap(byte[] bytes) {
+   return new ChannelStateByteBuffer() {
+   private int written = 0;
+
+   @Overri

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r404932029
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateCheckpointWriter.java
 ##
 @@ -0,0 +1,203 @@
+/*
+ * 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.runtime.checkpoint.channel.ChannelStateWriter.ChannelStateWriteResult;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import 
org.apache.flink.runtime.state.CheckpointStreamFactory.CheckpointStateOutputStream;
+import org.apache.flink.runtime.state.InputChannelStateHandle;
+import org.apache.flink.runtime.state.ResultSubpartitionStateHandle;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.BiFunction;
+
+import static org.apache.flink.runtime.state.CheckpointedStateScope.EXCLUSIVE;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Writes channel state for a specific checkpoint-subtask-attempt triple.
+ */
+@NotThreadSafe
+class ChannelStateCheckpointWriter {
+   private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateCheckpointWriter.class);
+
+   private final DataOutputStream dataStream;
+   private final CheckpointStateOutputStream checkpointStream;
+   private final ChannelStateWriteResult result;
+   private final Map> inputChannelOffsets = 
new HashMap<>();
+   private final Map> 
resultSubpartitionOffsets = new HashMap<>();
+   private final ChannelStateSerializer serializer;
+   private final long checkpointId;
+   private boolean allInputsReceived = false;
+   private boolean allOutputsReceived = false;
+   private final RunnableWithException onComplete;
+
+   ChannelStateCheckpointWriter(
+   CheckpointStartRequest startCheckpointItem,
+   CheckpointStreamFactory streamFactory,
+   ChannelStateSerializer serializer,
+   RunnableWithException onComplete) throws Exception {
+   this(
+   startCheckpointItem.getCheckpointId(),
+   startCheckpointItem.getTargetResult(),
+   
streamFactory.createCheckpointStateOutputStream(EXCLUSIVE),
+   serializer,
+   onComplete);
+   }
+
+   ChannelStateCheckpointWriter(
+   long checkpointId,
+   ChannelStateWriteResult result,
+   CheckpointStateOutputStream stream,
+   ChannelStateSerializer serializer,
+   RunnableWithException onComplete) throws Exception {
+   this(checkpointId, result, serializer, onComplete, stream, new 
DataOutputStream(stream));
+   }
+
+   ChannelStateCheckpointWriter(
+   long checkpointId,
+   ChannelStateWriteResult result,
+   ChannelStateSerializer serializer,
+   RunnableWithException onComplete,
+   CheckpointStateOutputStream checkpointStateOutputStream,
+   DataOutputStream dataStream) throws Exception {
+   this.checkpointId = checkpointId;
+   this.result = checkNotNull(result);
+   this.checkpointStream = 
checkNotNull(che

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r404927386
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateCheckpointWriter.java
 ##
 @@ -0,0 +1,203 @@
+/*
+ * 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.runtime.checkpoint.channel.ChannelStateWriter.ChannelStateWriteResult;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import 
org.apache.flink.runtime.state.CheckpointStreamFactory.CheckpointStateOutputStream;
+import org.apache.flink.runtime.state.InputChannelStateHandle;
+import org.apache.flink.runtime.state.ResultSubpartitionStateHandle;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.RunnableWithException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.BiFunction;
+
+import static org.apache.flink.runtime.state.CheckpointedStateScope.EXCLUSIVE;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * Writes channel state for a specific checkpoint-subtask-attempt triple.
+ */
+@NotThreadSafe
+class ChannelStateCheckpointWriter {
+   private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateCheckpointWriter.class);
+
+   private final DataOutputStream dataStream;
+   private final CheckpointStateOutputStream checkpointStream;
+   private final ChannelStateWriteResult result;
+   private final Map> inputChannelOffsets = 
new HashMap<>();
+   private final Map> 
resultSubpartitionOffsets = new HashMap<>();
+   private final ChannelStateSerializer serializer;
+   private final long checkpointId;
+   private boolean allInputsReceived = false;
+   private boolean allOutputsReceived = false;
+   private final RunnableWithException onComplete;
+
+   ChannelStateCheckpointWriter(
+   CheckpointStartRequest startCheckpointItem,
+   CheckpointStreamFactory streamFactory,
+   ChannelStateSerializer serializer,
+   RunnableWithException onComplete) throws Exception {
+   this(
+   startCheckpointItem.getCheckpointId(),
+   startCheckpointItem.getTargetResult(),
+   
streamFactory.createCheckpointStateOutputStream(EXCLUSIVE),
+   serializer,
+   onComplete);
+   }
+
+   ChannelStateCheckpointWriter(
+   long checkpointId,
+   ChannelStateWriteResult result,
+   CheckpointStateOutputStream stream,
+   ChannelStateSerializer serializer,
+   RunnableWithException onComplete) throws Exception {
+   this(checkpointId, result, serializer, onComplete, stream, new 
DataOutputStream(stream));
+   }
+
+   ChannelStateCheckpointWriter(
+   long checkpointId,
+   ChannelStateWriteResult result,
+   ChannelStateSerializer serializer,
+   RunnableWithException onComplete,
+   CheckpointStateOutputStream checkpointStateOutputStream,
+   DataOutputStream dataStream) throws Exception {
+   this.checkpointId = checkpointId;
+   this.result = checkNotNull(result);
+   this.checkpointStream = 
checkNotNull(che

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r404920180
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriter.java
 ##
 @@ -73,10 +73,20 @@
void addOutputData(long checkpointId, ResultSubpartitionInfo info, int 
startSeqNum, Buffer... data);
 
/**
-* Finalize write of channel state for the given checkpoint id.
-* Must be called after {@link #start(long)} and all of the data of the 
given checkpoint added.
+* Finalize write of channel state data for the given checkpoint id.
+* Must be called after {@link #start(long)} and all of the input data 
of the given checkpoint added.
+* When both {@link #finishInput} and {@link #finishOutput} were called 
the results can be (eventually) obtained
+* using {@link #getWriteCompletionFuture}
 */
-   void finish(long checkpointId);
 
 Review comment:
   we should also adjust the references of `{@link #finish(long)}` in above 
`addInputData` and `addOutput` descriptions for the commit "[FLINK-16744][task] 
split finish() in ChanStateWrite"


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r404908472
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java
 ##
 @@ -65,6 +75,8 @@ public void checkpointState(
CheckpointMetrics checkpointMetrics,
OperatorChain operatorChain,
Supplier isCanceled) throws Exception {
+   checkNotNull(checkpointOptions);
 
 Review comment:
   When checking the commit "[FLINK-16744][task][refactor] inline 
CheckpointingOperation", we only `checkNotNull` for `checkpointOptions` and 
`checkpointMetrics`.  But from the commit "[FLINK-16744][task][refactor] 
extract SubtaskCheckpointCoordinator" when this method is firstly introduced,  
we did not `checkNotNull` for other arguments which are actually referenced in 
that commit, so the criteria seems inconsistent. 


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r404901747
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java
 ##
 @@ -61,11 +65,6 @@
this.actionExecutor = actionExecutor;
}
 
-   @Override
 
 Review comment:
   From the commit "[FLINK-16744][task][refactor] inline 
CheckpointingOperation", it is unnecessary changes to move the position of this 
method, if we want to merge this commit separately.


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r404894581
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java
 ##
 @@ -0,0 +1,124 @@
+/*
+ * 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.runtime.tasks;
+
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
+import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.state.CheckpointStorageWorkerView;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ExecutorService;
+import java.util.function.Supplier;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+class SubtaskCheckpointCoordinatorImpl implements SubtaskCheckpointCoordinator 
{
+   private static final Logger LOG = 
LoggerFactory.getLogger(SubtaskCheckpointCoordinatorImpl.class);
+
+   private final CheckpointStorageWorkerView checkpointStorage;
+   private final String taskName;
+   private final CloseableRegistry closeableRegistry;
+   private final ExecutorService executorService;
+   private final Environment env;
+   private final AsyncExceptionHandler asyncExceptionHandler;
+   private final StreamTaskActionExecutor actionExecutor;
+
+   SubtaskCheckpointCoordinatorImpl(
+   CheckpointStorageWorkerView checkpointStorage,
+   String taskName,
+   StreamTaskActionExecutor actionExecutor,
+   CloseableRegistry closeableRegistry,
+   ExecutorService executorService,
+   Environment env,
+   AsyncExceptionHandler asyncExceptionHandler) {
+   this.checkpointStorage = checkNotNull(checkpointStorage);
+   this.taskName = checkNotNull(taskName);
+   this.closeableRegistry = checkNotNull(closeableRegistry);
+   this.executorService = checkNotNull(executorService);
+   this.env = checkNotNull(env);
+   this.asyncExceptionHandler = 
checkNotNull(asyncExceptionHandler);
+   this.actionExecutor = actionExecutor;
 
 Review comment:
   nit: only missing `checkNotNull` for the last argument `actionExecutor`


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r404804329
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironmentBuilder.java
 ##
 @@ -154,6 +159,12 @@ public MockEnvironment build() {
subtaskIndex,
userCodeClassLoader,
taskMetricGroup,
-   taskManagerRuntimeInfo);
+   taskManagerRuntimeInfo,
+   memoryManager);
+   }
+
+   public MockEnvironmentBuilder setMemoryManager(MemoryManager 
memoryManager) {
 
 Review comment:
   nit: actually this is not used now in all places and we can add it by 
demands future. If we want to rich the builder now, it is better to place it in 
front of `#build()` to make related methods close with each other.


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r404802815
 
 

 ##
 File path: 
flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/over/BufferDataOverWindowOperatorTest.java
 ##
 @@ -237,4 +241,9 @@ public StreamingRuntimeContext getRuntimeContext() {
private void addRow(Object... fields) throws Exception {
operator.processElement(new 
StreamRecord<>(GenericRow.of(fields)));
}
+
+   private interface EnvironmentSupport {
 
 Review comment:
   this seems never be used.


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r404736967
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateSerializer.java
 ##
 @@ -0,0 +1,199 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import static java.lang.Math.addExact;
+import static java.lang.Math.min;
+
+interface ChannelStateSerializer {
+
+   void writeHeader(DataOutputStream dataStream) throws IOException;
+
+   void writeData(DataOutputStream stream, Buffer... flinkBuffers) throws 
IOException;
+}
+
+interface ChannelStateDeserializer {
+
+   void readHeader(InputStream stream) throws IOException;
+
+   /**
+* Reads the length of state.
+*
+* @return size of state in bytes and offset of the current position 
resulted from reading.
+*/
+   Tuple2 readLength(InputStream stream) throws 
IOException;
+
+   int readData(InputStream stream, ChannelStateByteBuffer buffer, int 
bytes) throws IOException;
+}
+
+/**
+ * Wrapper around various buffers to receive channel state data.
+ */
+@Internal
+@NotThreadSafe
+interface ChannelStateByteBuffer {
+
+   boolean isWritable();
+
+   int writableBytes();
+
+   int writeBytes(InputStream input, int bytesToRead) throws IOException;
+
+   static ChannelStateByteBuffer wrap(Buffer buffer) {
+   return new ChannelStateByteBuffer() {
+
+   private ByteBuf byteBuf = buffer.asByteBuf();
+
+   @Override
+   public boolean isWritable() {
+   return byteBuf.isWritable();
+   }
+
+   @Override
+   public int writableBytes() {
+   return byteBuf.writableBytes();
+   }
+
+   @Override
+   public int writeBytes(InputStream input, int 
bytesToRead) throws IOException {
+   return byteBuf.writeBytes(input, 
Math.min(bytesToRead, byteBuf.writableBytes()));
+   }
+   };
+   }
+
+   static ChannelStateByteBuffer wrap(BufferBuilder bufferBuilder) {
+   final byte[] buf = new byte[1024];
 
 Review comment:
   Yes, I remembered this discussion before and I agree this extra copy can not 
be avoided based on current codes. My previous assumption was that it would 
reuse the same bytes always like we did in 
`SpillingAdaptiveSpanningRecordDeserializer#SpanningWrapper#buffer`, to reduce 
GC pressure for many short-live objects.
   
   If it is not easy to clear it, I am also fine with current way since it is 
not in critical path, only for recovery process. Then we can optimize it if 
necessary future. 


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r404730400
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java
 ##
 @@ -0,0 +1,225 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.state.CheckpointStorageWorkerView;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeInput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeOutput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.write;
+
+/**
+ * {@link ChannelStateWriter} implemented using
+ * {@link CheckpointStreamFactory.CheckpointStateOutputStream 
CheckpointStateOutputStreams}. Internally, it has
+ * 
+ * one stream per checkpoint; having multiple streams would mean more 
files written and more connections opened
+ * (and more latency on restore)
+ * one thread; having multiple threads means more connections, couples 
with the implementation and increases complexity
+ * 
+ */
+@Internal
+@ThreadSafe
+public class ChannelStateWriterImpl implements ChannelStateWriter {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriterImpl.class);
+   private static final int DEFAULT_HANDOVER_CAPACITY = 10;
+   private static final int DEFAULT_MAX_CHECKPOINTS = 5; // currently, 
only single in-flight checkpoint is supported
+
+   private class ProcessRequestsLoop implements Runnable {
+   private final ChannelStateWriteRequestProcessor 
requestProcessor;
+
+   private ProcessRequestsLoop(int maxCheckpoints, 
CheckpointStorageWorkerView streamFactoryFactory, ChannelStateSerializer 
serializer) {
+   this.requestProcessor = new 
ChannelStateWriteRequestProcessor(maxCheckpoints, streamFactoryFactory, 
serializer);
+   }
+
+   @Override
+   public void run() {
+   try {
+   loop();
+   } catch (Exception ex) {
+   thrown = ex;
+   } finally {
+   handover.clear();
+   requestProcessor.cleanup(thrown != null ? 
thrown : new RuntimeException("loop terminated, isRunning: " + isRunning));
+   }
+   LOG.debug("loop terminated");
+   }
+
+   private void loop() throws Exception {
+   while (isRunning || !handover.isEmpty()) {
+   try {
+   
requestProcessor.processRequest(handover.take());
+   } catch (InterruptedException e) {
+   if (isRunning || !handover.isEmpty()) {
+   LOG.info("interrupted while 
waiting for an item (continue waiting)", e);
+   } else {
+   
Thread.currentThread().interrupt();
+   return;
+   }
+   }
+   }
+   }
+   }
+
+   private final Thread asyncWriter;
+   private final BlockingQueue handover;
+   private final Map results;
+   pr

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r404720981
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReaderImpl.java
 ##
 @@ -0,0 +1,116 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
+import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
+import 
org.apache.flink.runtime.checkpoint.channel.RefCountingFSDataInputStream.RefCountingFSDataInputStreamFactory;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava18.com.google.common.io.Closer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import static java.util.Arrays.asList;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link ChannelStateReader} implementation. Usage considerations:
+ * 
+ * state of a channel can be read once per instance of this class; 
once done it returns
+ * {@link 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult#NO_MORE_DATA
 NO_MORE_DATA}
+ * reader/writer indices of the passed buffer are respected and 
updated
+ * buffers must be prepared (cleared) before passing to reader
+ * buffers must be released after use
+ * 
+ */
+@NotThreadSafe
+@Internal
+public class ChannelStateReaderImpl implements ChannelStateReader {
+   private static final Logger log = 
LoggerFactory.getLogger(ChannelStateReaderImpl.class);
+
+   private final Map 
inputChannelHandleReaders;
+   private final Map 
resultSubpartitionHandleReaders;
+
+   public ChannelStateReaderImpl(TaskStateSnapshot snapshot) {
+   this(snapshot, new ChannelStateSerializerImpl());
+   }
+
+   @SuppressWarnings("WeakerAccess")
+   ChannelStateReaderImpl(TaskStateSnapshot snapshot, 
ChannelStateDeserializer serializer) {
+   RefCountingFSDataInputStreamFactory streamFactory = new 
RefCountingFSDataInputStreamFactory(serializer);
+   final HashMap 
inputChannelHandleReadersTmp = new HashMap<>();
 
 Review comment:
   I guess you are indicating the concurrent issue. Before the constructor is 
finished, some other threads might access intermediate 
`inputChannelHandleReaders` and `resultSubpartitionHandleReaders`. 
   
   If so, it seems not valid because no one can get and reference 
`ChannelStateReaderImpl` before the constructor finishes. In real codes, the 
constructing is done before creating `Task` by RPC thread, then no one would 
reference it to read or close until it is created to pass into `task` class.
   
   Anyway it is not big issue on my side, just to clarify the consideration.


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r404603624
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java
 ##
 @@ -0,0 +1,225 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.state.CheckpointStorageWorkerView;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeInput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeOutput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.write;
+
+/**
+ * {@link ChannelStateWriter} implemented using
+ * {@link CheckpointStreamFactory.CheckpointStateOutputStream 
CheckpointStateOutputStreams}. Internally, it has
+ * 
+ * one stream per checkpoint; having multiple streams would mean more 
files written and more connections opened
+ * (and more latency on restore)
+ * one thread; having multiple threads means more connections, couples 
with the implementation and increases complexity
+ * 
+ */
+@Internal
+@ThreadSafe
+public class ChannelStateWriterImpl implements ChannelStateWriter {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriterImpl.class);
+   private static final int DEFAULT_HANDOVER_CAPACITY = 10;
+   private static final int DEFAULT_MAX_CHECKPOINTS = 5; // currently, 
only single in-flight checkpoint is supported
+
+   private class ProcessRequestsLoop implements Runnable {
+   private final ChannelStateWriteRequestProcessor 
requestProcessor;
+
+   private ProcessRequestsLoop(int maxCheckpoints, 
CheckpointStorageWorkerView streamFactoryFactory, ChannelStateSerializer 
serializer) {
+   this.requestProcessor = new 
ChannelStateWriteRequestProcessor(maxCheckpoints, streamFactoryFactory, 
serializer);
+   }
+
+   @Override
+   public void run() {
+   try {
+   loop();
+   } catch (Exception ex) {
+   thrown = ex;
+   } finally {
+   handover.clear();
+   requestProcessor.cleanup(thrown != null ? 
thrown : new RuntimeException("loop terminated, isRunning: " + isRunning));
+   }
+   LOG.debug("loop terminated");
+   }
+
+   private void loop() throws Exception {
+   while (isRunning || !handover.isEmpty()) {
+   try {
+   
requestProcessor.processRequest(handover.take());
+   } catch (InterruptedException e) {
+   if (isRunning || !handover.isEmpty()) {
+   LOG.info("interrupted while 
waiting for an item (continue waiting)", e);
+   } else {
+   
Thread.currentThread().interrupt();
+   return;
+   }
+   }
+   }
+   }
+   }
+
+   private final Thread asyncWriter;
+   private final BlockingQueue handover;
 
 Review comment:
   If it is easier for usi

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-07 Thread GitBox
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_r404597718
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReaderImpl.java
 ##
 @@ -0,0 +1,116 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
+import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
+import 
org.apache.flink.runtime.checkpoint.channel.RefCountingFSDataInputStream.RefCountingFSDataInputStreamFactory;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava18.com.google.common.io.Closer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import static java.util.Arrays.asList;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link ChannelStateReader} implementation. Usage considerations:
+ * 
+ * state of a channel can be read once per instance of this class; 
once done it returns
+ * {@link 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult#NO_MORE_DATA
 NO_MORE_DATA}
+ * reader/writer indices of the passed buffer are respected and 
updated
+ * buffers must be prepared (cleared) before passing to reader
+ * buffers must be released after use
+ * 
+ */
+@NotThreadSafe
+@Internal
+public class ChannelStateReaderImpl implements ChannelStateReader {
+   private static final Logger log = 
LoggerFactory.getLogger(ChannelStateReaderImpl.class);
+
+   private final Map 
inputChannelHandleReaders;
+   private final Map 
resultSubpartitionHandleReaders;
+
+   public ChannelStateReaderImpl(TaskStateSnapshot snapshot) {
+   this(snapshot, new ChannelStateSerializerImpl());
+   }
+
+   @SuppressWarnings("WeakerAccess")
+   ChannelStateReaderImpl(TaskStateSnapshot snapshot, 
ChannelStateDeserializer serializer) {
+   RefCountingFSDataInputStreamFactory streamFactory = new 
RefCountingFSDataInputStreamFactory(serializer);
+   final HashMap 
inputChannelHandleReadersTmp = new HashMap<>();
+   final HashMap 
resultSubpartitionHandleReadersTmp = new HashMap<>();
+   for (Map.Entry e : 
snapshot.getSubtaskStateMappings()) {
+   addReaders(inputChannelHandleReadersTmp, 
e.getValue().getInputChannelState(), streamFactory);
+   addReaders(resultSubpartitionHandleReadersTmp, 
e.getValue().getResultSubpartitionState(), streamFactory);
+   }
+   inputChannelHandleReaders = inputChannelHandleReadersTmp;
+   resultSubpartitionHandleReaders = 
resultSubpartitionHandleReadersTmp;
+   }
+
+   private  void addReaders(Map readerMap, 
Collection> handles, 
RefCountingFSDataInputStreamFactory streamFactory) {
+   for (AbstractChannelStateHandle handle : handles) {
+   checkState(!readerMap.containsKey(handle.getInfo()), 
"multiple states exist for channel: " + handle.getInfo());
+   readerMap.put(handle.getInfo(), new 
ChannelStateStreamReader(handle, streamFactory));
+   }
+   }
+
+   @Override
+   public ReadResult readInputData(InputChannelInfo info, Buffer buffer) 
throws IOException {
+   log.debug("readInputData, resultSubpartitionInfo: {} , 
bufferBuilder {}", info, buffer);
+   return getReader(info, 
inputChannelHandleReaders).readInto(buffer);
+   }
+
+   @Override
+   public ReadResult readOutputData(ResultSubpartitionInfo inf

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-05 Thread GitBox
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_r403721544
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java
 ##
 @@ -175,6 +203,77 @@ public void checkpointState(
}
}
 
+   private OperatorSnapshotFutures buildOperatorSnapshotFutures(
+   CheckpointMetaData checkpointMetaData,
+   CheckpointOptions checkpointOptions,
+   OperatorChain operatorChain,
+   StreamOperator op,
+   Supplier isCanceled,
+   ChannelStateWriteResult channelStateWriteResult) throws 
Exception {
+   CheckpointStreamFactory storage = 
checkpointStorage.resolveCheckpointStorageLocation(
+   checkpointMetaData.getCheckpointId(),
+   checkpointOptions.getTargetLocation());
+   OperatorSnapshotFutures snapshotInProgress = 
checkpointStreamOperator(
+   op,
+   checkpointMetaData,
+   checkpointOptions,
+   storage,
+   isCanceled);
+   if (op == operatorChain.getHeadOperator()) {
+   
snapshotInProgress.setInputChannelStateFuture(channelStateWriteResult
+   
.getInputChannelStateHandles()
+   
.thenApply(StateObjectCollection::new)
+   
.thenApply(SnapshotResult::of)
+   );
+   }
+   if (op == operatorChain.getTailOperator()) {
+   
snapshotInProgress.setResultSubpartitionStateFuture(channelStateWriteResult
+   

.getResultSubpartitionStateHandles()
+   

.thenApply(StateObjectCollection::new)
+   

.thenApply(SnapshotResult::of)
+   );
+   }
+   return snapshotInProgress;
+   }
+
+   @Override
+   public void close() {
+   channelStateWriter.close();
+   }
+
+   // Caches checkpoint output stream factories to prevent multiple output 
stream per checkpoint.
+   // This could result from requesting output stream by different 
entities (this and channelStateWriter)
+   // We can't just pass a stream to the channelStateWriter because it can 
receive checkpoint call earlier than this class
+   // in some unaligned checkpoints scenarios
+   private static class CachingCheckpointStorageWorkerView implements 
CheckpointStorageWorkerView {
+   private final Map cache = new 
ConcurrentHashMap<>();
+   private final CheckpointStorageWorkerView delegate;
+
+   private 
CachingCheckpointStorageWorkerView(CheckpointStorageWorkerView delegate) {
+   this.delegate = delegate;
+   }
+
+   void clearCacheFor(long checkpointId) {
+   cache.remove(checkpointId);
+   }
+
+   @Override
+   public CheckpointStreamFactory 
resolveCheckpointStorageLocation(long checkpointId, 
CheckpointStorageLocationReference reference) {
+   return cache.computeIfAbsent(checkpointId, id -> {
+   try {
+   return 
delegate.resolveCheckpointStorageLocation(checkpointId, reference);
+   } catch (IOException e) {
+   throw new WrappingRuntimeException(e);
 
 Review comment:
   Sorry for misleading. Some concerns are the same as 
https://github.com/apache/flink/pull/11515#discussion_r402050562, and another 
tiny concern is why not use `FlinkRuntimeException` directly? I have not found 
any special purpose for `WrappingRuntimeException`. 


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...@i

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-05 Thread GitBox
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_r403720984
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java
 ##
 @@ -0,0 +1,225 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.state.CheckpointStorageWorkerView;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeInput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeOutput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.write;
+
+/**
+ * {@link ChannelStateWriter} implemented using
+ * {@link CheckpointStreamFactory.CheckpointStateOutputStream 
CheckpointStateOutputStreams}. Internally, it has
+ * 
+ * one stream per checkpoint; having multiple streams would mean more 
files written and more connections opened
+ * (and more latency on restore)
+ * one thread; having multiple threads means more connections, couples 
with the implementation and increases complexity
+ * 
+ */
+@Internal
+@ThreadSafe
+public class ChannelStateWriterImpl implements ChannelStateWriter {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriterImpl.class);
+   private static final int DEFAULT_HANDOVER_CAPACITY = 10;
+   private static final int DEFAULT_MAX_CHECKPOINTS = 5; // currently, 
only single in-flight checkpoint is supported
+
+   private class ProcessRequestsLoop implements Runnable {
+   private final ChannelStateWriteRequestProcessor 
requestProcessor;
+
+   private ProcessRequestsLoop(int maxCheckpoints, 
CheckpointStorageWorkerView streamFactoryFactory, ChannelStateSerializer 
serializer) {
+   this.requestProcessor = new 
ChannelStateWriteRequestProcessor(maxCheckpoints, streamFactoryFactory, 
serializer);
+   }
+
+   @Override
+   public void run() {
+   try {
+   loop();
+   } catch (Exception ex) {
+   thrown = ex;
+   } finally {
+   handover.clear();
+   requestProcessor.cleanup(thrown != null ? 
thrown : new RuntimeException("loop terminated, isRunning: " + isRunning));
+   }
+   LOG.debug("loop terminated");
+   }
+
+   private void loop() throws Exception {
+   while (isRunning || !handover.isEmpty()) {
+   try {
+   
requestProcessor.processRequest(handover.take());
+   } catch (InterruptedException e) {
+   if (isRunning || !handover.isEmpty()) {
+   LOG.info("interrupted while 
waiting for an item (continue waiting)", e);
+   } else {
+   
Thread.currentThread().interrupt();
+   return;
+   }
+   }
+   }
+   }
+   }
+
+   private final Thread asyncWriter;
+   private final BlockingQueue handover;
+   private final Map results;
+   pr

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-05 Thread GitBox
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_r403717544
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java
 ##
 @@ -65,6 +75,8 @@ public void checkpointState(
CheckpointMetrics checkpointMetrics,
OperatorChain operatorChain,
Supplier isCanceled) throws Exception {
+   checkNotNull(checkpointOptions);
 
 Review comment:
   When I checked the commit "[FLINK-16744][task][refactor] extract 
SubtaskCheckpointCoordinator",  the arguments of `CheckpointMetaData`, 
`CheckpointOptions` and `operatorChain` are already referenced in below lines, 
so i suggested `checkNotNull` for them in that commit.


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-04 Thread GitBox
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_r403487299
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java
 ##
 @@ -0,0 +1,225 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.state.CheckpointStorageWorkerView;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeInput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeOutput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.write;
+
+/**
+ * {@link ChannelStateWriter} implemented using
+ * {@link CheckpointStreamFactory.CheckpointStateOutputStream 
CheckpointStateOutputStreams}. Internally, it has
+ * 
+ * one stream per checkpoint; having multiple streams would mean more 
files written and more connections opened
+ * (and more latency on restore)
+ * one thread; having multiple threads means more connections, couples 
with the implementation and increases complexity
+ * 
+ */
+@Internal
+@ThreadSafe
+public class ChannelStateWriterImpl implements ChannelStateWriter {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriterImpl.class);
+   private static final int DEFAULT_HANDOVER_CAPACITY = 10;
+   private static final int DEFAULT_MAX_CHECKPOINTS = 5; // currently, 
only single in-flight checkpoint is supported
+
+   private class ProcessRequestsLoop implements Runnable {
+   private final ChannelStateWriteRequestProcessor 
requestProcessor;
+
+   private ProcessRequestsLoop(int maxCheckpoints, 
CheckpointStorageWorkerView streamFactoryFactory, ChannelStateSerializer 
serializer) {
+   this.requestProcessor = new 
ChannelStateWriteRequestProcessor(maxCheckpoints, streamFactoryFactory, 
serializer);
+   }
+
+   @Override
+   public void run() {
+   try {
+   loop();
+   } catch (Exception ex) {
+   thrown = ex;
+   } finally {
+   handover.clear();
+   requestProcessor.cleanup(thrown != null ? 
thrown : new RuntimeException("loop terminated, isRunning: " + isRunning));
+   }
+   LOG.debug("loop terminated");
+   }
+
+   private void loop() throws Exception {
+   while (isRunning || !handover.isEmpty()) {
+   try {
+   
requestProcessor.processRequest(handover.take());
+   } catch (InterruptedException e) {
+   if (isRunning || !handover.isEmpty()) {
+   LOG.info("interrupted while 
waiting for an item (continue waiting)", e);
+   } else {
+   
Thread.currentThread().interrupt();
+   return;
+   }
+   }
+   }
+   }
+   }
+
+   private final Thread asyncWriter;
+   private final BlockingQueue handover;
 
 Review comment:
   Thanks for sharing your

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-04 Thread GitBox
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_r403487299
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java
 ##
 @@ -0,0 +1,225 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.state.CheckpointStorageWorkerView;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeInput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeOutput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.write;
+
+/**
+ * {@link ChannelStateWriter} implemented using
+ * {@link CheckpointStreamFactory.CheckpointStateOutputStream 
CheckpointStateOutputStreams}. Internally, it has
+ * 
+ * one stream per checkpoint; having multiple streams would mean more 
files written and more connections opened
+ * (and more latency on restore)
+ * one thread; having multiple threads means more connections, couples 
with the implementation and increases complexity
+ * 
+ */
+@Internal
+@ThreadSafe
+public class ChannelStateWriterImpl implements ChannelStateWriter {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriterImpl.class);
+   private static final int DEFAULT_HANDOVER_CAPACITY = 10;
+   private static final int DEFAULT_MAX_CHECKPOINTS = 5; // currently, 
only single in-flight checkpoint is supported
+
+   private class ProcessRequestsLoop implements Runnable {
+   private final ChannelStateWriteRequestProcessor 
requestProcessor;
+
+   private ProcessRequestsLoop(int maxCheckpoints, 
CheckpointStorageWorkerView streamFactoryFactory, ChannelStateSerializer 
serializer) {
+   this.requestProcessor = new 
ChannelStateWriteRequestProcessor(maxCheckpoints, streamFactoryFactory, 
serializer);
+   }
+
+   @Override
+   public void run() {
+   try {
+   loop();
+   } catch (Exception ex) {
+   thrown = ex;
+   } finally {
+   handover.clear();
+   requestProcessor.cleanup(thrown != null ? 
thrown : new RuntimeException("loop terminated, isRunning: " + isRunning));
+   }
+   LOG.debug("loop terminated");
+   }
+
+   private void loop() throws Exception {
+   while (isRunning || !handover.isEmpty()) {
+   try {
+   
requestProcessor.processRequest(handover.take());
+   } catch (InterruptedException e) {
+   if (isRunning || !handover.isEmpty()) {
+   LOG.info("interrupted while 
waiting for an item (continue waiting)", e);
+   } else {
+   
Thread.currentThread().interrupt();
+   return;
+   }
+   }
+   }
+   }
+   }
+
+   private final Thread asyncWriter;
+   private final BlockingQueue handover;
 
 Review comment:
   Thanks for sharing your

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-04 Thread GitBox
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_r403487299
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java
 ##
 @@ -0,0 +1,225 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.state.CheckpointStorageWorkerView;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeInput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeOutput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.write;
+
+/**
+ * {@link ChannelStateWriter} implemented using
+ * {@link CheckpointStreamFactory.CheckpointStateOutputStream 
CheckpointStateOutputStreams}. Internally, it has
+ * 
+ * one stream per checkpoint; having multiple streams would mean more 
files written and more connections opened
+ * (and more latency on restore)
+ * one thread; having multiple threads means more connections, couples 
with the implementation and increases complexity
+ * 
+ */
+@Internal
+@ThreadSafe
+public class ChannelStateWriterImpl implements ChannelStateWriter {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriterImpl.class);
+   private static final int DEFAULT_HANDOVER_CAPACITY = 10;
+   private static final int DEFAULT_MAX_CHECKPOINTS = 5; // currently, 
only single in-flight checkpoint is supported
+
+   private class ProcessRequestsLoop implements Runnable {
+   private final ChannelStateWriteRequestProcessor 
requestProcessor;
+
+   private ProcessRequestsLoop(int maxCheckpoints, 
CheckpointStorageWorkerView streamFactoryFactory, ChannelStateSerializer 
serializer) {
+   this.requestProcessor = new 
ChannelStateWriteRequestProcessor(maxCheckpoints, streamFactoryFactory, 
serializer);
+   }
+
+   @Override
+   public void run() {
+   try {
+   loop();
+   } catch (Exception ex) {
+   thrown = ex;
+   } finally {
+   handover.clear();
+   requestProcessor.cleanup(thrown != null ? 
thrown : new RuntimeException("loop terminated, isRunning: " + isRunning));
+   }
+   LOG.debug("loop terminated");
+   }
+
+   private void loop() throws Exception {
+   while (isRunning || !handover.isEmpty()) {
+   try {
+   
requestProcessor.processRequest(handover.take());
+   } catch (InterruptedException e) {
+   if (isRunning || !handover.isEmpty()) {
+   LOG.info("interrupted while 
waiting for an item (continue waiting)", e);
+   } else {
+   
Thread.currentThread().interrupt();
+   return;
+   }
+   }
+   }
+   }
+   }
+
+   private final Thread asyncWriter;
+   private final BlockingQueue handover;
 
 Review comment:
   Thanks for sharing your

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-04 Thread GitBox
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_r403487299
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java
 ##
 @@ -0,0 +1,225 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.state.CheckpointStorageWorkerView;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeInput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeOutput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.write;
+
+/**
+ * {@link ChannelStateWriter} implemented using
+ * {@link CheckpointStreamFactory.CheckpointStateOutputStream 
CheckpointStateOutputStreams}. Internally, it has
+ * 
+ * one stream per checkpoint; having multiple streams would mean more 
files written and more connections opened
+ * (and more latency on restore)
+ * one thread; having multiple threads means more connections, couples 
with the implementation and increases complexity
+ * 
+ */
+@Internal
+@ThreadSafe
+public class ChannelStateWriterImpl implements ChannelStateWriter {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriterImpl.class);
+   private static final int DEFAULT_HANDOVER_CAPACITY = 10;
+   private static final int DEFAULT_MAX_CHECKPOINTS = 5; // currently, 
only single in-flight checkpoint is supported
+
+   private class ProcessRequestsLoop implements Runnable {
+   private final ChannelStateWriteRequestProcessor 
requestProcessor;
+
+   private ProcessRequestsLoop(int maxCheckpoints, 
CheckpointStorageWorkerView streamFactoryFactory, ChannelStateSerializer 
serializer) {
+   this.requestProcessor = new 
ChannelStateWriteRequestProcessor(maxCheckpoints, streamFactoryFactory, 
serializer);
+   }
+
+   @Override
+   public void run() {
+   try {
+   loop();
+   } catch (Exception ex) {
+   thrown = ex;
+   } finally {
+   handover.clear();
+   requestProcessor.cleanup(thrown != null ? 
thrown : new RuntimeException("loop terminated, isRunning: " + isRunning));
+   }
+   LOG.debug("loop terminated");
+   }
+
+   private void loop() throws Exception {
+   while (isRunning || !handover.isEmpty()) {
+   try {
+   
requestProcessor.processRequest(handover.take());
+   } catch (InterruptedException e) {
+   if (isRunning || !handover.isEmpty()) {
+   LOG.info("interrupted while 
waiting for an item (continue waiting)", e);
+   } else {
+   
Thread.currentThread().interrupt();
+   return;
+   }
+   }
+   }
+   }
+   }
+
+   private final Thread asyncWriter;
+   private final BlockingQueue handover;
 
 Review comment:
   Thanks for sharing your

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-04 Thread GitBox
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_r403484530
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java
 ##
 @@ -175,6 +203,77 @@ public void checkpointState(
}
}
 
+   private OperatorSnapshotFutures buildOperatorSnapshotFutures(
+   CheckpointMetaData checkpointMetaData,
+   CheckpointOptions checkpointOptions,
+   OperatorChain operatorChain,
+   StreamOperator op,
+   Supplier isCanceled,
+   ChannelStateWriteResult channelStateWriteResult) throws 
Exception {
+   CheckpointStreamFactory storage = 
checkpointStorage.resolveCheckpointStorageLocation(
+   checkpointMetaData.getCheckpointId(),
+   checkpointOptions.getTargetLocation());
+   OperatorSnapshotFutures snapshotInProgress = 
checkpointStreamOperator(
+   op,
+   checkpointMetaData,
+   checkpointOptions,
+   storage,
+   isCanceled);
+   if (op == operatorChain.getHeadOperator()) {
+   
snapshotInProgress.setInputChannelStateFuture(channelStateWriteResult
+   
.getInputChannelStateHandles()
+   
.thenApply(StateObjectCollection::new)
+   
.thenApply(SnapshotResult::of)
+   );
+   }
+   if (op == operatorChain.getTailOperator()) {
+   
snapshotInProgress.setResultSubpartitionStateFuture(channelStateWriteResult
+   

.getResultSubpartitionStateHandles()
+   

.thenApply(StateObjectCollection::new)
+   

.thenApply(SnapshotResult::of)
+   );
+   }
+   return snapshotInProgress;
+   }
+
+   @Override
+   public void close() {
+   channelStateWriter.close();
+   }
+
+   // Caches checkpoint output stream factories to prevent multiple output 
stream per checkpoint.
 
 Review comment:
   I am not forcing that, it is up to you. Only saw that most of descriptions 
for class scope prefers to using above, and `//` is almost used in inline 
codes, `/** */` is for class fields. 


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-04 Thread GitBox
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_r403484530
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java
 ##
 @@ -175,6 +203,77 @@ public void checkpointState(
}
}
 
+   private OperatorSnapshotFutures buildOperatorSnapshotFutures(
+   CheckpointMetaData checkpointMetaData,
+   CheckpointOptions checkpointOptions,
+   OperatorChain operatorChain,
+   StreamOperator op,
+   Supplier isCanceled,
+   ChannelStateWriteResult channelStateWriteResult) throws 
Exception {
+   CheckpointStreamFactory storage = 
checkpointStorage.resolveCheckpointStorageLocation(
+   checkpointMetaData.getCheckpointId(),
+   checkpointOptions.getTargetLocation());
+   OperatorSnapshotFutures snapshotInProgress = 
checkpointStreamOperator(
+   op,
+   checkpointMetaData,
+   checkpointOptions,
+   storage,
+   isCanceled);
+   if (op == operatorChain.getHeadOperator()) {
+   
snapshotInProgress.setInputChannelStateFuture(channelStateWriteResult
+   
.getInputChannelStateHandles()
+   
.thenApply(StateObjectCollection::new)
+   
.thenApply(SnapshotResult::of)
+   );
+   }
+   if (op == operatorChain.getTailOperator()) {
+   
snapshotInProgress.setResultSubpartitionStateFuture(channelStateWriteResult
+   

.getResultSubpartitionStateHandles()
+   

.thenApply(StateObjectCollection::new)
+   

.thenApply(SnapshotResult::of)
+   );
+   }
+   return snapshotInProgress;
+   }
+
+   @Override
+   public void close() {
+   channelStateWriter.close();
+   }
+
+   // Caches checkpoint output stream factories to prevent multiple output 
stream per checkpoint.
 
 Review comment:
   I am not forcing that, it is up to you. Only saw that most of descriptions 
for class scope prefers to using above, and `//` is almost using inline codes, 
`/** */` is for class fields. 


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-04 Thread GitBox
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_r403484216
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateManagerImpl.java
 ##
 @@ -68,18 +70,36 @@
/** The checkpoint responder through which this manager can report to 
the job manager. */
private final CheckpointResponder checkpointResponder;
 
+   private final ChannelStateReader channelStateReader;
+
public TaskStateManagerImpl(
-   @Nonnull JobID jobId,
-   @Nonnull ExecutionAttemptID executionAttemptID,
-   @Nonnull TaskLocalStateStore localStateStore,
-   @Nullable JobManagerTaskRestore jobManagerTaskRestore,
-   @Nonnull CheckpointResponder checkpointResponder) {
+   @Nonnull JobID jobId,
+   @Nonnull ExecutionAttemptID executionAttemptID,
+   @Nonnull TaskLocalStateStore localStateStore,
+   @Nullable JobManagerTaskRestore jobManagerTaskRestore,
+   @Nonnull CheckpointResponder checkpointResponder) {
+   this(jobId,
+   executionAttemptID,
+   localStateStore,
+   jobManagerTaskRestore,
+   checkpointResponder,
+   new ChannelStateReaderImpl(jobManagerTaskRestore == 
null ? new TaskStateSnapshot() : jobManagerTaskRestore.getTaskStateSnapshot())
+   );
+   }
 
-   this.jobId = jobId;
+   TaskStateManagerImpl(
 
 Review comment:
   Yes, I agree with your point to some extent. I think the key concern is how 
we define the access modifier based on two considerations.
   
   - Based on current demands: if so, it should be private ATM, and then 
further extend it  by demands if necessary future.
   
   - Based on future considerations: if so, it can be defined as package public 
now, even public.  But it is hard to say whether it is alway fitting the 
expectation, then it might seem unnecessary for long time. If taking this 
option, we might even remove `@VisibleForTesting` annotation for previous 
usages, because any constructors might have the possibility to be used in core 
codes future.
   
   Anyway, I can accept both options and do not think it is a big issue. Just 
share some thoughts, feel free to take for your favor. :)


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-04 Thread GitBox
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_r403482247
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
 ##
 @@ -651,7 +651,7 @@ boolean isSerializingTimestamps() {
 * Gets the name of the task, in the form "taskname (2/5)".
 * @return The name of the task.
 */
-   public String getName() {
+   public final String getName() {
 
 Review comment:
   Yes, i agree with your consideration for `inconsistency between values used 
in constructor with ones in getters`.
   
   Regarding the `AbstractInvokable` and `StreamTask`, you misunderstood my 
previous comment. I mean that there might still have other methods inside 
parent `AbstractInvokable` and `StreamTask`,  which should not be overridden by 
subclasses, so it should also be defined as final as you already did in this 
PR. Now we only consider the requirements from 
`SubtaskCheckpointCoordinatorImpl` to handle the related methods. But from a 
more general motivation, we might need to go through all the necessary methods 
to lilmit them final to make this story complete. But it is up to you whether 
to touch more things in this PR. 


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-04 Thread GitBox
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_r403482247
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
 ##
 @@ -651,7 +651,7 @@ boolean isSerializingTimestamps() {
 * Gets the name of the task, in the form "taskname (2/5)".
 * @return The name of the task.
 */
-   public String getName() {
+   public final String getName() {
 
 Review comment:
   Yes, i agree with your consideration for `inconsistency between values used 
in constructor with ones in getters`.
   
   Regarding the `AbstractInvokable` and `StreamTask`, you misunderstood my 
previous comment. I mean that there might still have other methods inside 
parent `AbstractInvokable` and `StreamTask`,  which should not be overridden by 
subclasses, so it should also be defined as final as you already did in this 
PR. Now we only consider the requirements from 
`SubtaskCheckpointCoordinatorImpl` to handle the related methods. But from a 
more general motivation, we might need to go through all the necessary methods 
to limit them as final to make this story complete. But it is up to you whether 
to touch more things in this PR. 


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-04 Thread GitBox
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_r403482247
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
 ##
 @@ -651,7 +651,7 @@ boolean isSerializingTimestamps() {
 * Gets the name of the task, in the form "taskname (2/5)".
 * @return The name of the task.
 */
-   public String getName() {
+   public final String getName() {
 
 Review comment:
   Yes, i agree with your consideration for `inconsistency between values used 
in constructor with ones in getters`.
   
   Regarding the `AbstractInvokable` and `StreamTask`, you misunderstood my 
previous comment. I mean that there might still have other methods inside 
parent `AbstractInvokable` and `StreamTask`,  which should not be overridden by 
subclasses, so it should also be defined as final as you already did in this 
PR. Now we only consider the requirements from 
`SubtaskCheckpointCoordinatorImpl` to handle the related methods. But from a 
more general motivation, we might need to go through all the necessary methods 
to make them final to make this story complete. But it is up to you whether to 
touch more things in this PR. 


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402450729
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java
 ##
 @@ -0,0 +1,225 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.state.CheckpointStorageWorkerView;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeInput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeOutput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.write;
+
+/**
+ * {@link ChannelStateWriter} implemented using
+ * {@link CheckpointStreamFactory.CheckpointStateOutputStream 
CheckpointStateOutputStreams}. Internally, it has
+ * 
+ * one stream per checkpoint; having multiple streams would mean more 
files written and more connections opened
+ * (and more latency on restore)
+ * one thread; having multiple threads means more connections, couples 
with the implementation and increases complexity
+ * 
+ */
+@Internal
+@ThreadSafe
+public class ChannelStateWriterImpl implements ChannelStateWriter {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriterImpl.class);
+   private static final int DEFAULT_HANDOVER_CAPACITY = 10;
+   private static final int DEFAULT_MAX_CHECKPOINTS = 5; // currently, 
only single in-flight checkpoint is supported
+
+   private class ProcessRequestsLoop implements Runnable {
+   private final ChannelStateWriteRequestProcessor 
requestProcessor;
+
+   private ProcessRequestsLoop(int maxCheckpoints, 
CheckpointStorageWorkerView streamFactoryFactory, ChannelStateSerializer 
serializer) {
+   this.requestProcessor = new 
ChannelStateWriteRequestProcessor(maxCheckpoints, streamFactoryFactory, 
serializer);
+   }
+
+   @Override
+   public void run() {
+   try {
+   loop();
+   } catch (Exception ex) {
+   thrown = ex;
+   } finally {
+   handover.clear();
+   requestProcessor.cleanup(thrown != null ? 
thrown : new RuntimeException("loop terminated, isRunning: " + isRunning));
+   }
+   LOG.debug("loop terminated");
+   }
+
+   private void loop() throws Exception {
+   while (isRunning || !handover.isEmpty()) {
+   try {
+   
requestProcessor.processRequest(handover.take());
+   } catch (InterruptedException e) {
+   if (isRunning || !handover.isEmpty()) {
+   LOG.info("interrupted while 
waiting for an item (continue waiting)", e);
+   } else {
+   
Thread.currentThread().interrupt();
+   return;
+   }
+   }
+   }
+   }
+   }
+
+   private final Thread asyncWriter;
+   private final BlockingQueue handover;
+   private final Map results;
+   pr

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402442003
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/state/TestTaskStateManager.java
 ##
 @@ -94,7 +95,6 @@ public void reportTaskStateSnapshots(
@Nullable TaskStateSnapshot acknowledgedState,
@Nullable TaskStateSnapshot localState) {
 
-
 
 Review comment:
   To be strict, any unrelated formatting change should be a separate `hotfix` 
commit, not need a separate PR, can commit separately in this PR.
   
   Because I was also told the same issue multiple times by other committers 
when submitting PR in the early period. So I just mention it, it is not 
mandatory from my side. :)


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402437475
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
 ##
 @@ -914,6 +914,11 @@ private void closeNetworkResources() {
LOG.error("Failed to release input gate for 
task {}.", taskNameWithSubtask, t);
}
}
+   try {
+   taskStateManager.close();
 
 Review comment:
   The previous `closeNetworkResources` can be used in two scenarios, one is 
for task exit in finally region, another is used by canceler thread before task 
exiting. The motivation to close network resource by canceler thread is to 
release the buffers ASAP, then the task thread can cause exception while 
interacting with buffer to make it exit early.
   
   If the close of `taskStateManager` does not have the effect to make task 
thread exit early, then it should not be called by canceler thread, and it only 
needs to be called in task thread finally region.


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402434184
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateManagerImpl.java
 ##
 @@ -68,18 +70,36 @@
/** The checkpoint responder through which this manager can report to 
the job manager. */
private final CheckpointResponder checkpointResponder;
 
+   private final ChannelStateReader channelStateReader;
+
public TaskStateManagerImpl(
-   @Nonnull JobID jobId,
-   @Nonnull ExecutionAttemptID executionAttemptID,
-   @Nonnull TaskLocalStateStore localStateStore,
-   @Nullable JobManagerTaskRestore jobManagerTaskRestore,
-   @Nonnull CheckpointResponder checkpointResponder) {
+   @Nonnull JobID jobId,
+   @Nonnull ExecutionAttemptID executionAttemptID,
+   @Nonnull TaskLocalStateStore localStateStore,
+   @Nullable JobManagerTaskRestore jobManagerTaskRestore,
+   @Nonnull CheckpointResponder checkpointResponder) {
+   this(jobId,
+   executionAttemptID,
+   localStateStore,
+   jobManagerTaskRestore,
+   checkpointResponder,
+   new ChannelStateReaderImpl(jobManagerTaskRestore == 
null ? new TaskStateSnapshot() : jobManagerTaskRestore.getTaskStateSnapshot())
+   );
+   }
 
-   this.jobId = jobId;
+   TaskStateManagerImpl(
 
 Review comment:
   @VisibleForTesting


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402430996
 
 

 ##
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
 ##
 @@ -335,7 +335,7 @@ public void acknowledgeCheckpoint(long checkpointId, 
CheckpointMetrics checkpoin
 
@Override
public void declineCheckpoint(long checkpointId, Throwable cause) {
-   throw new UnsupportedOperationException();
+   throw new UnsupportedOperationException(cause);
 
 Review comment:
   Based on my experience, any unrelated tiny changes should be submitted as 
`hotfix` commit instead, such as typo, indentation formatting, etc. 
   But I guess there are no explicit guidelines for it, so feel free to do it 
or not.


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402425665
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferBuilder.java
 ##
 @@ -117,6 +117,11 @@ public boolean isFull() {
return positionMarker.getCached() == getMaxCapacity();
}
 
+   public int writableBytes() {
 
 Review comment:
   nit: getWritableBytes()?


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402424172
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java
 ##
 @@ -0,0 +1,225 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.state.CheckpointStorageWorkerView;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeInput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeOutput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.write;
+
+/**
+ * {@link ChannelStateWriter} implemented using
+ * {@link CheckpointStreamFactory.CheckpointStateOutputStream 
CheckpointStateOutputStreams}. Internally, it has
+ * 
+ * one stream per checkpoint; having multiple streams would mean more 
files written and more connections opened
+ * (and more latency on restore)
+ * one thread; having multiple threads means more connections, couples 
with the implementation and increases complexity
+ * 
+ */
+@Internal
+@ThreadSafe
+public class ChannelStateWriterImpl implements ChannelStateWriter {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriterImpl.class);
+   private static final int DEFAULT_HANDOVER_CAPACITY = 10;
+   private static final int DEFAULT_MAX_CHECKPOINTS = 5; // currently, 
only single in-flight checkpoint is supported
+
+   private class ProcessRequestsLoop implements Runnable {
+   private final ChannelStateWriteRequestProcessor 
requestProcessor;
+
+   private ProcessRequestsLoop(int maxCheckpoints, 
CheckpointStorageWorkerView streamFactoryFactory, ChannelStateSerializer 
serializer) {
+   this.requestProcessor = new 
ChannelStateWriteRequestProcessor(maxCheckpoints, streamFactoryFactory, 
serializer);
+   }
+
+   @Override
+   public void run() {
+   try {
+   loop();
+   } catch (Exception ex) {
+   thrown = ex;
+   } finally {
+   handover.clear();
+   requestProcessor.cleanup(thrown != null ? 
thrown : new RuntimeException("loop terminated, isRunning: " + isRunning));
+   }
+   LOG.debug("loop terminated");
+   }
+
+   private void loop() throws Exception {
+   while (isRunning || !handover.isEmpty()) {
+   try {
+   
requestProcessor.processRequest(handover.take());
+   } catch (InterruptedException e) {
+   if (isRunning || !handover.isEmpty()) {
+   LOG.info("interrupted while 
waiting for an item (continue waiting)", e);
+   } else {
+   
Thread.currentThread().interrupt();
+   return;
+   }
+   }
+   }
+   }
+   }
+
+   private final Thread asyncWriter;
+   private final BlockingQueue handover;
+   private final Map results;
+   pr

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402422467
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java
 ##
 @@ -0,0 +1,225 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.state.CheckpointStorageWorkerView;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeInput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeOutput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.write;
+
+/**
+ * {@link ChannelStateWriter} implemented using
+ * {@link CheckpointStreamFactory.CheckpointStateOutputStream 
CheckpointStateOutputStreams}. Internally, it has
+ * 
+ * one stream per checkpoint; having multiple streams would mean more 
files written and more connections opened
+ * (and more latency on restore)
+ * one thread; having multiple threads means more connections, couples 
with the implementation and increases complexity
+ * 
+ */
+@Internal
+@ThreadSafe
+public class ChannelStateWriterImpl implements ChannelStateWriter {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriterImpl.class);
+   private static final int DEFAULT_HANDOVER_CAPACITY = 10;
+   private static final int DEFAULT_MAX_CHECKPOINTS = 5; // currently, 
only single in-flight checkpoint is supported
+
+   private class ProcessRequestsLoop implements Runnable {
+   private final ChannelStateWriteRequestProcessor 
requestProcessor;
+
+   private ProcessRequestsLoop(int maxCheckpoints, 
CheckpointStorageWorkerView streamFactoryFactory, ChannelStateSerializer 
serializer) {
+   this.requestProcessor = new 
ChannelStateWriteRequestProcessor(maxCheckpoints, streamFactoryFactory, 
serializer);
+   }
+
+   @Override
+   public void run() {
+   try {
+   loop();
+   } catch (Exception ex) {
+   thrown = ex;
+   } finally {
+   handover.clear();
+   requestProcessor.cleanup(thrown != null ? 
thrown : new RuntimeException("loop terminated, isRunning: " + isRunning));
+   }
+   LOG.debug("loop terminated");
+   }
+
+   private void loop() throws Exception {
+   while (isRunning || !handover.isEmpty()) {
+   try {
+   
requestProcessor.processRequest(handover.take());
+   } catch (InterruptedException e) {
+   if (isRunning || !handover.isEmpty()) {
+   LOG.info("interrupted while 
waiting for an item (continue waiting)", e);
+   } else {
+   
Thread.currentThread().interrupt();
+   return;
+   }
+   }
+   }
+   }
+   }
+
+   private final Thread asyncWriter;
+   private final BlockingQueue handover;
+   private final Map results;
+   pr

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402421283
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java
 ##
 @@ -0,0 +1,225 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.state.CheckpointStorageWorkerView;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeInput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeOutput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.write;
+
+/**
+ * {@link ChannelStateWriter} implemented using
+ * {@link CheckpointStreamFactory.CheckpointStateOutputStream 
CheckpointStateOutputStreams}. Internally, it has
+ * 
+ * one stream per checkpoint; having multiple streams would mean more 
files written and more connections opened
+ * (and more latency on restore)
+ * one thread; having multiple threads means more connections, couples 
with the implementation and increases complexity
+ * 
+ */
+@Internal
+@ThreadSafe
+public class ChannelStateWriterImpl implements ChannelStateWriter {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriterImpl.class);
+   private static final int DEFAULT_HANDOVER_CAPACITY = 10;
+   private static final int DEFAULT_MAX_CHECKPOINTS = 5; // currently, 
only single in-flight checkpoint is supported
+
+   private class ProcessRequestsLoop implements Runnable {
+   private final ChannelStateWriteRequestProcessor 
requestProcessor;
+
+   private ProcessRequestsLoop(int maxCheckpoints, 
CheckpointStorageWorkerView streamFactoryFactory, ChannelStateSerializer 
serializer) {
+   this.requestProcessor = new 
ChannelStateWriteRequestProcessor(maxCheckpoints, streamFactoryFactory, 
serializer);
+   }
+
+   @Override
+   public void run() {
+   try {
+   loop();
+   } catch (Exception ex) {
+   thrown = ex;
+   } finally {
+   handover.clear();
+   requestProcessor.cleanup(thrown != null ? 
thrown : new RuntimeException("loop terminated, isRunning: " + isRunning));
+   }
+   LOG.debug("loop terminated");
+   }
+
+   private void loop() throws Exception {
+   while (isRunning || !handover.isEmpty()) {
+   try {
+   
requestProcessor.processRequest(handover.take());
+   } catch (InterruptedException e) {
+   if (isRunning || !handover.isEmpty()) {
+   LOG.info("interrupted while 
waiting for an item (continue waiting)", e);
+   } else {
+   
Thread.currentThread().interrupt();
+   return;
+   }
+   }
+   }
+   }
+   }
+
+   private final Thread asyncWriter;
+   private final BlockingQueue handover;
+   private final Map results;
+   pr

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402420317
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java
 ##
 @@ -0,0 +1,225 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.state.CheckpointStorageWorkerView;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeInput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeOutput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.write;
+
+/**
+ * {@link ChannelStateWriter} implemented using
+ * {@link CheckpointStreamFactory.CheckpointStateOutputStream 
CheckpointStateOutputStreams}. Internally, it has
+ * 
+ * one stream per checkpoint; having multiple streams would mean more 
files written and more connections opened
+ * (and more latency on restore)
+ * one thread; having multiple threads means more connections, couples 
with the implementation and increases complexity
+ * 
+ */
+@Internal
+@ThreadSafe
+public class ChannelStateWriterImpl implements ChannelStateWriter {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriterImpl.class);
+   private static final int DEFAULT_HANDOVER_CAPACITY = 10;
+   private static final int DEFAULT_MAX_CHECKPOINTS = 5; // currently, 
only single in-flight checkpoint is supported
+
+   private class ProcessRequestsLoop implements Runnable {
 
 Review comment:
   Not mandatory. In general I found many internal class were placed at the 
bottom. For me, this class is placed among the class fields to break them down, 
then it is not convenient to directly overview all the class fields. 


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402417315
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriter.java
 ##
 @@ -46,60 +78,64 @@
/**
 * Initiate write of channel state for the given checkpoint id.
 */
-   void start(long checkpointId);
+   void start(long checkpointId, CheckpointOptions checkpointOptions);
 
/**
 * Add in-flight buffers from the {@link 
org.apache.flink.runtime.io.network.partition.consumer.InputChannel 
InputChannel}.
-* Must be called after {@link #start(long)} and before {@link 
#finish(long)}.
+* Must be called after {@link #start} (long)} and before {@link 
#finishInput(long)}.
+* Buffers are recycled after they are written.
 * @param startSeqNum sequence number of the 1st passed buffer.
 *It is intended to use for incremental snapshots.
 *If no data is passed it is ignored.
-* @param data zero or more buffers ordered by their sequence numbers
+* @param data zero or more data buffers ordered by their 
sequence numbers
+* @throws IllegalArgumentException if one or more passed buffers 
{@link Buffer#isBuffer()  isn't a buffer}
 * @see 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_RESTORED
 * @see 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_UNKNOWN
 */
-   void addInputData(long checkpointId, InputChannelInfo info, int 
startSeqNum, Buffer... data);
+   void addInputData(long checkpointId, InputChannelInfo info, int 
startSeqNum, Buffer... data) throws IllegalArgumentException;
 
/**
 * Add in-flight buffers from the {@link 
org.apache.flink.runtime.io.network.partition.ResultSubpartition 
ResultSubpartition}.
-* Must be called after {@link #start(long)} and before {@link 
#finish(long)}.
+* Must be called after {@link #start} and before {@link 
#finishOutput(long)}.
+* Buffers are recycled after they are written.
 * @param startSeqNum sequence number of the 1st passed buffer.
 *It is intended to use for incremental snapshots.
 *If no data is passed it is ignored.
-* @param data zero or more buffers ordered by their sequence numbers
+* @param data zero or more data buffers ordered by their 
sequence numbers
+* @throws IllegalArgumentException if one or more passed buffers 
{@link Buffer#isBuffer()  isn't a buffer}
 * @see 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_RESTORED
 * @see 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_UNKNOWN
 */
-   void addOutputData(long checkpointId, ResultSubpartitionInfo info, int 
startSeqNum, Buffer... data);
+   void addOutputData(long checkpointId, ResultSubpartitionInfo info, int 
startSeqNum, Buffer... data) throws IllegalArgumentException;
 
/**
 * Finalize write of channel state data for the given checkpoint id.
 * Must be called after {@link #start(long)} and all of the input data 
of the given checkpoint added.
 * When both {@link #finishInput} and {@link #finishOutput} were called 
the results can be (eventually) obtained
-* using {@link #getWriteCompletionFuture}
+* using {@link #getWriteResult}
 */
void finishInput(long checkpointId);
 
/**
 * Finalize write of channel state data for the given checkpoint id.
 * Must be called after {@link #start(long)} and all of the output data 
of the given checkpoint added.
 * When both {@link #finishInput} and {@link #finishOutput} were called 
the results can be (eventually) obtained
-* using {@link #getWriteCompletionFuture}
+* using {@link #getWriteResult}
 */
void finishOutput(long checkpointId);
 
/**
-* Must be called after {@link #start(long)}.
+* Must be called after {@link #start}.
 */
-   Future> getWriteCompletionFuture(long 
checkpointId);
+   ChannelStateWriteResult getWriteResult(long checkpointId);
 
@Override
-   void close() throws Exception;
+   void close();
 
 Review comment:
   remove this method directly?


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402416768
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriter.java
 ##
 @@ -46,60 +78,64 @@
/**
 * Initiate write of channel state for the given checkpoint id.
 */
-   void start(long checkpointId);
+   void start(long checkpointId, CheckpointOptions checkpointOptions);
 
/**
 * Add in-flight buffers from the {@link 
org.apache.flink.runtime.io.network.partition.consumer.InputChannel 
InputChannel}.
-* Must be called after {@link #start(long)} and before {@link 
#finish(long)}.
+* Must be called after {@link #start} (long)} and before {@link 
#finishInput(long)}.
+* Buffers are recycled after they are written.
 * @param startSeqNum sequence number of the 1st passed buffer.
 *It is intended to use for incremental snapshots.
 *If no data is passed it is ignored.
-* @param data zero or more buffers ordered by their sequence numbers
+* @param data zero or more data buffers ordered by their 
sequence numbers
+* @throws IllegalArgumentException if one or more passed buffers 
{@link Buffer#isBuffer()  isn't a buffer}
 * @see 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_RESTORED
 * @see 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_UNKNOWN
 */
-   void addInputData(long checkpointId, InputChannelInfo info, int 
startSeqNum, Buffer... data);
+   void addInputData(long checkpointId, InputChannelInfo info, int 
startSeqNum, Buffer... data) throws IllegalArgumentException;
 
/**
 * Add in-flight buffers from the {@link 
org.apache.flink.runtime.io.network.partition.ResultSubpartition 
ResultSubpartition}.
-* Must be called after {@link #start(long)} and before {@link 
#finish(long)}.
+* Must be called after {@link #start} and before {@link 
#finishOutput(long)}.
+* Buffers are recycled after they are written.
 * @param startSeqNum sequence number of the 1st passed buffer.
 *It is intended to use for incremental snapshots.
 *If no data is passed it is ignored.
-* @param data zero or more buffers ordered by their sequence numbers
+* @param data zero or more data buffers ordered by their 
sequence numbers
+* @throws IllegalArgumentException if one or more passed buffers 
{@link Buffer#isBuffer()  isn't a buffer}
 * @see 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_RESTORED
 * @see 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_UNKNOWN
 */
-   void addOutputData(long checkpointId, ResultSubpartitionInfo info, int 
startSeqNum, Buffer... data);
+   void addOutputData(long checkpointId, ResultSubpartitionInfo info, int 
startSeqNum, Buffer... data) throws IllegalArgumentException;
 
/**
 * Finalize write of channel state data for the given checkpoint id.
 * Must be called after {@link #start(long)} and all of the input data 
of the given checkpoint added.
 
 Review comment:
   #start missing one argument


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402410595
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateSerializer.java
 ##
 @@ -0,0 +1,199 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import static java.lang.Math.addExact;
+import static java.lang.Math.min;
+
+interface ChannelStateSerializer {
+
+   void writeHeader(DataOutputStream dataStream) throws IOException;
+
+   void writeData(DataOutputStream stream, Buffer... flinkBuffers) throws 
IOException;
+}
+
+interface ChannelStateDeserializer {
+
+   void readHeader(InputStream stream) throws IOException;
+
+   /**
+* Reads the length of state.
+*
+* @return size of state in bytes and offset of the current position 
resulted from reading.
+*/
+   Tuple2 readLength(InputStream stream) throws 
IOException;
+
+   int readData(InputStream stream, ChannelStateByteBuffer buffer, int 
bytes) throws IOException;
+}
+
+/**
+ * Wrapper around various buffers to receive channel state data.
+ */
+@Internal
+@NotThreadSafe
+interface ChannelStateByteBuffer {
+
+   boolean isWritable();
+
+   int writableBytes();
+
+   int writeBytes(InputStream input, int bytesToRead) throws IOException;
+
+   static ChannelStateByteBuffer wrap(Buffer buffer) {
+   return new ChannelStateByteBuffer() {
+
+   private ByteBuf byteBuf = buffer.asByteBuf();
+
+   @Override
+   public boolean isWritable() {
+   return byteBuf.isWritable();
+   }
+
+   @Override
+   public int writableBytes() {
+   return byteBuf.writableBytes();
+   }
+
+   @Override
+   public int writeBytes(InputStream input, int 
bytesToRead) throws IOException {
+   return byteBuf.writeBytes(input, 
Math.min(bytesToRead, byteBuf.writableBytes()));
+   }
+   };
+   }
+
+   static ChannelStateByteBuffer wrap(BufferBuilder bufferBuilder) {
+   final byte[] buf = new byte[1024];
 
 Review comment:
   TBH I am concerning of creating the temporary byte array for every buffer 
level, it might be not friendly for GC. And it also brings additional copy 
while reading. But i have not thought of a better option now. Maybe at-least to 
reuse the same `buf` for every wrap?


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402413513
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriter.java
 ##
 @@ -46,60 +78,64 @@
/**
 * Initiate write of channel state for the given checkpoint id.
 */
-   void start(long checkpointId);
+   void start(long checkpointId, CheckpointOptions checkpointOptions);
 
/**
 * Add in-flight buffers from the {@link 
org.apache.flink.runtime.io.network.partition.consumer.InputChannel 
InputChannel}.
-* Must be called after {@link #start(long)} and before {@link 
#finish(long)}.
+* Must be called after {@link #start} (long)} and before {@link 
#finishInput(long)}.
+* Buffers are recycled after they are written.
 * @param startSeqNum sequence number of the 1st passed buffer.
 *It is intended to use for incremental snapshots.
 *If no data is passed it is ignored.
-* @param data zero or more buffers ordered by their sequence numbers
+* @param data zero or more data buffers ordered by their 
sequence numbers
+* @throws IllegalArgumentException if one or more passed buffers 
{@link Buffer#isBuffer()  isn't a buffer}
 * @see 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_RESTORED
 * @see 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_UNKNOWN
 */
-   void addInputData(long checkpointId, InputChannelInfo info, int 
startSeqNum, Buffer... data);
+   void addInputData(long checkpointId, InputChannelInfo info, int 
startSeqNum, Buffer... data) throws IllegalArgumentException;
 
 Review comment:
   nit: seems no need to throw this exception explicitly, because the subclass 
implementation actually does not throw such exception.


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402413827
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriter.java
 ##
 @@ -46,60 +78,64 @@
/**
 * Initiate write of channel state for the given checkpoint id.
 */
-   void start(long checkpointId);
+   void start(long checkpointId, CheckpointOptions checkpointOptions);
 
/**
 * Add in-flight buffers from the {@link 
org.apache.flink.runtime.io.network.partition.consumer.InputChannel 
InputChannel}.
-* Must be called after {@link #start(long)} and before {@link 
#finish(long)}.
+* Must be called after {@link #start} (long)} and before {@link 
#finishInput(long)}.
+* Buffers are recycled after they are written.
 * @param startSeqNum sequence number of the 1st passed buffer.
 *It is intended to use for incremental snapshots.
 *If no data is passed it is ignored.
-* @param data zero or more buffers ordered by their sequence numbers
+* @param data zero or more data buffers ordered by their 
sequence numbers
+* @throws IllegalArgumentException if one or more passed buffers 
{@link Buffer#isBuffer()  isn't a buffer}
 * @see 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_RESTORED
 * @see 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_UNKNOWN
 */
-   void addInputData(long checkpointId, InputChannelInfo info, int 
startSeqNum, Buffer... data);
+   void addInputData(long checkpointId, InputChannelInfo info, int 
startSeqNum, Buffer... data) throws IllegalArgumentException;
 
/**
 * Add in-flight buffers from the {@link 
org.apache.flink.runtime.io.network.partition.ResultSubpartition 
ResultSubpartition}.
-* Must be called after {@link #start(long)} and before {@link 
#finish(long)}.
+* Must be called after {@link #start} and before {@link 
#finishOutput(long)}.
+* Buffers are recycled after they are written.
 * @param startSeqNum sequence number of the 1st passed buffer.
 *It is intended to use for incremental snapshots.
 *If no data is passed it is ignored.
-* @param data zero or more buffers ordered by their sequence numbers
+* @param data zero or more data buffers ordered by their 
sequence numbers
+* @throws IllegalArgumentException if one or more passed buffers 
{@link Buffer#isBuffer()  isn't a buffer}
 * @see 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_RESTORED
 * @see 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter#SEQUENCE_NUMBER_UNKNOWN
 */
-   void addOutputData(long checkpointId, ResultSubpartitionInfo info, int 
startSeqNum, Buffer... data);
+   void addOutputData(long checkpointId, ResultSubpartitionInfo info, int 
startSeqNum, Buffer... data) throws IllegalArgumentException;
 
 Review comment:
   ditto: no need to throw explicitly


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402410595
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateSerializer.java
 ##
 @@ -0,0 +1,199 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import static java.lang.Math.addExact;
+import static java.lang.Math.min;
+
+interface ChannelStateSerializer {
+
+   void writeHeader(DataOutputStream dataStream) throws IOException;
+
+   void writeData(DataOutputStream stream, Buffer... flinkBuffers) throws 
IOException;
+}
+
+interface ChannelStateDeserializer {
+
+   void readHeader(InputStream stream) throws IOException;
+
+   /**
+* Reads the length of state.
+*
+* @return size of state in bytes and offset of the current position 
resulted from reading.
+*/
+   Tuple2 readLength(InputStream stream) throws 
IOException;
+
+   int readData(InputStream stream, ChannelStateByteBuffer buffer, int 
bytes) throws IOException;
+}
+
+/**
+ * Wrapper around various buffers to receive channel state data.
+ */
+@Internal
+@NotThreadSafe
+interface ChannelStateByteBuffer {
+
+   boolean isWritable();
+
+   int writableBytes();
+
+   int writeBytes(InputStream input, int bytesToRead) throws IOException;
+
+   static ChannelStateByteBuffer wrap(Buffer buffer) {
+   return new ChannelStateByteBuffer() {
+
+   private ByteBuf byteBuf = buffer.asByteBuf();
+
+   @Override
+   public boolean isWritable() {
+   return byteBuf.isWritable();
+   }
+
+   @Override
+   public int writableBytes() {
+   return byteBuf.writableBytes();
+   }
+
+   @Override
+   public int writeBytes(InputStream input, int 
bytesToRead) throws IOException {
+   return byteBuf.writeBytes(input, 
Math.min(bytesToRead, byteBuf.writableBytes()));
+   }
+   };
+   }
+
+   static ChannelStateByteBuffer wrap(BufferBuilder bufferBuilder) {
+   final byte[] buf = new byte[1024];
 
 Review comment:
   TBH I am concerning of creating the temporary byte array for every buffer 
level, it might be not friendly for GC. And it also brings additional copy 
while reading. But i have not thought of a better option now. :(


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402407219
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateSerializer.java
 ##
 @@ -0,0 +1,199 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import static java.lang.Math.addExact;
+import static java.lang.Math.min;
+
+interface ChannelStateSerializer {
+
+   void writeHeader(DataOutputStream dataStream) throws IOException;
+
+   void writeData(DataOutputStream stream, Buffer... flinkBuffers) throws 
IOException;
+}
+
+interface ChannelStateDeserializer {
+
+   void readHeader(InputStream stream) throws IOException;
+
+   /**
+* Reads the length of state.
+*
+* @return size of state in bytes and offset of the current position 
resulted from reading.
+*/
+   Tuple2 readLength(InputStream stream) throws 
IOException;
+
+   int readData(InputStream stream, ChannelStateByteBuffer buffer, int 
bytes) throws IOException;
+}
+
+/**
+ * Wrapper around various buffers to receive channel state data.
+ */
+@Internal
+@NotThreadSafe
+interface ChannelStateByteBuffer {
+
+   boolean isWritable();
+
+   int writableBytes();
+
+   int writeBytes(InputStream input, int bytesToRead) throws IOException;
+
+   static ChannelStateByteBuffer wrap(Buffer buffer) {
+   return new ChannelStateByteBuffer() {
+
+   private ByteBuf byteBuf = buffer.asByteBuf();
+
+   @Override
+   public boolean isWritable() {
+   return byteBuf.isWritable();
+   }
+
+   @Override
+   public int writableBytes() {
+   return byteBuf.writableBytes();
+   }
+
+   @Override
+   public int writeBytes(InputStream input, int 
bytesToRead) throws IOException {
+   return byteBuf.writeBytes(input, 
Math.min(bytesToRead, byteBuf.writableBytes()));
+   }
+   };
+   }
+
+   static ChannelStateByteBuffer wrap(BufferBuilder bufferBuilder) {
+   final byte[] buf = new byte[1024];
+   return new ChannelStateByteBuffer() {
+   @Override
+   public boolean isWritable() {
+   return !bufferBuilder.isFull();
+   }
+
+   @Override
+   public int writableBytes() {
+   return bufferBuilder.writableBytes();
+   }
+
+   @Override
+   public int writeBytes(InputStream input, int upToBytes) 
throws IOException {
+   int left = upToBytes;
+   for (int toRead = getToRead(left); toRead > 0; 
toRead = getToRead(left)) {
+   int read = input.read(buf, 0, toRead);
+   int copied = 
bufferBuilder.append(java.nio.ByteBuffer.wrap(buf, 0, read));
+   Preconditions.checkState(copied == 
read);
+   left -= read;
+   }
+   bufferBuilder.commit();
+   return upToBytes - left;
+   }
+
+  

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402402384
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateSerializer.java
 ##
 @@ -0,0 +1,199 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import static java.lang.Math.addExact;
+import static java.lang.Math.min;
+
+interface ChannelStateSerializer {
+
+   void writeHeader(DataOutputStream dataStream) throws IOException;
+
+   void writeData(DataOutputStream stream, Buffer... flinkBuffers) throws 
IOException;
+}
+
+interface ChannelStateDeserializer {
+
+   void readHeader(InputStream stream) throws IOException;
+
+   /**
+* Reads the length of state.
+*
+* @return size of state in bytes and offset of the current position 
resulted from reading.
+*/
+   Tuple2 readLength(InputStream stream) throws 
IOException;
+
+   int readData(InputStream stream, ChannelStateByteBuffer buffer, int 
bytes) throws IOException;
+}
+
+/**
+ * Wrapper around various buffers to receive channel state data.
+ */
+@Internal
+@NotThreadSafe
+interface ChannelStateByteBuffer {
+
+   boolean isWritable();
+
+   int writableBytes();
+
+   int writeBytes(InputStream input, int bytesToRead) throws IOException;
 
 Review comment:
   better to give javadoc for this method, then it is easy to understand the 
arguments especially for the meaning of the return value.


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r40240
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateSerializer.java
 ##
 @@ -0,0 +1,199 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import static java.lang.Math.addExact;
+import static java.lang.Math.min;
+
+interface ChannelStateSerializer {
+
+   void writeHeader(DataOutputStream dataStream) throws IOException;
+
+   void writeData(DataOutputStream stream, Buffer... flinkBuffers) throws 
IOException;
+}
+
+interface ChannelStateDeserializer {
+
+   void readHeader(InputStream stream) throws IOException;
+
+   /**
+* Reads the length of state.
+*
+* @return size of state in bytes and offset of the current position 
resulted from reading.
+*/
+   Tuple2 readLength(InputStream stream) throws 
IOException;
+
+   int readData(InputStream stream, ChannelStateByteBuffer buffer, int 
bytes) throws IOException;
+}
+
+/**
+ * Wrapper around various buffers to receive channel state data.
+ */
+@Internal
+@NotThreadSafe
+interface ChannelStateByteBuffer {
+
+   boolean isWritable();
+
+   int writableBytes();
+
+   int writeBytes(InputStream input, int bytesToRead) throws IOException;
+
+   static ChannelStateByteBuffer wrap(Buffer buffer) {
+   return new ChannelStateByteBuffer() {
+
+   private ByteBuf byteBuf = buffer.asByteBuf();
+
+   @Override
+   public boolean isWritable() {
+   return byteBuf.isWritable();
+   }
+
+   @Override
+   public int writableBytes() {
+   return byteBuf.writableBytes();
+   }
+
+   @Override
+   public int writeBytes(InputStream input, int 
bytesToRead) throws IOException {
+   return byteBuf.writeBytes(input, 
Math.min(bytesToRead, byteBuf.writableBytes()));
+   }
+   };
+   }
+
+   static ChannelStateByteBuffer wrap(BufferBuilder bufferBuilder) {
+   final byte[] buf = new byte[1024];
+   return new ChannelStateByteBuffer() {
+   @Override
+   public boolean isWritable() {
+   return !bufferBuilder.isFull();
+   }
+
+   @Override
+   public int writableBytes() {
+   return bufferBuilder.writableBytes();
+   }
+
+   @Override
+   public int writeBytes(InputStream input, int upToBytes) 
throws IOException {
+   int left = upToBytes;
+   for (int toRead = getToRead(left); toRead > 0; 
toRead = getToRead(left)) {
+   int read = input.read(buf, 0, toRead);
+   int copied = 
bufferBuilder.append(java.nio.ByteBuffer.wrap(buf, 0, read));
+   Preconditions.checkState(copied == 
read);
+   left -= read;
+   }
+   bufferBuilder.commit();
+   return upToBytes - left;
+   }
+
+  

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402400824
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateSerializer.java
 ##
 @@ -0,0 +1,199 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import static java.lang.Math.addExact;
+import static java.lang.Math.min;
+
+interface ChannelStateSerializer {
+
+   void writeHeader(DataOutputStream dataStream) throws IOException;
+
+   void writeData(DataOutputStream stream, Buffer... flinkBuffers) throws 
IOException;
+}
+
+interface ChannelStateDeserializer {
+
+   void readHeader(InputStream stream) throws IOException;
+
+   /**
+* Reads the length of state.
+*
+* @return size of state in bytes and offset of the current position 
resulted from reading.
+*/
+   Tuple2 readLength(InputStream stream) throws 
IOException;
+
+   int readData(InputStream stream, ChannelStateByteBuffer buffer, int 
bytes) throws IOException;
+}
+
+/**
+ * Wrapper around various buffers to receive channel state data.
+ */
+@Internal
+@NotThreadSafe
+interface ChannelStateByteBuffer {
+
+   boolean isWritable();
+
+   int writableBytes();
 
 Review comment:
   I guess we can remove this explicit method actually, because it is only used 
inside `#wrap() method and would not be called by outsides.  See 
https://github.com/apache/flink/pull/11515/commits/dad865f157135abd13585f9b560af76cac07b127#r402400614


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402400614
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateSerializer.java
 ##
 @@ -0,0 +1,199 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import static java.lang.Math.addExact;
+import static java.lang.Math.min;
+
+interface ChannelStateSerializer {
+
+   void writeHeader(DataOutputStream dataStream) throws IOException;
+
+   void writeData(DataOutputStream stream, Buffer... flinkBuffers) throws 
IOException;
+}
+
+interface ChannelStateDeserializer {
+
+   void readHeader(InputStream stream) throws IOException;
+
+   /**
+* Reads the length of state.
+*
+* @return size of state in bytes and offset of the current position 
resulted from reading.
+*/
+   Tuple2 readLength(InputStream stream) throws 
IOException;
+
+   int readData(InputStream stream, ChannelStateByteBuffer buffer, int 
bytes) throws IOException;
+}
+
+/**
+ * Wrapper around various buffers to receive channel state data.
+ */
+@Internal
+@NotThreadSafe
+interface ChannelStateByteBuffer {
+
+   boolean isWritable();
+
+   int writableBytes();
+
+   int writeBytes(InputStream input, int bytesToRead) throws IOException;
+
+   static ChannelStateByteBuffer wrap(Buffer buffer) {
+   return new ChannelStateByteBuffer() {
+
+   private ByteBuf byteBuf = buffer.asByteBuf();
+
+   @Override
+   public boolean isWritable() {
+   return byteBuf.isWritable();
+   }
+
+   @Override
+   public int writableBytes() {
+   return byteBuf.writableBytes();
+   }
+
+   @Override
+   public int writeBytes(InputStream input, int 
bytesToRead) throws IOException {
+   return byteBuf.writeBytes(input, 
Math.min(bytesToRead, byteBuf.writableBytes()));
+   }
+   };
+   }
+
+   static ChannelStateByteBuffer wrap(BufferBuilder bufferBuilder) {
+   final byte[] buf = new byte[1024];
+   return new ChannelStateByteBuffer() {
+   @Override
+   public boolean isWritable() {
+   return !bufferBuilder.isFull();
+   }
+
+   @Override
+   public int writableBytes() {
+   return bufferBuilder.writableBytes();
+   }
+
+   @Override
+   public int writeBytes(InputStream input, int upToBytes) 
throws IOException {
+   int left = upToBytes;
+   for (int toRead = getToRead(left); toRead > 0; 
toRead = getToRead(left)) {
+   int read = input.read(buf, 0, toRead);
+   int copied = 
bufferBuilder.append(java.nio.ByteBuffer.wrap(buf, 0, read));
+   Preconditions.checkState(copied == 
read);
+   left -= read;
+   }
+   bufferBuilder.commit();
+   return upToBytes - left;
+   }
+
+  

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402389785
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateSerializer.java
 ##
 @@ -0,0 +1,199 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import static java.lang.Math.addExact;
+import static java.lang.Math.min;
+
+interface ChannelStateSerializer {
+
+   void writeHeader(DataOutputStream dataStream) throws IOException;
+
+   void writeData(DataOutputStream stream, Buffer... flinkBuffers) throws 
IOException;
+}
+
+interface ChannelStateDeserializer {
+
+   void readHeader(InputStream stream) throws IOException;
+
+   /**
+* Reads the length of state.
+*
+* @return size of state in bytes and offset of the current position 
resulted from reading.
+*/
+   Tuple2 readLength(InputStream stream) throws 
IOException;
+
+   int readData(InputStream stream, ChannelStateByteBuffer buffer, int 
bytes) throws IOException;
+}
+
+/**
+ * Wrapper around various buffers to receive channel state data.
+ */
+@Internal
+@NotThreadSafe
+interface ChannelStateByteBuffer {
+
+   boolean isWritable();
+
+   int writableBytes();
+
+   int writeBytes(InputStream input, int bytesToRead) throws IOException;
+
+   static ChannelStateByteBuffer wrap(Buffer buffer) {
+   return new ChannelStateByteBuffer() {
+
+   private ByteBuf byteBuf = buffer.asByteBuf();
+
+   @Override
+   public boolean isWritable() {
+   return byteBuf.isWritable();
+   }
+
+   @Override
+   public int writableBytes() {
+   return byteBuf.writableBytes();
+   }
+
+   @Override
+   public int writeBytes(InputStream input, int 
bytesToRead) throws IOException {
+   return byteBuf.writeBytes(input, 
Math.min(bytesToRead, byteBuf.writableBytes()));
+   }
+   };
+   }
+
+   static ChannelStateByteBuffer wrap(BufferBuilder bufferBuilder) {
+   final byte[] buf = new byte[1024];
+   return new ChannelStateByteBuffer() {
+   @Override
+   public boolean isWritable() {
+   return !bufferBuilder.isFull();
+   }
+
+   @Override
+   public int writableBytes() {
+   return bufferBuilder.writableBytes();
+   }
+
+   @Override
+   public int writeBytes(InputStream input, int upToBytes) 
throws IOException {
 
 Review comment:
   `upToBytes` -> `bytesToRead` to be consistent with above 
`ChannelStateByteBuffer wrap(Buffer buffer)`


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402285834
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateSerializer.java
 ##
 @@ -0,0 +1,199 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import static java.lang.Math.addExact;
+import static java.lang.Math.min;
+
+interface ChannelStateSerializer {
+
+   void writeHeader(DataOutputStream dataStream) throws IOException;
+
+   void writeData(DataOutputStream stream, Buffer... flinkBuffers) throws 
IOException;
+}
+
+interface ChannelStateDeserializer {
+
+   void readHeader(InputStream stream) throws IOException;
+
+   /**
+* Reads the length of state.
+*
+* @return size of state in bytes and offset of the current position 
resulted from reading.
+*/
+   Tuple2 readLength(InputStream stream) throws 
IOException;
+
+   int readData(InputStream stream, ChannelStateByteBuffer buffer, int 
bytes) throws IOException;
+}
+
+/**
+ * Wrapper around various buffers to receive channel state data.
+ */
+@Internal
+@NotThreadSafe
+interface ChannelStateByteBuffer {
+
+   boolean isWritable();
+
+   int writableBytes();
+
+   int writeBytes(InputStream input, int bytesToRead) throws IOException;
+
+   static ChannelStateByteBuffer wrap(Buffer buffer) {
+   return new ChannelStateByteBuffer() {
+
+   private ByteBuf byteBuf = buffer.asByteBuf();
 
 Review comment:
   nit: final


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402282625
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateSerializer.java
 ##
 @@ -0,0 +1,199 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import static java.lang.Math.addExact;
+import static java.lang.Math.min;
+
+interface ChannelStateSerializer {
+
+   void writeHeader(DataOutputStream dataStream) throws IOException;
+
+   void writeData(DataOutputStream stream, Buffer... flinkBuffers) throws 
IOException;
 
 Review comment:
   ditto: flinkBuffers


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402281766
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReaderImpl.java
 ##
 @@ -0,0 +1,116 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
+import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
+import 
org.apache.flink.runtime.checkpoint.channel.RefCountingFSDataInputStream.RefCountingFSDataInputStreamFactory;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava18.com.google.common.io.Closer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import static java.util.Arrays.asList;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link ChannelStateReader} implementation. Usage considerations:
+ * 
+ * state of a channel can be read once per instance of this class; 
once done it returns
+ * {@link 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult#NO_MORE_DATA
 NO_MORE_DATA}
+ * reader/writer indices of the passed buffer are respected and 
updated
+ * buffers must be prepared (cleared) before passing to reader
+ * buffers must be released after use
+ * 
+ */
+@NotThreadSafe
+@Internal
+public class ChannelStateReaderImpl implements ChannelStateReader {
+   private static final Logger log = 
LoggerFactory.getLogger(ChannelStateReaderImpl.class);
+
+   private final Map 
inputChannelHandleReaders;
+   private final Map 
resultSubpartitionHandleReaders;
+
+   public ChannelStateReaderImpl(TaskStateSnapshot snapshot) {
+   this(snapshot, new ChannelStateSerializerImpl());
+   }
+
+   @SuppressWarnings("WeakerAccess")
+   ChannelStateReaderImpl(TaskStateSnapshot snapshot, 
ChannelStateDeserializer serializer) {
+   RefCountingFSDataInputStreamFactory streamFactory = new 
RefCountingFSDataInputStreamFactory(serializer);
+   final HashMap 
inputChannelHandleReadersTmp = new HashMap<>();
+   final HashMap 
resultSubpartitionHandleReadersTmp = new HashMap<>();
+   for (Map.Entry e : 
snapshot.getSubtaskStateMappings()) {
+   addReaders(inputChannelHandleReadersTmp, 
e.getValue().getInputChannelState(), streamFactory);
+   addReaders(resultSubpartitionHandleReadersTmp, 
e.getValue().getResultSubpartitionState(), streamFactory);
+   }
+   inputChannelHandleReaders = inputChannelHandleReadersTmp;
+   resultSubpartitionHandleReaders = 
resultSubpartitionHandleReadersTmp;
+   }
+
+   private  void addReaders(Map readerMap, 
Collection> handles, 
RefCountingFSDataInputStreamFactory streamFactory) {
+   for (AbstractChannelStateHandle handle : handles) {
+   checkState(!readerMap.containsKey(handle.getInfo()), 
"multiple states exist for channel: " + handle.getInfo());
+   readerMap.put(handle.getInfo(), new 
ChannelStateStreamReader(handle, streamFactory));
+   }
+   }
+
+   @Override
+   public ReadResult readInputData(InputChannelInfo info, Buffer buffer) 
throws IOException {
+   log.debug("readInputData, resultSubpartitionInfo: {} , 
bufferBuilder {}", info, buffer);
+   return getReader(info, 
inputChannelHandleReaders).readInto(buffer);
+   }
+
+   @Override
+   public ReadResult readOutputData(ResultSubpartitionInfo inf

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402276598
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriter.java
 ##
 @@ -46,60 +78,64 @@
/**
 * Initiate write of channel state for the given checkpoint id.
 */
-   void start(long checkpointId);
+   void start(long checkpointId, CheckpointOptions checkpointOptions);
 
/**
 * Add in-flight buffers from the {@link 
org.apache.flink.runtime.io.network.partition.consumer.InputChannel 
InputChannel}.
-* Must be called after {@link #start(long)} and before {@link 
#finish(long)}.
+* Must be called after {@link #start} (long)} and before {@link 
#finishInput(long)}.
 
 Review comment:
   #start misses one argument


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402275569
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/RefCountingFSDataInputStream.java
 ##
 @@ -0,0 +1,126 @@
+/*
+ * 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.fs.FSDataInputStream;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+@NotThreadSafe
+class RefCountingFSDataInputStream extends FSDataInputStream {
+
+   @NotThreadSafe
+   static class RefCountingFSDataInputStreamFactory {
+   private final Map map = new HashMap<>(); // not clearing: expecting 
short life
+   private final ChannelStateDeserializer serializer;
+
+   RefCountingFSDataInputStreamFactory(ChannelStateDeserializer 
serializer) {
+   this.serializer = serializer;
+   }
+
+RefCountingFSDataInputStream 
forHandle(AbstractChannelStateHandle handle) {
+   StreamStateHandle streamStateHandle = 
handle.getDelegate();
+   RefCountingFSDataInputStream stream = 
map.get(streamStateHandle);
+   if (stream == null) {
+   stream = new 
RefCountingFSDataInputStream(streamStateHandle::openInputStream, serializer);
+   map.put(streamStateHandle, stream);
 
 Review comment:
   Will one state handle be read multiple times, so we need a map to avoid 
opening it multiple times?


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402272574
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/RefCountingFSDataInputStream.java
 ##
 @@ -0,0 +1,126 @@
+/*
+ * 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.fs.FSDataInputStream;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+@NotThreadSafe
+class RefCountingFSDataInputStream extends FSDataInputStream {
+
+   @NotThreadSafe
+   static class RefCountingFSDataInputStreamFactory {
+   private final Map map = new HashMap<>(); // not clearing: expecting 
short life
+   private final ChannelStateDeserializer serializer;
+
+   RefCountingFSDataInputStreamFactory(ChannelStateDeserializer 
serializer) {
+   this.serializer = serializer;
+   }
+
+RefCountingFSDataInputStream 
forHandle(AbstractChannelStateHandle handle) {
 
 Review comment:
   forHandle->create/buildInputStream


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402270225
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/RefCountingFSDataInputStream.java
 ##
 @@ -0,0 +1,126 @@
+/*
+ * 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.fs.FSDataInputStream;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+@NotThreadSafe
+class RefCountingFSDataInputStream extends FSDataInputStream {
+
+   @NotThreadSafe
+   static class RefCountingFSDataInputStreamFactory {
+   private final Map map = new HashMap<>(); // not clearing: expecting 
short life
+   private final ChannelStateDeserializer serializer;
+
+   RefCountingFSDataInputStreamFactory(ChannelStateDeserializer 
serializer) {
+   this.serializer = serializer;
 
 Review comment:
   nit: checkNotNull


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402269793
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/RefCountingFSDataInputStream.java
 ##
 @@ -0,0 +1,126 @@
+/*
+ * 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.fs.FSDataInputStream;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+@NotThreadSafe
+class RefCountingFSDataInputStream extends FSDataInputStream {
+
+   @NotThreadSafe
+   static class RefCountingFSDataInputStreamFactory {
+   private final Map map = new HashMap<>(); // not clearing: expecting 
short life
 
 Review comment:
   map -> streams


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402267095
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/RefCountingFSDataInputStream.java
 ##
 @@ -0,0 +1,126 @@
+/*
+ * 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.fs.FSDataInputStream;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+@NotThreadSafe
+class RefCountingFSDataInputStream extends FSDataInputStream {
+
+   @NotThreadSafe
+   static class RefCountingFSDataInputStreamFactory {
+   private final Map map = new HashMap<>(); // not clearing: expecting 
short life
+   private final ChannelStateDeserializer serializer;
+
+   RefCountingFSDataInputStreamFactory(ChannelStateDeserializer 
serializer) {
+   this.serializer = serializer;
+   }
+
+RefCountingFSDataInputStream 
forHandle(AbstractChannelStateHandle handle) {
+   StreamStateHandle streamStateHandle = 
handle.getDelegate();
+   RefCountingFSDataInputStream stream = 
map.get(streamStateHandle);
+   if (stream == null) {
+   stream = new 
RefCountingFSDataInputStream(streamStateHandle::openInputStream, serializer);
+   map.put(streamStateHandle, stream);
+   }
+   return stream;
+   }
+
+   ChannelStateDeserializer getSerializer() {
+   return serializer;
+   }
+   }
+
+   private enum State {NEW, OPENED, CLOSED}
+
+   private final SupplierWithException 
streamSupplier;
+   private FSDataInputStream stream;
+   private final ChannelStateDeserializer serializer;
+   private int numReaders = 0;
+   private State state = State.NEW;
+
+   private 
RefCountingFSDataInputStream(SupplierWithException streamSupplier, ChannelStateDeserializer serializer) {
+   this.streamSupplier = streamSupplier;
+   this.serializer = serializer;
+   }
+
+   void incNumReaders() {
+   checkNotClosed();
+   numReaders++;
+   }
+
+   void decNumReaders() throws IOException {
+   checkNotClosed();
+   numReaders--;
+   if (numReaders == 0) {
+   close();
+   }
+   }
+
+   @Override
+   public int read() throws IOException {
+   ensureOpen();
+   return stream.read();
+   }
+
+   @Override
+   public void seek(long desired) throws IOException {
 
 Review comment:
   nit: desired -> pos


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402266152
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/RefCountingFSDataInputStream.java
 ##
 @@ -0,0 +1,126 @@
+/*
+ * 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.fs.FSDataInputStream;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+@NotThreadSafe
+class RefCountingFSDataInputStream extends FSDataInputStream {
+
+   @NotThreadSafe
+   static class RefCountingFSDataInputStreamFactory {
+   private final Map map = new HashMap<>(); // not clearing: expecting 
short life
+   private final ChannelStateDeserializer serializer;
+
+   RefCountingFSDataInputStreamFactory(ChannelStateDeserializer 
serializer) {
+   this.serializer = serializer;
+   }
+
+RefCountingFSDataInputStream 
forHandle(AbstractChannelStateHandle handle) {
+   StreamStateHandle streamStateHandle = 
handle.getDelegate();
+   RefCountingFSDataInputStream stream = 
map.get(streamStateHandle);
+   if (stream == null) {
+   stream = new 
RefCountingFSDataInputStream(streamStateHandle::openInputStream, serializer);
+   map.put(streamStateHandle, stream);
+   }
+   return stream;
+   }
+
+   ChannelStateDeserializer getSerializer() {
+   return serializer;
+   }
+   }
+
+   private enum State {NEW, OPENED, CLOSED}
+
+   private final SupplierWithException 
streamSupplier;
+   private FSDataInputStream stream;
+   private final ChannelStateDeserializer serializer;
+   private int numReaders = 0;
+   private State state = State.NEW;
+
+   private 
RefCountingFSDataInputStream(SupplierWithException streamSupplier, ChannelStateDeserializer serializer) {
+   this.streamSupplier = streamSupplier;
+   this.serializer = serializer;
+   }
+
+   void incNumReaders() {
 
 Review comment:
   nit: `incRef` seems more fit into the class name. also `numReaders` -> 
`refCounter`


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402258193
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/RefCountingFSDataInputStream.java
 ##
 @@ -0,0 +1,126 @@
+/*
+ * 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.fs.FSDataInputStream;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+@NotThreadSafe
+class RefCountingFSDataInputStream extends FSDataInputStream {
+
+   @NotThreadSafe
+   static class RefCountingFSDataInputStreamFactory {
+   private final Map map = new HashMap<>(); // not clearing: expecting 
short life
+   private final ChannelStateDeserializer serializer;
+
+   RefCountingFSDataInputStreamFactory(ChannelStateDeserializer 
serializer) {
+   this.serializer = serializer;
+   }
+
+RefCountingFSDataInputStream 
forHandle(AbstractChannelStateHandle handle) {
+   StreamStateHandle streamStateHandle = 
handle.getDelegate();
+   RefCountingFSDataInputStream stream = 
map.get(streamStateHandle);
+   if (stream == null) {
+   stream = new 
RefCountingFSDataInputStream(streamStateHandle::openInputStream, serializer);
+   map.put(streamStateHandle, stream);
+   }
+   return stream;
+   }
+
+   ChannelStateDeserializer getSerializer() {
+   return serializer;
+   }
+   }
+
+   private enum State {NEW, OPENED, CLOSED}
+
+   private final SupplierWithException 
streamSupplier;
+   private FSDataInputStream stream;
+   private final ChannelStateDeserializer serializer;
+   private int numReaders = 0;
+   private State state = State.NEW;
+
+   private 
RefCountingFSDataInputStream(SupplierWithException streamSupplier, ChannelStateDeserializer serializer) {
 
 Review comment:
   too long line for splitting the arguments


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402257906
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/RefCountingFSDataInputStream.java
 ##
 @@ -0,0 +1,126 @@
+/*
+ * 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.fs.FSDataInputStream;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+@NotThreadSafe
+class RefCountingFSDataInputStream extends FSDataInputStream {
+
+   @NotThreadSafe
+   static class RefCountingFSDataInputStreamFactory {
 
 Review comment:
   nit: i think it is better to place the factory class at the bottom of this 
class


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402253915
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateStreamReader.java
 ##
 @@ -0,0 +1,117 @@
+/*
+ * 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.api.java.tuple.Tuple2;
+import 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult;
+import 
org.apache.flink.runtime.checkpoint.channel.RefCountingFSDataInputStream.RefCountingFSDataInputStreamFactory;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+
+import static java.lang.Math.addExact;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateByteBuffer.wrap;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.HAS_MORE_DATA;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.NO_MORE_DATA;
+
+/**
+ * Reads the state of a single channel pointed by {@link 
org.apache.flink.runtime.state.AbstractChannelStateHandle 
AbstractChannelStateHandle}.
+ * Single-use.
+ * Uses {@link RefCountingFSDataInputStream} internally.
+ */
+@NotThreadSafe
+class ChannelStateStreamReader implements Closeable {
+
+   private final RefCountingFSDataInputStream stream; // todo: buffer?
+   private final Queue offsets;
+   private final ChannelStateDeserializer serializer;
+   private long pos = -1;
+   private int rem;
+   private boolean closed = false;
+
+   ChannelStateStreamReader(AbstractChannelStateHandle handle, 
RefCountingFSDataInputStreamFactory streamFactory) {
+   this(streamFactory.forHandle(handle), handle.getOffsets(), 
streamFactory.getSerializer());
+   }
+
+   private ChannelStateStreamReader(RefCountingFSDataInputStream stream, 
List offsets, ChannelStateDeserializer serializer) {
+   this.stream = stream;
+   this.stream.incNumReaders();
+   this.serializer = serializer;
+   this.offsets = new LinkedList<>(offsets);
+   }
+
+   ReadResult readInto(Buffer buffer) throws IOException {
+   return readInto(wrap(buffer));
+   }
+
+   ReadResult readInto(BufferBuilder bufferBuilder) throws IOException {
+   return readInto(wrap(bufferBuilder));
+   }
+
+   private ReadResult readInto(ChannelStateByteBuffer buffer) throws 
IOException {
+   Preconditions.checkState(!closed, "reader is closed");
+   readWhilePossible(buffer);
+   if (haveMoreData()) {
+   return HAS_MORE_DATA;
+   } else {
+   close();
+   return NO_MORE_DATA;
+   }
+   }
+
+   private void readWhilePossible(ChannelStateByteBuffer buffer) throws 
IOException {
+   while (haveMoreData() && buffer.isWritable()) {
+   if (pos < 0 || rem <= 0) {
 
 Review comment:
   I guess we can get ride of `pos` field to only judge `rem <= 0` to advance 
offset. If so we can also avoid return `tuple2` in 
`serializer.readLength(stream)`


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402218677
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateStreamReader.java
 ##
 @@ -0,0 +1,117 @@
+/*
+ * 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.api.java.tuple.Tuple2;
+import 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult;
+import 
org.apache.flink.runtime.checkpoint.channel.RefCountingFSDataInputStream.RefCountingFSDataInputStreamFactory;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+
+import static java.lang.Math.addExact;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateByteBuffer.wrap;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.HAS_MORE_DATA;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.NO_MORE_DATA;
+
+/**
+ * Reads the state of a single channel pointed by {@link 
org.apache.flink.runtime.state.AbstractChannelStateHandle 
AbstractChannelStateHandle}.
+ * Single-use.
+ * Uses {@link RefCountingFSDataInputStream} internally.
+ */
+@NotThreadSafe
+class ChannelStateStreamReader implements Closeable {
+
+   private final RefCountingFSDataInputStream stream; // todo: buffer?
 
 Review comment:
   should be clear for `TODO`


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402218297
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateStreamReader.java
 ##
 @@ -0,0 +1,117 @@
+/*
+ * 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.api.java.tuple.Tuple2;
+import 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult;
+import 
org.apache.flink.runtime.checkpoint.channel.RefCountingFSDataInputStream.RefCountingFSDataInputStreamFactory;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+
+import static java.lang.Math.addExact;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateByteBuffer.wrap;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.HAS_MORE_DATA;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.NO_MORE_DATA;
+
+/**
+ * Reads the state of a single channel pointed by {@link 
org.apache.flink.runtime.state.AbstractChannelStateHandle 
AbstractChannelStateHandle}.
+ * Single-use.
 
 Review comment:
   nit: What is it indicating for : `Single-use`.


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402216971
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateStreamReader.java
 ##
 @@ -0,0 +1,117 @@
+/*
+ * 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.api.java.tuple.Tuple2;
+import 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult;
+import 
org.apache.flink.runtime.checkpoint.channel.RefCountingFSDataInputStream.RefCountingFSDataInputStreamFactory;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+
+import static java.lang.Math.addExact;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateByteBuffer.wrap;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.HAS_MORE_DATA;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.NO_MORE_DATA;
+
+/**
+ * Reads the state of a single channel pointed by {@link 
org.apache.flink.runtime.state.AbstractChannelStateHandle 
AbstractChannelStateHandle}.
+ * Single-use.
+ * Uses {@link RefCountingFSDataInputStream} internally.
+ */
+@NotThreadSafe
+class ChannelStateStreamReader implements Closeable {
+
+   private final RefCountingFSDataInputStream stream; // todo: buffer?
+   private final Queue offsets;
+   private final ChannelStateDeserializer serializer;
+   private long pos = -1;
+   private int rem;
 
 Review comment:
   Better to give some descriptions for these fields for better understanding, 
especially for `rem`.


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402215570
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateStreamReader.java
 ##
 @@ -0,0 +1,117 @@
+/*
+ * 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.api.java.tuple.Tuple2;
+import 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult;
+import 
org.apache.flink.runtime.checkpoint.channel.RefCountingFSDataInputStream.RefCountingFSDataInputStreamFactory;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+
+import static java.lang.Math.addExact;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateByteBuffer.wrap;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.HAS_MORE_DATA;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult.NO_MORE_DATA;
+
+/**
+ * Reads the state of a single channel pointed by {@link 
org.apache.flink.runtime.state.AbstractChannelStateHandle 
AbstractChannelStateHandle}.
+ * Single-use.
+ * Uses {@link RefCountingFSDataInputStream} internally.
+ */
+@NotThreadSafe
+class ChannelStateStreamReader implements Closeable {
+
+   private final RefCountingFSDataInputStream stream; // todo: buffer?
+   private final Queue offsets;
+   private final ChannelStateDeserializer serializer;
+   private long pos = -1;
+   private int rem;
+   private boolean closed = false;
+
+   ChannelStateStreamReader(AbstractChannelStateHandle handle, 
RefCountingFSDataInputStreamFactory streamFactory) {
+   this(streamFactory.forHandle(handle), handle.getOffsets(), 
streamFactory.getSerializer());
 
 Review comment:
   nit: checkNotNull


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


[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402207068
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReaderImpl.java
 ##
 @@ -0,0 +1,116 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
+import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
+import 
org.apache.flink.runtime.checkpoint.channel.RefCountingFSDataInputStream.RefCountingFSDataInputStreamFactory;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava18.com.google.common.io.Closer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import static java.util.Arrays.asList;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link ChannelStateReader} implementation. Usage considerations:
+ * 
+ * state of a channel can be read once per instance of this class; 
once done it returns
+ * {@link 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult#NO_MORE_DATA
 NO_MORE_DATA}
+ * reader/writer indices of the passed buffer are respected and 
updated
+ * buffers must be prepared (cleared) before passing to reader
+ * buffers must be released after use
+ * 
+ */
+@NotThreadSafe
+@Internal
+public class ChannelStateReaderImpl implements ChannelStateReader {
+   private static final Logger log = 
LoggerFactory.getLogger(ChannelStateReaderImpl.class);
+
+   private final Map 
inputChannelHandleReaders;
+   private final Map 
resultSubpartitionHandleReaders;
+
+   public ChannelStateReaderImpl(TaskStateSnapshot snapshot) {
+   this(snapshot, new ChannelStateSerializerImpl());
+   }
+
+   @SuppressWarnings("WeakerAccess")
+   ChannelStateReaderImpl(TaskStateSnapshot snapshot, 
ChannelStateDeserializer serializer) {
+   RefCountingFSDataInputStreamFactory streamFactory = new 
RefCountingFSDataInputStreamFactory(serializer);
+   final HashMap 
inputChannelHandleReadersTmp = new HashMap<>();
+   final HashMap 
resultSubpartitionHandleReadersTmp = new HashMap<>();
+   for (Map.Entry e : 
snapshot.getSubtaskStateMappings()) {
+   addReaders(inputChannelHandleReadersTmp, 
e.getValue().getInputChannelState(), streamFactory);
+   addReaders(resultSubpartitionHandleReadersTmp, 
e.getValue().getResultSubpartitionState(), streamFactory);
+   }
+   inputChannelHandleReaders = inputChannelHandleReadersTmp;
+   resultSubpartitionHandleReaders = 
resultSubpartitionHandleReadersTmp;
+   }
+
+   private  void addReaders(Map readerMap, 
Collection> handles, 
RefCountingFSDataInputStreamFactory streamFactory) {
+   for (AbstractChannelStateHandle handle : handles) {
+   checkState(!readerMap.containsKey(handle.getInfo()), 
"multiple states exist for channel: " + handle.getInfo());
+   readerMap.put(handle.getInfo(), new 
ChannelStateStreamReader(handle, streamFactory));
+   }
+   }
+
+   @Override
+   public ReadResult readInputData(InputChannelInfo info, Buffer buffer) 
throws IOException {
+   log.debug("readInputData, resultSubpartitionInfo: {} , 
bufferBuilder {}", info, buffer);
 
 Review comment:
   nit: bufferBuilder -> buffer:


This is an automated message from the Apache Git Service.
To res

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402207068
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReaderImpl.java
 ##
 @@ -0,0 +1,116 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
+import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
+import 
org.apache.flink.runtime.checkpoint.channel.RefCountingFSDataInputStream.RefCountingFSDataInputStreamFactory;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava18.com.google.common.io.Closer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import static java.util.Arrays.asList;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link ChannelStateReader} implementation. Usage considerations:
+ * 
+ * state of a channel can be read once per instance of this class; 
once done it returns
+ * {@link 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult#NO_MORE_DATA
 NO_MORE_DATA}
+ * reader/writer indices of the passed buffer are respected and 
updated
+ * buffers must be prepared (cleared) before passing to reader
+ * buffers must be released after use
+ * 
+ */
+@NotThreadSafe
+@Internal
+public class ChannelStateReaderImpl implements ChannelStateReader {
+   private static final Logger log = 
LoggerFactory.getLogger(ChannelStateReaderImpl.class);
+
+   private final Map 
inputChannelHandleReaders;
+   private final Map 
resultSubpartitionHandleReaders;
+
+   public ChannelStateReaderImpl(TaskStateSnapshot snapshot) {
+   this(snapshot, new ChannelStateSerializerImpl());
+   }
+
+   @SuppressWarnings("WeakerAccess")
+   ChannelStateReaderImpl(TaskStateSnapshot snapshot, 
ChannelStateDeserializer serializer) {
+   RefCountingFSDataInputStreamFactory streamFactory = new 
RefCountingFSDataInputStreamFactory(serializer);
+   final HashMap 
inputChannelHandleReadersTmp = new HashMap<>();
+   final HashMap 
resultSubpartitionHandleReadersTmp = new HashMap<>();
+   for (Map.Entry e : 
snapshot.getSubtaskStateMappings()) {
+   addReaders(inputChannelHandleReadersTmp, 
e.getValue().getInputChannelState(), streamFactory);
+   addReaders(resultSubpartitionHandleReadersTmp, 
e.getValue().getResultSubpartitionState(), streamFactory);
+   }
+   inputChannelHandleReaders = inputChannelHandleReadersTmp;
+   resultSubpartitionHandleReaders = 
resultSubpartitionHandleReadersTmp;
+   }
+
+   private  void addReaders(Map readerMap, 
Collection> handles, 
RefCountingFSDataInputStreamFactory streamFactory) {
+   for (AbstractChannelStateHandle handle : handles) {
+   checkState(!readerMap.containsKey(handle.getInfo()), 
"multiple states exist for channel: " + handle.getInfo());
+   readerMap.put(handle.getInfo(), new 
ChannelStateStreamReader(handle, streamFactory));
+   }
+   }
+
+   @Override
+   public ReadResult readInputData(InputChannelInfo info, Buffer buffer) 
throws IOException {
+   log.debug("readInputData, resultSubpartitionInfo: {} , 
bufferBuilder {}", info, buffer);
 
 Review comment:
   nit: bufferBuilder -> buffer


This is an automated message from the Apache Git Service.
To resp

[GitHub] [flink] zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints

2020-04-02 Thread GitBox
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_r402205044
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReaderImpl.java
 ##
 @@ -0,0 +1,116 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
+import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
+import 
org.apache.flink.runtime.checkpoint.channel.RefCountingFSDataInputStream.RefCountingFSDataInputStreamFactory;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.state.AbstractChannelStateHandle;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.guava18.com.google.common.io.Closer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import static java.util.Arrays.asList;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link ChannelStateReader} implementation. Usage considerations:
+ * 
+ * state of a channel can be read once per instance of this class; 
once done it returns
+ * {@link 
org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult#NO_MORE_DATA
 NO_MORE_DATA}
+ * reader/writer indices of the passed buffer are respected and 
updated
+ * buffers must be prepared (cleared) before passing to reader
+ * buffers must be released after use
+ * 
+ */
+@NotThreadSafe
+@Internal
+public class ChannelStateReaderImpl implements ChannelStateReader {
+   private static final Logger log = 
LoggerFactory.getLogger(ChannelStateReaderImpl.class);
+
+   private final Map 
inputChannelHandleReaders;
+   private final Map 
resultSubpartitionHandleReaders;
+
+   public ChannelStateReaderImpl(TaskStateSnapshot snapshot) {
+   this(snapshot, new ChannelStateSerializerImpl());
+   }
+
+   @SuppressWarnings("WeakerAccess")
+   ChannelStateReaderImpl(TaskStateSnapshot snapshot, 
ChannelStateDeserializer serializer) {
+   RefCountingFSDataInputStreamFactory streamFactory = new 
RefCountingFSDataInputStreamFactory(serializer);
+   final HashMap 
inputChannelHandleReadersTmp = new HashMap<>();
 
 Review comment:
   we can pass class field `inputChannelHandleReaders` directly  in below 
`addReaders` to avoid temporary variables.


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


  1   2   >