You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2021/01/21 15:08:22 UTC

[GitHub] [flink] AHeise opened a new pull request #14721: [FLINK-20645][network] Fix corrupted unaligned checkpoints.

AHeise opened a new pull request #14721:
URL: https://github.com/apache/flink/pull/14721


   <!--
   *Thank you very much for contributing to Apache Flink - we are happy that you want to help us improve Flink. To help the community review your contribution in the best possible way, please go through the checklist below, which will get the contribution into a shape in which it can be best reviewed.*
   
   *Please understand that we do not do this to make contributions to Flink a hassle. In order to uphold a high standard of quality for code contributions, while at the same time managing a large number of contributions, we need contributors to prepare the contributions well, and give reviewers enough contextual information for the review. Please also understand that contributions that do not follow this guide will take longer to review and thus typically be picked up with lower priority by the community.*
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [JIRA issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue.
     
     - Name the pull request in the form "[FLINK-XXXX] [component] Title of the pull request", where *FLINK-XXXX* should be replaced by the actual issue number. Skip *component* if you are unsure about which is the best component.
     Typo fixes that have no associated JIRA issue should be named following this pattern: `[hotfix] [docs] Fix typo in event time introduction` or `[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`.
   
     - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
     
     - Make sure that the change passes the automated tests, i.e., `mvn clean verify` passes. You can set up Azure Pipelines CI to do that following [this guide](https://cwiki.apache.org/confluence/display/FLINK/Azure+Pipelines#AzurePipelines-Tutorial:SettingupAzurePipelinesforaforkoftheFlinkrepository).
   
     - Each pull request should address only one issue, not mix up code from multiple issues.
     
     - Each commit in the pull request has a meaningful commit message (including the JIRA id)
   
     - Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.
   
   
   **(The sections below can be removed for hotfixes of typos)**
   -->
   
   ## What is the purpose of the change
   
   With unusual barrier flow, checkpoints may miss certain buffers and become unrecoverable.
   
   ## Brief change log
   
   - Adding more trace output for network and checkpoint buffers.
   - Fix ChannelStatePersister. 
   - Improve UnalignedCheckpointITCase to detect data corruption faster.
   
   ```
   9996 [forward (1/1)#0] TRACE org.apache.flink.runtime.io.network.logger.NetworkActionsLogger [] - LocalInputChannel#getNextBuffer Buffer{size=1996, hash=-1568575630}, seq 4, ChannelStatePersister(1 (COMPLETED)} @ InputChannelInfo{gateIdx=0, inputChannelIdx=0}
   9996 [Channel state writer Source: source (1/1)#0] TRACE org.apache.flink.runtime.io.network.logger.NetworkActionsLogger [] - ChannelStateWriteRequest#writeOutput Buffer{size=20, hash=1432560496}, checkpoint 2 @ ResultSubpartitionInfo{partitionIdx=0, subPartitionIdx=0}
   9996 [forward (1/1)#0] TRACE org.apache.flink.runtime.io.network.logger.NetworkActionsLogger [] - PipelinedSubpartition#pollBuffer Buffer{size=38, hash=-1801242596} @ ResultSubpartitionInfo{partitionIdx=0, subPartitionIdx=0}
   9996 [forward (1/1)#0] TRACE org.apache.flink.runtime.io.network.logger.NetworkActionsLogger [] - LocalInputChannel#getNextBuffer Buffer{size=38, hash=-1801242596}, seq 5, ChannelStatePersister(2 (BARRIER_RECEIVED)} @ InputChannelInfo{gateIdx=0, inputChannelIdx=0}
   9996 [Source: source (1/1)#0] INFO  org.apache.flink.test.checkpointing.UnalignedCheckpointTestBase [] - Snapshotted LongSplit{increment=1, nextNumber=199, numCompletedCheckpoints=1} @ 0 subtask (0 attempt)
   ```
   
   ## Verifying this change
   
   Already covered by existing UnalignedControllerTest when assertion is added.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / **no**)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
     - The serializers: (yes / **no** / don't know)
     - The runtime per-record code paths (performance sensitive): (**yes** / no / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
     - The S3 file system connector: (yes / **no** / don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (yes / **no**)
     - If yes, how is the feature documented? (**not applicable** / docs / JavaDocs / not documented)
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14721: [FLINK-20645][network] Fix corrupted unaligned checkpoints.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14721:
URL: https://github.com/apache/flink/pull/14721#issuecomment-764731800


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f120eb37387e00fb7ac7ef8ea628520fc2dddaa4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12339",
       "triggerID" : "f120eb37387e00fb7ac7ef8ea628520fc2dddaa4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b96629a36462b3b4b253c920cf036b951fafdc6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9b96629a36462b3b4b253c920cf036b951fafdc6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f120eb37387e00fb7ac7ef8ea628520fc2dddaa4 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12339) 
   * 9b96629a36462b3b4b253c920cf036b951fafdc6 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] AHeise commented on a change in pull request #14721: [FLINK-20645][network] Fix corrupted unaligned checkpoints.

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #14721:
URL: https://github.com/apache/flink/pull/14721#discussion_r562439086



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/logger/NetworkActionsLogger.java
##########
@@ -18,51 +18,89 @@
 
 package org.apache.flink.runtime.io.network.logger;
 
+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.BufferConsumer;
+import org.apache.flink.runtime.io.network.partition.consumer.ChannelStatePersister;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Arrays;
-
-import static org.apache.flink.util.Preconditions.checkState;
-
 /** Utility class for logging actions that happened in the network stack for debugging purposes. */
 public class NetworkActionsLogger {
     private static final Logger LOG = LoggerFactory.getLogger(NetworkActionsLogger.class);
-
     private static final boolean ENABLED = LOG.isTraceEnabled();
     private static final boolean INCLUDE_HASH = true;
 
-    public static void log(Class<?> clazz, String action, Buffer buffer) {
+    public static void traceInput(
+            Class<?> clazz,
+            String action,
+            Buffer buffer,
+            InputChannelInfo channelInfo,
+            ChannelStatePersister channelStatePersister,
+            int sequenceNumber) {
+        if (ENABLED) {
+            LOG.trace(
+                    "{}#{} {}, seq {}, {} @ {}",
+                    clazz.getSimpleName(),
+                    action,

Review comment:
       I merged this parameter now with the class constant.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14721: [FLINK-20645][network] Fix corrupted unaligned checkpoints.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14721:
URL: https://github.com/apache/flink/pull/14721#issuecomment-764731800






----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] AHeise merged pull request #14721: [FLINK-20645][network] Fix corrupted unaligned checkpoints.

Posted by GitBox <gi...@apache.org>.
AHeise merged pull request #14721:
URL: https://github.com/apache/flink/pull/14721


   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rkhachatryan commented on a change in pull request #14721: [FLINK-20645][network] Fix corrupted unaligned checkpoints.

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #14721:
URL: https://github.com/apache/flink/pull/14721#discussion_r562029491



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/logger/NetworkActionsLogger.java
##########
@@ -18,51 +18,89 @@
 
 package org.apache.flink.runtime.io.network.logger;
 
+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.BufferConsumer;
+import org.apache.flink.runtime.io.network.partition.consumer.ChannelStatePersister;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Arrays;
-
-import static org.apache.flink.util.Preconditions.checkState;
-
 /** Utility class for logging actions that happened in the network stack for debugging purposes. */
 public class NetworkActionsLogger {
     private static final Logger LOG = LoggerFactory.getLogger(NetworkActionsLogger.class);
-
     private static final boolean ENABLED = LOG.isTraceEnabled();
     private static final boolean INCLUDE_HASH = true;
 
-    public static void log(Class<?> clazz, String action, Buffer buffer) {
+    public static void traceInput(
+            Class<?> clazz,
+            String action,
+            Buffer buffer,
+            InputChannelInfo channelInfo,
+            ChannelStatePersister channelStatePersister,
+            int sequenceNumber) {
+        if (ENABLED) {
+            LOG.trace(
+                    "{}#{} {}, seq {}, {} @ {}",
+                    clazz.getSimpleName(),

Review comment:
       This call has some overhead, as well as `getClass()` (though this code shouldn't execute in prod it can affect debugging).
   How about using some string constant (in caller)?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/logger/NetworkActionsLogger.java
##########
@@ -18,51 +18,89 @@
 
 package org.apache.flink.runtime.io.network.logger;
 
+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.BufferConsumer;
+import org.apache.flink.runtime.io.network.partition.consumer.ChannelStatePersister;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Arrays;
-
-import static org.apache.flink.util.Preconditions.checkState;
-
 /** Utility class for logging actions that happened in the network stack for debugging purposes. */
 public class NetworkActionsLogger {
     private static final Logger LOG = LoggerFactory.getLogger(NetworkActionsLogger.class);
-
     private static final boolean ENABLED = LOG.isTraceEnabled();
     private static final boolean INCLUDE_HASH = true;
 
-    public static void log(Class<?> clazz, String action, Buffer buffer) {
+    public static void traceInput(
+            Class<?> clazz,
+            String action,
+            Buffer buffer,
+            InputChannelInfo channelInfo,
+            ChannelStatePersister channelStatePersister,
+            int sequenceNumber) {
+        if (ENABLED) {
+            LOG.trace(
+                    "{}#{} {}, seq {}, {} @ {}",
+                    clazz.getSimpleName(),
+                    action,
+                    buffer.toDebugString(INCLUDE_HASH),
+                    sequenceNumber,
+                    channelStatePersister,
+                    channelInfo);
+        }
+    }
+
+    public static void traceOutput(
+            Class<?> clazz, String action, Buffer buffer, ResultSubpartitionInfo channelInfo) {
+        if (ENABLED) {
+            LOG.trace(
+                    "{}#{} {} @ {}",
+                    clazz.getSimpleName(),
+                    action,
+                    buffer.toDebugString(INCLUDE_HASH),
+                    channelInfo);
+        }
+    }
+
+    public static void traceRecover(
+            Class<?> clazz, String action, Buffer buffer, InputChannelInfo channelInfo) {
         if (ENABLED) {
-            LOG.trace("{}#{} buffer = [{}]", clazz.getSimpleName(), action, toPrettyString(buffer));
+            LOG.trace(
+                    "{}#{} {} @ {}",
+                    clazz.getSimpleName(),
+                    action,
+                    buffer.toDebugString(INCLUDE_HASH),
+                    channelInfo);
         }
     }
 
-    public static void log(Class<?> clazz, String action, BufferConsumer bufferConsumer) {
+    public static void traceRecover(
+            Class<?> clazz,
+            String action,
+            BufferConsumer bufferConsumer,
+            ResultSubpartitionInfo channelInfo) {
         if (ENABLED) {
-            Buffer buffer = null;
-            try (BufferConsumer copiedBufferConsumer = bufferConsumer.copy()) {
-                buffer = copiedBufferConsumer.build();
-                log(clazz, action, buffer);
-                checkState(copiedBufferConsumer.isFinished());
-            } finally {
-                if (buffer != null) {
-                    buffer.recycleBuffer();
-                }
-            }
+            LOG.trace(
+                    "{}#{} {} @ {}",
+                    clazz.getSimpleName(),
+                    action,
+                    bufferConsumer.toDebugString(INCLUDE_HASH),
+                    channelInfo);
         }
     }
 
-    private static String toPrettyString(Buffer buffer) {
-        StringBuilder prettyString = new StringBuilder("size=").append(buffer.getSize());
-        if (INCLUDE_HASH) {
-            byte[] bytes = new byte[buffer.getSize()];
-            buffer.readOnlySlice().asByteBuf().readBytes(bytes);
-            prettyString.append(", hash=").append(Arrays.hashCode(bytes));
+    public static void traceWrite(

Review comment:
       nit: `tracePersist`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/ChannelStatePersister.java
##########
@@ -62,9 +68,18 @@
     }
 
     protected void startPersisting(long barrierId, List<Buffer> knownBuffers) {
+        LOG.debug(

Review comment:
       Does it makes sense to log `channelInfo`?
   
   nit: extract log method?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/ChannelStatePersister.java
##########
@@ -95,16 +115,39 @@ protected void maybePersist(Buffer buffer) {
     protected Optional<Long> checkForBarrier(Buffer buffer) throws IOException {
         final AbstractEvent event = parseEvent(buffer);
         if (event instanceof CheckpointBarrier) {
-            if (((CheckpointBarrier) event).getId() >= lastSeenBarrier) {
+            final long barrierId = ((CheckpointBarrier) event).getId();
+            long expectedBarrierId =
+                    checkpointStatus == CheckpointStatus.COMPLETED
+                            ? lastSeenBarrier + 1

Review comment:
       Good catch! :1st_place_medal: :)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/logger/NetworkActionsLogger.java
##########
@@ -18,51 +18,89 @@
 
 package org.apache.flink.runtime.io.network.logger;
 
+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.BufferConsumer;
+import org.apache.flink.runtime.io.network.partition.consumer.ChannelStatePersister;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Arrays;
-
-import static org.apache.flink.util.Preconditions.checkState;
-
 /** Utility class for logging actions that happened in the network stack for debugging purposes. */
 public class NetworkActionsLogger {
     private static final Logger LOG = LoggerFactory.getLogger(NetworkActionsLogger.class);
-
     private static final boolean ENABLED = LOG.isTraceEnabled();
     private static final boolean INCLUDE_HASH = true;
 
-    public static void log(Class<?> clazz, String action, Buffer buffer) {
+    public static void traceInput(
+            Class<?> clazz,
+            String action,
+            Buffer buffer,
+            InputChannelInfo channelInfo,
+            ChannelStatePersister channelStatePersister,
+            int sequenceNumber) {
+        if (ENABLED) {
+            LOG.trace(
+                    "{}#{} {}, seq {}, {} @ {}",
+                    clazz.getSimpleName(),
+                    action,

Review comment:
       nit: I think `action` is not needed if we have method per action.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14721: [FLINK-20645][network] Fix corrupted unaligned checkpoints.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14721:
URL: https://github.com/apache/flink/pull/14721#issuecomment-764731800


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f120eb37387e00fb7ac7ef8ea628520fc2dddaa4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12339",
       "triggerID" : "f120eb37387e00fb7ac7ef8ea628520fc2dddaa4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b96629a36462b3b4b253c920cf036b951fafdc6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12369",
       "triggerID" : "9b96629a36462b3b4b253c920cf036b951fafdc6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9b96629a36462b3b4b253c920cf036b951fafdc6 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12369) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14721: [FLINK-20645][network] Fix corrupted unaligned checkpoints.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14721:
URL: https://github.com/apache/flink/pull/14721#issuecomment-764731800


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f120eb37387e00fb7ac7ef8ea628520fc2dddaa4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12339",
       "triggerID" : "f120eb37387e00fb7ac7ef8ea628520fc2dddaa4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9b96629a36462b3b4b253c920cf036b951fafdc6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12369",
       "triggerID" : "9b96629a36462b3b4b253c920cf036b951fafdc6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f120eb37387e00fb7ac7ef8ea628520fc2dddaa4 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12339) 
   * 9b96629a36462b3b4b253c920cf036b951fafdc6 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12369) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] rkhachatryan commented on a change in pull request #14721: [FLINK-20645][network] Fix corrupted unaligned checkpoints.

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on a change in pull request #14721:
URL: https://github.com/apache/flink/pull/14721#discussion_r562029491



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/logger/NetworkActionsLogger.java
##########
@@ -18,51 +18,89 @@
 
 package org.apache.flink.runtime.io.network.logger;
 
+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.BufferConsumer;
+import org.apache.flink.runtime.io.network.partition.consumer.ChannelStatePersister;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Arrays;
-
-import static org.apache.flink.util.Preconditions.checkState;
-
 /** Utility class for logging actions that happened in the network stack for debugging purposes. */
 public class NetworkActionsLogger {
     private static final Logger LOG = LoggerFactory.getLogger(NetworkActionsLogger.class);
-
     private static final boolean ENABLED = LOG.isTraceEnabled();
     private static final boolean INCLUDE_HASH = true;
 
-    public static void log(Class<?> clazz, String action, Buffer buffer) {
+    public static void traceInput(
+            Class<?> clazz,
+            String action,
+            Buffer buffer,
+            InputChannelInfo channelInfo,
+            ChannelStatePersister channelStatePersister,
+            int sequenceNumber) {
+        if (ENABLED) {
+            LOG.trace(
+                    "{}#{} {}, seq {}, {} @ {}",
+                    clazz.getSimpleName(),

Review comment:
       This call has some overhead, as well as `getClass()` (though this code shouldn't execute in prod it can affect debugging).
   How about using some string constant (in caller)?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/logger/NetworkActionsLogger.java
##########
@@ -18,51 +18,89 @@
 
 package org.apache.flink.runtime.io.network.logger;
 
+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.BufferConsumer;
+import org.apache.flink.runtime.io.network.partition.consumer.ChannelStatePersister;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Arrays;
-
-import static org.apache.flink.util.Preconditions.checkState;
-
 /** Utility class for logging actions that happened in the network stack for debugging purposes. */
 public class NetworkActionsLogger {
     private static final Logger LOG = LoggerFactory.getLogger(NetworkActionsLogger.class);
-
     private static final boolean ENABLED = LOG.isTraceEnabled();
     private static final boolean INCLUDE_HASH = true;
 
-    public static void log(Class<?> clazz, String action, Buffer buffer) {
+    public static void traceInput(
+            Class<?> clazz,
+            String action,
+            Buffer buffer,
+            InputChannelInfo channelInfo,
+            ChannelStatePersister channelStatePersister,
+            int sequenceNumber) {
+        if (ENABLED) {
+            LOG.trace(
+                    "{}#{} {}, seq {}, {} @ {}",
+                    clazz.getSimpleName(),
+                    action,
+                    buffer.toDebugString(INCLUDE_HASH),
+                    sequenceNumber,
+                    channelStatePersister,
+                    channelInfo);
+        }
+    }
+
+    public static void traceOutput(
+            Class<?> clazz, String action, Buffer buffer, ResultSubpartitionInfo channelInfo) {
+        if (ENABLED) {
+            LOG.trace(
+                    "{}#{} {} @ {}",
+                    clazz.getSimpleName(),
+                    action,
+                    buffer.toDebugString(INCLUDE_HASH),
+                    channelInfo);
+        }
+    }
+
+    public static void traceRecover(
+            Class<?> clazz, String action, Buffer buffer, InputChannelInfo channelInfo) {
         if (ENABLED) {
-            LOG.trace("{}#{} buffer = [{}]", clazz.getSimpleName(), action, toPrettyString(buffer));
+            LOG.trace(
+                    "{}#{} {} @ {}",
+                    clazz.getSimpleName(),
+                    action,
+                    buffer.toDebugString(INCLUDE_HASH),
+                    channelInfo);
         }
     }
 
-    public static void log(Class<?> clazz, String action, BufferConsumer bufferConsumer) {
+    public static void traceRecover(
+            Class<?> clazz,
+            String action,
+            BufferConsumer bufferConsumer,
+            ResultSubpartitionInfo channelInfo) {
         if (ENABLED) {
-            Buffer buffer = null;
-            try (BufferConsumer copiedBufferConsumer = bufferConsumer.copy()) {
-                buffer = copiedBufferConsumer.build();
-                log(clazz, action, buffer);
-                checkState(copiedBufferConsumer.isFinished());
-            } finally {
-                if (buffer != null) {
-                    buffer.recycleBuffer();
-                }
-            }
+            LOG.trace(
+                    "{}#{} {} @ {}",
+                    clazz.getSimpleName(),
+                    action,
+                    bufferConsumer.toDebugString(INCLUDE_HASH),
+                    channelInfo);
         }
     }
 
-    private static String toPrettyString(Buffer buffer) {
-        StringBuilder prettyString = new StringBuilder("size=").append(buffer.getSize());
-        if (INCLUDE_HASH) {
-            byte[] bytes = new byte[buffer.getSize()];
-            buffer.readOnlySlice().asByteBuf().readBytes(bytes);
-            prettyString.append(", hash=").append(Arrays.hashCode(bytes));
+    public static void traceWrite(

Review comment:
       nit: `tracePersist`?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/ChannelStatePersister.java
##########
@@ -62,9 +68,18 @@
     }
 
     protected void startPersisting(long barrierId, List<Buffer> knownBuffers) {
+        LOG.debug(

Review comment:
       Does it makes sense to log `channelInfo`?
   
   nit: extract log method?

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/ChannelStatePersister.java
##########
@@ -95,16 +115,39 @@ protected void maybePersist(Buffer buffer) {
     protected Optional<Long> checkForBarrier(Buffer buffer) throws IOException {
         final AbstractEvent event = parseEvent(buffer);
         if (event instanceof CheckpointBarrier) {
-            if (((CheckpointBarrier) event).getId() >= lastSeenBarrier) {
+            final long barrierId = ((CheckpointBarrier) event).getId();
+            long expectedBarrierId =
+                    checkpointStatus == CheckpointStatus.COMPLETED
+                            ? lastSeenBarrier + 1

Review comment:
       Good catch! :1st_place_medal: :)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/logger/NetworkActionsLogger.java
##########
@@ -18,51 +18,89 @@
 
 package org.apache.flink.runtime.io.network.logger;
 
+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.BufferConsumer;
+import org.apache.flink.runtime.io.network.partition.consumer.ChannelStatePersister;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Arrays;
-
-import static org.apache.flink.util.Preconditions.checkState;
-
 /** Utility class for logging actions that happened in the network stack for debugging purposes. */
 public class NetworkActionsLogger {
     private static final Logger LOG = LoggerFactory.getLogger(NetworkActionsLogger.class);
-
     private static final boolean ENABLED = LOG.isTraceEnabled();
     private static final boolean INCLUDE_HASH = true;
 
-    public static void log(Class<?> clazz, String action, Buffer buffer) {
+    public static void traceInput(
+            Class<?> clazz,
+            String action,
+            Buffer buffer,
+            InputChannelInfo channelInfo,
+            ChannelStatePersister channelStatePersister,
+            int sequenceNumber) {
+        if (ENABLED) {
+            LOG.trace(
+                    "{}#{} {}, seq {}, {} @ {}",
+                    clazz.getSimpleName(),
+                    action,

Review comment:
       nit: I think `action` is not needed if we have method per action.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] AHeise commented on a change in pull request #14721: [FLINK-20645][network] Fix corrupted unaligned checkpoints.

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #14721:
URL: https://github.com/apache/flink/pull/14721#discussion_r562438971



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/logger/NetworkActionsLogger.java
##########
@@ -18,51 +18,89 @@
 
 package org.apache.flink.runtime.io.network.logger;
 
+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.BufferConsumer;
+import org.apache.flink.runtime.io.network.partition.consumer.ChannelStatePersister;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Arrays;
-
-import static org.apache.flink.util.Preconditions.checkState;
-
 /** Utility class for logging actions that happened in the network stack for debugging purposes. */
 public class NetworkActionsLogger {
     private static final Logger LOG = LoggerFactory.getLogger(NetworkActionsLogger.class);
-
     private static final boolean ENABLED = LOG.isTraceEnabled();
     private static final boolean INCLUDE_HASH = true;
 
-    public static void log(Class<?> clazz, String action, Buffer buffer) {
+    public static void traceInput(
+            Class<?> clazz,
+            String action,
+            Buffer buffer,
+            InputChannelInfo channelInfo,
+            ChannelStatePersister channelStatePersister,
+            int sequenceNumber) {
+        if (ENABLED) {
+            LOG.trace(
+                    "{}#{} {}, seq {}, {} @ {}",
+                    clazz.getSimpleName(),

Review comment:
       Good catch, I didn't know that `getSimpleName` is implemented without a cache. I have combined your idea of a string to also merge the action in it. So action now contains "{simpleName}#{method}" (or anything else that identifies what Flink does with the 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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14721: [FLINK-20645][network] Fix corrupted unaligned checkpoints.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14721:
URL: https://github.com/apache/flink/pull/14721#issuecomment-764731800


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f120eb37387e00fb7ac7ef8ea628520fc2dddaa4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12339",
       "triggerID" : "f120eb37387e00fb7ac7ef8ea628520fc2dddaa4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f120eb37387e00fb7ac7ef8ea628520fc2dddaa4 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12339) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #14721: [FLINK-20645][network] Fix corrupted unaligned checkpoints.

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #14721:
URL: https://github.com/apache/flink/pull/14721#issuecomment-764710661






----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #14721: [FLINK-20645][network] Fix corrupted unaligned checkpoints.

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #14721:
URL: https://github.com/apache/flink/pull/14721#issuecomment-764710661


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit f120eb37387e00fb7ac7ef8ea628520fc2dddaa4 (Thu Jan 21 15:11:52 UTC 2021)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
    * **This pull request references an unassigned [Jira ticket](https://issues.apache.org/jira/browse/FLINK-20645).** According to the [code contribution guide](https://flink.apache.org/contributing/contribute-code.html), tickets need to be assigned before starting with the implementation work.
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] AHeise commented on a change in pull request #14721: [FLINK-20645][network] Fix corrupted unaligned checkpoints.

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #14721:
URL: https://github.com/apache/flink/pull/14721#discussion_r562466265



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/ChannelStatePersister.java
##########
@@ -62,9 +68,18 @@
     }
 
     protected void startPersisting(long barrierId, List<Buffer> knownBuffers) {
+        LOG.debug(

Review comment:
       Yes, good idea. Without `channelInfo`, it's a bit useless. (Can be probably inferred in TRACE, but the idea was that these state transitions also appear in DEBUG)
   Also I like the idea of extracting a log method.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #14721: [FLINK-20645][network] Fix corrupted unaligned checkpoints.

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #14721:
URL: https://github.com/apache/flink/pull/14721#issuecomment-764731800


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f120eb37387e00fb7ac7ef8ea628520fc2dddaa4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f120eb37387e00fb7ac7ef8ea628520fc2dddaa4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f120eb37387e00fb7ac7ef8ea628520fc2dddaa4 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14721: [FLINK-20645][network] Fix corrupted unaligned checkpoints.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14721:
URL: https://github.com/apache/flink/pull/14721#issuecomment-764731800


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "f120eb37387e00fb7ac7ef8ea628520fc2dddaa4",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12339",
       "triggerID" : "f120eb37387e00fb7ac7ef8ea628520fc2dddaa4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f120eb37387e00fb7ac7ef8ea628520fc2dddaa4 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12339) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
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:
users@infra.apache.org