sjwiesman commented on a change in pull request #13309:
URL: https://github.com/apache/flink/pull/13309#discussion_r482329358



##########
File path: 
flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/WritableSavepoint.java
##########
@@ -88,40 +106,119 @@ public final void write(String path) {
                List<BootstrapTransformationWithID<?>> 
newOperatorTransformations = metadata.getNewOperators();
                DataSet<OperatorState> newOperatorStates = 
writeOperatorStates(newOperatorTransformations, savepointPath);
 
-               List<OperatorState> existingOperators = 
metadata.getExistingOperators();
+               List<OperatorState> existingOperatorStateList = 
metadata.getExistingOperators();

Review comment:
       unrelated change, please revert

##########
File path: 
flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/WritableSavepoint.java
##########
@@ -88,40 +106,119 @@ public final void write(String path) {
                List<BootstrapTransformationWithID<?>> 
newOperatorTransformations = metadata.getNewOperators();
                DataSet<OperatorState> newOperatorStates = 
writeOperatorStates(newOperatorTransformations, savepointPath);
 
-               List<OperatorState> existingOperators = 
metadata.getExistingOperators();
+               List<OperatorState> existingOperatorStateList = 
metadata.getExistingOperators();
 
-               DataSet<OperatorState> finalOperatorStates = 
unionOperatorStates(newOperatorStates, existingOperators);
+               DataSet<OperatorState> finalOperatorStates;
+               if (existingOperatorStateList.isEmpty()) {
+                       finalOperatorStates = newOperatorStates;
+               } else {
+                       DataSet<OperatorState> existingOperatorStates = 
newOperatorStates.getExecutionEnvironment()
+                               .fromCollection(existingOperatorStateList);
+
+                       existingOperatorStates
+                               .map(new OperatorState2PathSetMapFunction())
+                               .flatMap(new 
FlatMapFunction<Set<Optional<Path>>, Optional<Path>>() {
+                                       @Override
+                                       public void flatMap(Set<Optional<Path>> 
value, Collector<Optional<Path>> out) throws Exception {
+                                               for (Optional<Path> path : 
value) {
+                                                       out.collect(path);
+                                               }
+                                       }
+                               })
+                               .filter(Optional<Path>::isPresent)
+                               .map(Optional<Path>::get)
+                               .returns(TypeInformation.of(new 
TypeHint<Path>(){}))
+                               .map(new MapFunction<Path, Path>() {
+                                       @Override
+                                       public Path map(Path existingStateFile) 
throws IOException {
+                                               
savepointPath.getFileSystem().mkdirs(savepointPath);

Review comment:
       If you make this a `RichMapFunction` then this operation can be 
completed inside of open, instead of on every element.

##########
File path: 
flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointDeepCopyTest.java
##########
@@ -0,0 +1,212 @@
+/*
+ * 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.state.api;
+
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.state.api.functions.KeyedStateBootstrapFunction;
+import org.apache.flink.state.api.functions.KeyedStateReaderFunction;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.util.AbstractID;
+import org.apache.flink.util.Collector;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.File;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.hamcrest.Matchers.everyItem;
+import static org.hamcrest.Matchers.isIn;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test the savepoint deep copy.
+ */
+@RunWith(value = Parameterized.class)
+public class SavepointDeepCopyTest extends AbstractTestBase {
+
+       private static final String TEXT = "The quick brown fox jumps over the 
lazy dog";
+       private static final String RANDOM_VALUE = 
RandomStringUtils.randomAlphanumeric(120);
+
+       private final StateBackend backend;
+
+       public SavepointDeepCopyTest(StateBackend backend) throws Exception {
+               this.backend = backend;
+               //reset the cluster so we can change the state backend
+               miniClusterResource.after();
+               miniClusterResource.before();
+       }
+
+       @Parameterized.Parameters(name = "State Backend: {0}")
+       public static Collection<StateBackend> data() {
+               // set the threshold to 1024 bytes to allow generate additional 
state data files with a small state
+               int fileStateSizeThreshold = 1024;
+               return Arrays.asList(
+                       new FsStateBackend(new Path("file:///tmp").toUri(), 
fileStateSizeThreshold),
+                       new RocksDBStateBackend(
+                               (StateBackend) new FsStateBackend(new 
Path("file:///tmp").toUri(), fileStateSizeThreshold)
+                       )
+               );
+       }
+
+       /**
+        * To bootstrapper a savepoint for testing.
+        */
+       static class WordMapBootstrapper extends 
KeyedStateBootstrapFunction<String, String> {
+               private ValueState<Tuple2<String, String>> state;
+
+               @Override
+               public void open(Configuration parameters) {
+                       ValueStateDescriptor<Tuple2<String, String>> descriptor 
= new ValueStateDescriptor<>(
+                               "state",
+                               TypeInformation.of(new TypeHint<Tuple2<String, 
String>>(){}));

Review comment:
       fyi, there's an easier way to do this. 
   ```suggestion
                                Types.Tuple(Types.String, Types.String)
   ```

##########
File path: 
flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/WritableSavepoint.java
##########
@@ -88,40 +106,119 @@ public final void write(String path) {
                List<BootstrapTransformationWithID<?>> 
newOperatorTransformations = metadata.getNewOperators();
                DataSet<OperatorState> newOperatorStates = 
writeOperatorStates(newOperatorTransformations, savepointPath);
 
-               List<OperatorState> existingOperators = 
metadata.getExistingOperators();
+               List<OperatorState> existingOperatorStateList = 
metadata.getExistingOperators();
 
-               DataSet<OperatorState> finalOperatorStates = 
unionOperatorStates(newOperatorStates, existingOperators);
+               DataSet<OperatorState> finalOperatorStates;
+               if (existingOperatorStateList.isEmpty()) {
+                       finalOperatorStates = newOperatorStates;
+               } else {
+                       DataSet<OperatorState> existingOperatorStates = 
newOperatorStates.getExecutionEnvironment()
+                               .fromCollection(existingOperatorStateList);
+
+                       existingOperatorStates
+                               .map(new OperatorState2PathSetMapFunction())
+                               .flatMap(new 
FlatMapFunction<Set<Optional<Path>>, Optional<Path>>() {
+                                       @Override
+                                       public void flatMap(Set<Optional<Path>> 
value, Collector<Optional<Path>> out) throws Exception {
+                                               for (Optional<Path> path : 
value) {
+                                                       out.collect(path);
+                                               }
+                                       }
+                               })
+                               .filter(Optional<Path>::isPresent)
+                               .map(Optional<Path>::get)
+                               .returns(TypeInformation.of(new 
TypeHint<Path>(){}))
+                               .map(new MapFunction<Path, Path>() {
+                                       @Override
+                                       public Path map(Path existingStateFile) 
throws IOException {
+                                               
savepointPath.getFileSystem().mkdirs(savepointPath);

Review comment:
       Also probably only have one subtask perform the operation. 
   
   ```java
   @Override
   public void open(Configuration configuratio) throws Exception {
       if (getRuntimeContext.getIndexOfThisSubtask() == 0) {
            savepointPath.getFileSystem().mkdirs(savepointPath);
       } 
   }
   ```

##########
File path: 
flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/WritableSavepoint.java
##########
@@ -88,40 +106,119 @@ public final void write(String path) {
                List<BootstrapTransformationWithID<?>> 
newOperatorTransformations = metadata.getNewOperators();
                DataSet<OperatorState> newOperatorStates = 
writeOperatorStates(newOperatorTransformations, savepointPath);
 
-               List<OperatorState> existingOperators = 
metadata.getExistingOperators();
+               List<OperatorState> existingOperatorStateList = 
metadata.getExistingOperators();
 
-               DataSet<OperatorState> finalOperatorStates = 
unionOperatorStates(newOperatorStates, existingOperators);
+               DataSet<OperatorState> finalOperatorStates;
+               if (existingOperatorStateList.isEmpty()) {
+                       finalOperatorStates = newOperatorStates;
+               } else {
+                       DataSet<OperatorState> existingOperatorStates = 
newOperatorStates.getExecutionEnvironment()
+                               .fromCollection(existingOperatorStateList);
+
+                       existingOperatorStates
+                               .map(new OperatorState2PathSetMapFunction())
+                               .flatMap(new 
FlatMapFunction<Set<Optional<Path>>, Optional<Path>>() {
+                                       @Override
+                                       public void flatMap(Set<Optional<Path>> 
value, Collector<Optional<Path>> out) throws Exception {
+                                               for (Optional<Path> path : 
value) {
+                                                       out.collect(path);
+                                               }
+                                       }
+                               })
+                               .filter(Optional<Path>::isPresent)
+                               .map(Optional<Path>::get)
+                               .returns(TypeInformation.of(new 
TypeHint<Path>(){}))
+                               .map(new MapFunction<Path, Path>() {
+                                       @Override
+                                       public Path map(Path existingStateFile) 
throws IOException {
+                                               
savepointPath.getFileSystem().mkdirs(savepointPath);
+                                               Files.copy(
+                                                       
Paths.get(existingStateFile.getPath()), // source file
+                                                       Paths.get(path + "/" + 
existingStateFile.getName()) // destination file
+                                               );
+                                               return existingStateFile;
+                                       }
+                               })
+                               .output(new DiscardingOutputFormat<>());
+
+                       finalOperatorStates = 
newOperatorStates.union(existingOperatorStates);
 
+               }
                finalOperatorStates
                        .reduceGroup(new 
MergeOperatorStates(metadata.getMasterStates()))
                        .name("reduce(OperatorState)")
                        .output(new SavepointOutputFormat(savepointPath))
                        .name(path);
        }
 
-       private DataSet<OperatorState> 
unionOperatorStates(DataSet<OperatorState> newOperatorStates, 
List<OperatorState> existingOperators) {
-               DataSet<OperatorState> finalOperatorStates;
-               if (existingOperators.isEmpty()) {
-                       finalOperatorStates = newOperatorStates;
-               } else {
-                       DataSet<OperatorState> wrappedCollection = 
newOperatorStates
-                               .getExecutionEnvironment()
-                               .fromCollection(existingOperators);
-
-                       finalOperatorStates = 
newOperatorStates.union(wrappedCollection);
-               }
-               return finalOperatorStates;
-       }
-
        private DataSet<OperatorState> writeOperatorStates(
-                       List<BootstrapTransformationWithID<?>> 
newOperatorStates,
-                       Path savepointWritePath) {
+               List<BootstrapTransformationWithID<?>> newOperatorStates,
+               Path savepointWritePath) {
                return newOperatorStates
                        .stream()
                        .map(newOperatorState -> newOperatorState
                                .getBootstrapTransformation()
                                
.writeOperatorState(newOperatorState.getOperatorID(), stateBackend, 
metadata.getMaxParallelism(), savepointWritePath))
                        .reduce(DataSet::union)
-                       .orElseThrow(() -> new 
IllegalStateException("Savepoint's must contain at least one operator"));
+                       .orElseThrow(() -> new IllegalStateException("Savepoint 
must contain at least one operator"));
+       }
+
+       /**
+        * Map an OperatorState to a set of Paths which represent the state 
files.
+        */
+       static class OperatorState2PathSetMapFunction implements 
MapFunction<OperatorState, Set<Optional<Path>>>  {

Review comment:
       We can greatly simplify this by changing to a flatMap function that just 
returns the paths to be copied instead of this Set of Options. I'd also like to 
pass the paths as a String so we don't have to deal with Kryo typeinfo. 
   
   ```suggestion
        static class StatePathExtractor implements 
FlatMapFunction<OperatorState, String>  {
   ```

##########
File path: 
flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/WritableSavepoint.java
##########
@@ -88,40 +106,119 @@ public final void write(String path) {
                List<BootstrapTransformationWithID<?>> 
newOperatorTransformations = metadata.getNewOperators();
                DataSet<OperatorState> newOperatorStates = 
writeOperatorStates(newOperatorTransformations, savepointPath);
 
-               List<OperatorState> existingOperators = 
metadata.getExistingOperators();
+               List<OperatorState> existingOperatorStateList = 
metadata.getExistingOperators();
 
-               DataSet<OperatorState> finalOperatorStates = 
unionOperatorStates(newOperatorStates, existingOperators);
+               DataSet<OperatorState> finalOperatorStates;
+               if (existingOperatorStateList.isEmpty()) {
+                       finalOperatorStates = newOperatorStates;
+               } else {
+                       DataSet<OperatorState> existingOperatorStates = 
newOperatorStates.getExecutionEnvironment()
+                               .fromCollection(existingOperatorStateList);
+
+                       existingOperatorStates
+                               .map(new OperatorState2PathSetMapFunction())
+                               .flatMap(new 
FlatMapFunction<Set<Optional<Path>>, Optional<Path>>() {
+                                       @Override
+                                       public void flatMap(Set<Optional<Path>> 
value, Collector<Optional<Path>> out) throws Exception {
+                                               for (Optional<Path> path : 
value) {
+                                                       out.collect(path);
+                                               }
+                                       }
+                               })
+                               .filter(Optional<Path>::isPresent)
+                               .map(Optional<Path>::get)
+                               .returns(TypeInformation.of(new 
TypeHint<Path>(){}))

Review comment:
       With the other change this all goes away. 
   
   ```suggestion
                                .flatMap(new StatePathExtractor())
   ```

##########
File path: 
flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/WritableSavepoint.java
##########
@@ -88,40 +106,119 @@ public final void write(String path) {
                List<BootstrapTransformationWithID<?>> 
newOperatorTransformations = metadata.getNewOperators();
                DataSet<OperatorState> newOperatorStates = 
writeOperatorStates(newOperatorTransformations, savepointPath);
 
-               List<OperatorState> existingOperators = 
metadata.getExistingOperators();
+               List<OperatorState> existingOperatorStateList = 
metadata.getExistingOperators();
 
-               DataSet<OperatorState> finalOperatorStates = 
unionOperatorStates(newOperatorStates, existingOperators);
+               DataSet<OperatorState> finalOperatorStates;
+               if (existingOperatorStateList.isEmpty()) {
+                       finalOperatorStates = newOperatorStates;
+               } else {
+                       DataSet<OperatorState> existingOperatorStates = 
newOperatorStates.getExecutionEnvironment()
+                               .fromCollection(existingOperatorStateList);
+
+                       existingOperatorStates
+                               .map(new OperatorState2PathSetMapFunction())
+                               .flatMap(new 
FlatMapFunction<Set<Optional<Path>>, Optional<Path>>() {
+                                       @Override
+                                       public void flatMap(Set<Optional<Path>> 
value, Collector<Optional<Path>> out) throws Exception {
+                                               for (Optional<Path> path : 
value) {
+                                                       out.collect(path);
+                                               }
+                                       }
+                               })
+                               .filter(Optional<Path>::isPresent)
+                               .map(Optional<Path>::get)
+                               .returns(TypeInformation.of(new 
TypeHint<Path>(){}))
+                               .map(new MapFunction<Path, Path>() {

Review comment:
       Please make this its own class. 

##########
File path: 
flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/WritableSavepoint.java
##########
@@ -88,40 +106,119 @@ public final void write(String path) {
                List<BootstrapTransformationWithID<?>> 
newOperatorTransformations = metadata.getNewOperators();
                DataSet<OperatorState> newOperatorStates = 
writeOperatorStates(newOperatorTransformations, savepointPath);
 
-               List<OperatorState> existingOperators = 
metadata.getExistingOperators();
+               List<OperatorState> existingOperatorStateList = 
metadata.getExistingOperators();
 
-               DataSet<OperatorState> finalOperatorStates = 
unionOperatorStates(newOperatorStates, existingOperators);
+               DataSet<OperatorState> finalOperatorStates;
+               if (existingOperatorStateList.isEmpty()) {
+                       finalOperatorStates = newOperatorStates;
+               } else {
+                       DataSet<OperatorState> existingOperatorStates = 
newOperatorStates.getExecutionEnvironment()
+                               .fromCollection(existingOperatorStateList);
+
+                       existingOperatorStates
+                               .map(new OperatorState2PathSetMapFunction())
+                               .flatMap(new 
FlatMapFunction<Set<Optional<Path>>, Optional<Path>>() {
+                                       @Override
+                                       public void flatMap(Set<Optional<Path>> 
value, Collector<Optional<Path>> out) throws Exception {
+                                               for (Optional<Path> path : 
value) {
+                                                       out.collect(path);
+                                               }
+                                       }
+                               })
+                               .filter(Optional<Path>::isPresent)
+                               .map(Optional<Path>::get)
+                               .returns(TypeInformation.of(new 
TypeHint<Path>(){}))
+                               .map(new MapFunction<Path, Path>() {
+                                       @Override
+                                       public Path map(Path existingStateFile) 
throws IOException {
+                                               
savepointPath.getFileSystem().mkdirs(savepointPath);
+                                               Files.copy(
+                                                       
Paths.get(existingStateFile.getPath()), // source file
+                                                       Paths.get(path + "/" + 
existingStateFile.getName()) // destination file
+                                               );
+                                               return existingStateFile;
+                                       }
+                               })
+                               .output(new DiscardingOutputFormat<>());
+
+                       finalOperatorStates = 
newOperatorStates.union(existingOperatorStates);
 
+               }
                finalOperatorStates
                        .reduceGroup(new 
MergeOperatorStates(metadata.getMasterStates()))
                        .name("reduce(OperatorState)")
                        .output(new SavepointOutputFormat(savepointPath))
                        .name(path);
        }
 
-       private DataSet<OperatorState> 
unionOperatorStates(DataSet<OperatorState> newOperatorStates, 
List<OperatorState> existingOperators) {
-               DataSet<OperatorState> finalOperatorStates;
-               if (existingOperators.isEmpty()) {
-                       finalOperatorStates = newOperatorStates;
-               } else {
-                       DataSet<OperatorState> wrappedCollection = 
newOperatorStates
-                               .getExecutionEnvironment()
-                               .fromCollection(existingOperators);
-
-                       finalOperatorStates = 
newOperatorStates.union(wrappedCollection);
-               }
-               return finalOperatorStates;
-       }
-
        private DataSet<OperatorState> writeOperatorStates(
-                       List<BootstrapTransformationWithID<?>> 
newOperatorStates,
-                       Path savepointWritePath) {
+               List<BootstrapTransformationWithID<?>> newOperatorStates,
+               Path savepointWritePath) {
                return newOperatorStates
                        .stream()
                        .map(newOperatorState -> newOperatorState
                                .getBootstrapTransformation()
                                
.writeOperatorState(newOperatorState.getOperatorID(), stateBackend, 
metadata.getMaxParallelism(), savepointWritePath))
                        .reduce(DataSet::union)
-                       .orElseThrow(() -> new 
IllegalStateException("Savepoint's must contain at least one operator"));
+                       .orElseThrow(() -> new IllegalStateException("Savepoint 
must contain at least one operator"));
+       }
+
+       /**
+        * Map an OperatorState to a set of Paths which represent the state 
files.
+        */
+       static class OperatorState2PathSetMapFunction implements 
MapFunction<OperatorState, Set<Optional<Path>>>  {

Review comment:
       Let's make this a top level class instead of static inner




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


Reply via email to