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/27 20:10:02 UTC

[GitHub] [flink] dawidwys opened a new pull request #14782: [FLINK-21167] Make StateTable snapshots iterable

dawidwys opened a new pull request #14782:
URL: https://github.com/apache/flink/pull/14782


   ## What is the purpose of the change
   
   In order to implement an iterator required by a binary unified savepoint we need a way to iterate a snapshot.
   
   
   ## Verifying this change
   
   Added tests in:
   * CopyOnWriteStateMapTest
   * NestedMapsStateTableTest
   * CopyOnWriteSkipListStateMapComplexOpTest
   
   
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (**yes** / no): test dependencies
     - 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] dawidwys commented on a change in pull request #14782: [FLINK-21167] Make StateTable snapshots iterable

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/NestedStateMapSnapshot.java
##########
@@ -48,6 +53,35 @@ public NestedStateMapSnapshot(NestedStateMap<K, N, S> owningStateMap) {
         super(owningStateMap);
     }
 
+    @Override
+    public Iterator<StateEntry<K, N, S>> getIterator(
+            @Nonnull TypeSerializer<K> keySerializer,
+            @Nonnull TypeSerializer<N> namespaceSerializer,
+            @Nonnull TypeSerializer<S> stateSerializer,
+            @Nullable StateSnapshotTransformer<S> stateSnapshotTransformer) {
+        if (stateSnapshotTransformer == null) {
+            return owningStateMap.iterator();
+        } else {
+            return StreamSupport.stream(
+                            Spliterators.spliteratorUnknownSize(owningStateMap.iterator(), 0),
+                            false)
+                    .map(entry -> transformEntry(stateSnapshotTransformer, entry))
+                    .filter(Objects::nonNull)
+                    .iterator();
+        }
+    }
+
+    private StateEntry<K, N, S> transformEntry(
+            StateSnapshotTransformer<S> stateSnapshotTransformer, StateEntry<K, N, S> entry) {
+        S transformedState = stateSnapshotTransformer.filterOrTransform(entry.getState());
+        if (transformedState != null) {
+            return new StateEntry.SimpleStateEntry<>(

Review comment:
       Do you think it would work if I just add a default method in the `StateEntry` interface:
   
   ```
   public interface StateEntry<K, N, S> {
   
       /** Returns the key of this entry. */
       K getKey();
   
       /** Returns the namespace of this entry. */
       N getNamespace();
   
       /** Returns the state of this entry. */
       S getState();
   
       default StateEntry<K, N, S> filterOrTransform(StateSnapshotTransformer<S> transformer) {
           S newState = transformer.filterOrTransform(getState());
           if (newState != null) {
               return new SimpleStateEntry<>(getKey(), getNamespace(), newState);
           } else {
               return null;
           }
       }
   }
   ```
   
   The `StateMapEntry` holds additional metadata such as entryVersion, stateVersion etc. which we shouldn't mingle with during the `transformEntry`.




----------------------------------------------------------------
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] dawidwys merged pull request #14782: [FLINK-21167] Make StateTable snapshots iterable

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


   


----------------------------------------------------------------
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 #14782: [FLINK-21167] Make StateTable snapshots iterable

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d3fa4291770bf08966eafb0ca2e4d7f3ffe0912a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12570",
       "triggerID" : "d3fa4291770bf08966eafb0ca2e4d7f3ffe0912a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ecfee10472fd12f67a0fdb9feab4e45ebfe55b6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12616",
       "triggerID" : "4ecfee10472fd12f67a0fdb9feab4e45ebfe55b6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d3fa4291770bf08966eafb0ca2e4d7f3ffe0912a Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12570) 
   * 4ecfee10472fd12f67a0fdb9feab4e45ebfe55b6 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12616) 
   
   <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] aljoscha commented on a change in pull request #14782: [FLINK-21167] Make StateTable snapshots iterable

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/NestedStateMapSnapshot.java
##########
@@ -48,6 +53,35 @@ public NestedStateMapSnapshot(NestedStateMap<K, N, S> owningStateMap) {
         super(owningStateMap);
     }
 
+    @Override
+    public Iterator<StateEntry<K, N, S>> getIterator(
+            @Nonnull TypeSerializer<K> keySerializer,
+            @Nonnull TypeSerializer<N> namespaceSerializer,
+            @Nonnull TypeSerializer<S> stateSerializer,
+            @Nullable StateSnapshotTransformer<S> stateSnapshotTransformer) {
+        if (stateSnapshotTransformer == null) {
+            return owningStateMap.iterator();
+        } else {
+            return StreamSupport.stream(
+                            Spliterators.spliteratorUnknownSize(owningStateMap.iterator(), 0),
+                            false)
+                    .map(entry -> transformEntry(stateSnapshotTransformer, entry))
+                    .filter(Objects::nonNull)
+                    .iterator();
+        }
+    }
+
+    private StateEntry<K, N, S> transformEntry(
+            StateSnapshotTransformer<S> stateSnapshotTransformer, StateEntry<K, N, S> entry) {
+        S transformedState = stateSnapshotTransformer.filterOrTransform(entry.getState());
+        if (transformedState != null) {
+            return new StateEntry.SimpleStateEntry<>(

Review comment:
       Looks very good now, yes!




----------------------------------------------------------------
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 #14782: [FLINK-21167] Make StateTable snapshots iterable

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d3fa4291770bf08966eafb0ca2e4d7f3ffe0912a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12570",
       "triggerID" : "d3fa4291770bf08966eafb0ca2e4d7f3ffe0912a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d3fa4291770bf08966eafb0ca2e4d7f3ffe0912a Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12570) 
   
   <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 #14782: [FLINK-21167] Make StateTable snapshots iterable

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


   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 d3fa4291770bf08966eafb0ca2e4d7f3ffe0912a (Wed Jan 27 20:12:50 UTC 2021)
   
   **Warnings:**
    * **1 pom.xml files were touched**: Check for build and licensing issues.
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <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] dawidwys commented on a change in pull request #14782: [FLINK-21167] Make StateTable snapshots iterable

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/NestedStateMapSnapshot.java
##########
@@ -48,6 +53,35 @@ public NestedStateMapSnapshot(NestedStateMap<K, N, S> owningStateMap) {
         super(owningStateMap);
     }
 
+    @Override
+    public Iterator<StateEntry<K, N, S>> getIterator(
+            @Nonnull TypeSerializer<K> keySerializer,
+            @Nonnull TypeSerializer<N> namespaceSerializer,
+            @Nonnull TypeSerializer<S> stateSerializer,
+            @Nullable StateSnapshotTransformer<S> stateSnapshotTransformer) {
+        if (stateSnapshotTransformer == null) {
+            return owningStateMap.iterator();
+        } else {
+            return StreamSupport.stream(
+                            Spliterators.spliteratorUnknownSize(owningStateMap.iterator(), 0),
+                            false)
+                    .map(entry -> transformEntry(stateSnapshotTransformer, entry))
+                    .filter(Objects::nonNull)
+                    .iterator();
+        }
+    }
+
+    private StateEntry<K, N, S> transformEntry(
+            StateSnapshotTransformer<S> stateSnapshotTransformer, StateEntry<K, N, S> entry) {
+        S transformedState = stateSnapshotTransformer.filterOrTransform(entry.getState());
+        if (transformedState != null) {
+            return new StateEntry.SimpleStateEntry<>(

Review comment:
       Do you think it would work if I just add a default method in the `StateEntry` interface?:
   
   ```
   public interface StateEntry<K, N, S> {
   
       /** Returns the key of this entry. */
       K getKey();
   
       /** Returns the namespace of this entry. */
       N getNamespace();
   
       /** Returns the state of this entry. */
       S getState();
   
       default StateEntry<K, N, S> filterOrTransform(StateSnapshotTransformer<S> transformer) {
           S newState = transformer.filterOrTransform(getState());
           if (newState != null) {
               return new SimpleStateEntry<>(getKey(), getNamespace(), newState);
           } else {
               return null;
           }
       }
   }
   ```
   
   The `StateMapEntry` holds additional metadata such as entryVersion, stateVersion etc. which we shouldn't mingle with during the `transformEntry`.




----------------------------------------------------------------
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 #14782: [FLINK-21167] Make StateTable snapshots iterable

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d3fa4291770bf08966eafb0ca2e4d7f3ffe0912a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d3fa4291770bf08966eafb0ca2e4d7f3ffe0912a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d3fa4291770bf08966eafb0ca2e4d7f3ffe0912a 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 #14782: [FLINK-21167] Make StateTable snapshots iterable

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d3fa4291770bf08966eafb0ca2e4d7f3ffe0912a",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12570",
       "triggerID" : "d3fa4291770bf08966eafb0ca2e4d7f3ffe0912a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ecfee10472fd12f67a0fdb9feab4e45ebfe55b6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4ecfee10472fd12f67a0fdb9feab4e45ebfe55b6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d3fa4291770bf08966eafb0ca2e4d7f3ffe0912a Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12570) 
   * 4ecfee10472fd12f67a0fdb9feab4e45ebfe55b6 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 #14782: [FLINK-21167] Make StateTable snapshots iterable

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "d3fa4291770bf08966eafb0ca2e4d7f3ffe0912a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12570",
       "triggerID" : "d3fa4291770bf08966eafb0ca2e4d7f3ffe0912a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d3fa4291770bf08966eafb0ca2e4d7f3ffe0912a Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12570) 
   
   <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] aljoscha commented on a change in pull request #14782: [FLINK-21167] Make StateTable snapshots iterable

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/IterableStateSnapshot.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.annotation.Internal;
+
+import java.util.Iterator;
+
+/**
+ * A {@link StateSnapshot} that can be represented and thus iterated over {@link StateEntry
+ * StateEntries}.

Review comment:
       ```suggestion
    * A {@link StateSnapshot} that can return an iterator over all contained {@link StateEntry
    * StateEntries}.
   ```

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateMapSnapshot.java
##########
@@ -110,6 +110,19 @@ int getSnapshotVersion() {
         return snapshotVersion;
     }
 
+    @Override
+    public SnapshotIterator<K, N, S> getIterator(

Review comment:
       That was easy... 😅

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/NestedStateMapSnapshot.java
##########
@@ -48,6 +53,35 @@ public NestedStateMapSnapshot(NestedStateMap<K, N, S> owningStateMap) {
         super(owningStateMap);
     }
 
+    @Override
+    public Iterator<StateEntry<K, N, S>> getIterator(
+            @Nonnull TypeSerializer<K> keySerializer,
+            @Nonnull TypeSerializer<N> namespaceSerializer,
+            @Nonnull TypeSerializer<S> stateSerializer,
+            @Nullable StateSnapshotTransformer<S> stateSnapshotTransformer) {
+        if (stateSnapshotTransformer == null) {
+            return owningStateMap.iterator();
+        } else {
+            return StreamSupport.stream(
+                            Spliterators.spliteratorUnknownSize(owningStateMap.iterator(), 0),
+                            false)
+                    .map(entry -> transformEntry(stateSnapshotTransformer, entry))
+                    .filter(Objects::nonNull)
+                    .iterator();
+        }
+    }
+
+    private StateEntry<K, N, S> transformEntry(
+            StateSnapshotTransformer<S> stateSnapshotTransformer, StateEntry<K, N, S> entry) {
+        S transformedState = stateSnapshotTransformer.filterOrTransform(entry.getState());
+        if (transformedState != null) {
+            return new StateEntry.SimpleStateEntry<>(

Review comment:
       It seems a bit icky that we hardcode `SimpleStateEntry` here. Maybe adding a `StateEntry.filterOrTransform(StateSnapshotTransformer)` that returns a new `StateEntry` could do the trick?




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