klion26 commented on a change in pull request #13309: URL: https://github.com/apache/flink/pull/13309#discussion_r482752931
########## File path: flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/functions/StatePathExtractor.java ########## @@ -0,0 +1,99 @@ +/* + * 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.functions; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.checkpoint.OperatorState; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.state.KeyGroupsStateHandle; +import org.apache.flink.runtime.state.KeyedStateHandle; +import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.runtime.state.filesystem.FileStateHandle; +import org.apache.flink.util.Collector; + +/** + * Extract from an OperatorState a set of state file paths. + */ +public class StatePathExtractor implements FlatMapFunction<OperatorState, String> { + /** + * This method recursively looks for the contained {@link FileStateHandle}s in a given {@link StreamStateHandle}. + * + * @param handle the {@code StreamStateHandle} to check for a contained {@code FileStateHandle} + * @return the file path if the given {@code StreamStateHandle} contains a {@code FileStateHandle} object, null + * otherwise + */ + private Path getStateFilePathFromStreamStateHandle(StreamStateHandle handle) { + if (handle instanceof FileStateHandle) { + return ((FileStateHandle) handle).getFilePath(); + } else if (handle instanceof OperatorStateHandle) { + return getStateFilePathFromStreamStateHandle( + ((OperatorStateHandle) handle).getDelegateStateHandle() + ); + } else if (handle instanceof KeyedStateHandle) { + if (handle instanceof KeyGroupsStateHandle) { Review comment: Do we need to comment here why skip other `KeyedStateHandle`? ########## File path: flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/functions/FileCopyRichMapFunction.java ########## @@ -0,0 +1,56 @@ +/* + * 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.functions; + +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.Path; + +import java.nio.file.Files; +import java.nio.file.Paths; + +/** + * A {@link RichMapFunction} for copying files. + */ +public class FileCopyRichMapFunction extends RichMapFunction<String, String> { + // the destination path to copy file + private String path; Review comment: can be final ########## File path: flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/functions/FileCopyRichMapFunction.java ########## @@ -0,0 +1,56 @@ +/* + * 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.functions; + +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.Path; + +import java.nio.file.Files; +import java.nio.file.Paths; + +/** + * A {@link RichMapFunction} for copying files. + */ +public class FileCopyRichMapFunction extends RichMapFunction<String, String> { + // the destination path to copy file + private String path; + + public FileCopyRichMapFunction(String path) { + this.path = path; + } + + @Override + public void open(Configuration configuration) throws Exception { + // create the parent dir only in the first subtask + if (getRuntimeContext().getIndexOfThisSubtask() == 0) { Review comment: From the implementation, we assume that the `open` function(of subtask 0) will be executed before all the `map` functions. I'm not sure this is sure or not. or can we make the directory in `WriteableSavepoint` before copying the files. ---------------------------------------------------------------- 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: [email protected]
