[ https://issues.apache.org/jira/browse/FLINK-10461?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16667360#comment-16667360 ]
ASF GitHub Bot commented on FLINK-10461: ---------------------------------------- azagrebin 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_r228970266 ########## File path: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDbStateDataTransfer.java ########## @@ -0,0 +1,149 @@ +/* + * 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.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +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 resotreThreadNum, + CloseableRegistry closeableRegistry) throws Exception { + + final Map<StateHandleID, StreamStateHandle> sstFiles = + restoreStateHandle.getSharedState(); + final Map<StateHandleID, StreamStateHandle> miscFiles = + restoreStateHandle.getPrivateState(); + + transferAllDataFromStateHandles(sstFiles, dest, resotreThreadNum, closeableRegistry); + transferAllDataFromStateHandles(miscFiles, dest, resotreThreadNum, 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 transferAllDataFromStateHandles( + Map<StateHandleID, StreamStateHandle> stateHandleMap, + Path restoreInstancePath, + int resotreThreadNum, + CloseableRegistry closeableRegistry + ) throws Exception { + + final ExecutorService executorService = Executors.newFixedThreadPool(resotreThreadNum); + List<CompletableFuture<Void>> futures = new LinkedList<>(); + List<Closeable> closeables = new LinkedList<>(); + + try { + closeables.add(() -> executorService.shutdownNow()); Review comment: I think the tasks should be first properly canceled and then `executorService` should shutdown. `executorService.shutdownNow` also calls `future.cancel` which is interrupting the task thread. At the moment `while (true)` in `copyStateDataHandleData` does not support explicit cancelation. I suggest to create an `running = AtomicBoolean(true)` in the beginning then `while (true)` can be `while(running.get())`. This will allow to cancel all tasks at once and shutdown `executorService` in one `closeable`: `closeables.add(() -> { running.set(false); executorService.shutdownNow()); };` List of `closeables` is not needed then. ---------------------------------------------------------------- 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: us...@infra.apache.org > Speed up download file procedure when restore > ---------------------------------------------- > > Key: FLINK-10461 > URL: https://issues.apache.org/jira/browse/FLINK-10461 > Project: Flink > Issue Type: Improvement > Components: State Backends, Checkpointing > Reporter: Congxian Qiu > Assignee: Congxian Qiu > Priority: Major > Labels: pull-request-available > > In the current master branch, the restore will download file from DFS, the > download procedure are single-thread, this could speed up by using > multi-thread for downloading states from DFS. > > In my company, the states will come to some terabytes, so the restore > procedure will become a litter slow, after a bit digging, I find download > states from DFS using single thread, this could using multi-thread for speed > up. > I test the time used for download states from DFS with ~2 terabytes states. > With single thread it used 640+s, and 130+s when using 5 threads for download. -- This message was sent by Atlassian JIRA (v7.6.3#76005)