tillrohrmann commented on a change in pull request #6777: [FLINK-10461] [State Backends, Checkpointing] Speed up download files when restore from DFS URL: https://github.com/apache/flink/pull/6777#discussion_r238320013
########## File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDbStateDataTransfer.java ########## @@ -0,0 +1,177 @@ +/* + * 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.contrib.streaming.state; + +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.state.IncrementalKeyedStateHandle; +import org.apache.flink.runtime.state.StateHandleID; +import org.apache.flink.runtime.state.StreamStateHandle; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +/** + * Data transfer utils for RocksDbKeyedStateBackend. + */ +public class RocksDbStateDataTransfer { + + public static void transferAllStateDataToDirectory( + IncrementalKeyedStateHandle restoreStateHandle, + Path dest, + int restoringThreadNum, + CloseableRegistry closeableRegistry) throws Exception { + + final Map<StateHandleID, StreamStateHandle> sstFiles = + restoreStateHandle.getSharedState(); + final Map<StateHandleID, StreamStateHandle> miscFiles = + restoreStateHandle.getPrivateState(); + + downloadDataForAllStateHandles(sstFiles, dest, restoringThreadNum, closeableRegistry); + downloadDataForAllStateHandles(miscFiles, dest, restoringThreadNum, closeableRegistry); + } + + /** + * Copies all the files from the given stream state handles to the given path, renaming the files w.r.t. their + * {@link StateHandleID}. + */ + private static void downloadDataForAllStateHandles( + Map<StateHandleID, StreamStateHandle> stateHandleMap, + Path restoreInstancePath, + int restoringThreadNum, + CloseableRegistry closeableRegistry + ) throws Exception { + + List<Runnable> runnables = createDownloadRunnables(stateHandleMap, restoreInstancePath, closeableRegistry); + + if (enableMultiThreadDownload(restoringThreadNum)) { + runTransferWithMultipleThreads(runnables, restoringThreadNum, closeableRegistry); + } else { + runTransferWithSingleThread(runnables); + } + } + + private static List<Runnable> createDownloadRunnables( + Map<StateHandleID, StreamStateHandle> stateHandleMap, + Path restoreInstancePath, + CloseableRegistry closeableRegistry + ) { + List<Runnable> runnables = new ArrayList<>(stateHandleMap.size()); + for (Map.Entry<StateHandleID, StreamStateHandle> entry : stateHandleMap.entrySet()) { + StateHandleID stateHandleID = entry.getKey(); + StreamStateHandle remoteFileHandle = entry.getValue(); + + Path path = new Path(restoreInstancePath, stateHandleID.toString()); + runnables.add(() -> downloadDataForStateHandle(path, remoteFileHandle, closeableRegistry)); + } + return runnables; + } + + private static void runTransferWithMultipleThreads( + List<Runnable> runnables, + int threadNum, + CloseableRegistry closeableRegistry) throws ExecutionException, InterruptedException, IOException { + + final ExecutorService executorService = Executors.newFixedThreadPool(threadNum); + Closeable shutDownAllTasksAndExecutorService = () -> { + if (executorService != null) { + executorService.shutdownNow(); + } + }; + closeableRegistry.registerCloseable(shutDownAllTasksAndExecutorService); + + try { + List<CompletableFuture<Void>> futures = new ArrayList<>(runnables.size()); + for (Runnable runnable : runnables) { + CompletableFuture<Void> future = CompletableFuture.runAsync(runnable, executorService); + futures.add(future); + } + FutureUtils.waitForAll(futures).get(); + } finally { + if (closeableRegistry.unregisterCloseable(shutDownAllTasksAndExecutorService)) { + shutDownAllTasksAndExecutorService.close(); + } + } + } + + private static void runTransferWithSingleThread(List<Runnable> runnables) { + for (Runnable runnable : runnables) { + runnable.run(); + } + } + + /** + * Copies the file from a single state handle to the given path. + */ + private static void downloadDataForStateHandle( + Path restoreFilePath, + StreamStateHandle remoteFileHandle, + CloseableRegistry closeableRegistry) { + + FSDataInputStream inputStream = null; + FSDataOutputStream outputStream = null; + + try { + FileSystem restoreFileSystem = restoreFilePath.getFileSystem(); + inputStream = remoteFileHandle.openInputStream(); + closeableRegistry.registerCloseable(inputStream); + + outputStream = restoreFileSystem.create(restoreFilePath, FileSystem.WriteMode.OVERWRITE); + closeableRegistry.registerCloseable(outputStream); + + byte[] buffer = new byte[8 * 1024]; + while (true) { + int numBytes = inputStream.read(buffer); + if (numBytes == -1) { + break; + } + + outputStream.write(buffer, 0, numBytes); + } + } catch (IOException e) { + throw new RuntimeException("Copy state from DFS error.", e); + } finally { + try { + if (closeableRegistry.unregisterCloseable(inputStream)) { + inputStream.close(); + } + + if (closeableRegistry.unregisterCloseable(outputStream)) { + outputStream.close(); + } + } catch (IOException e) { + throw new RuntimeException("Close stream error.", e); + } + } + } + + private static boolean enableMultiThreadDownload(int threadNum) { Review comment: method should be called `isMultiThreadDownloadEnabled` ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services
