|
20 | 20 | import java.io.IOException;
|
21 | 21 | import java.util.Collection;
|
22 | 22 | import java.util.Collections;
|
| 23 | +import java.util.List; |
23 | 24 | import java.util.Map;
|
| 25 | +import java.util.stream.Collectors; |
| 26 | +import java.util.stream.StreamSupport; |
24 | 27 | import org.apache.hadoop.conf.Configuration;
|
25 | 28 | import org.apache.hadoop.fs.FileStatus;
|
26 | 29 | import org.apache.hadoop.fs.FileSystem;
|
@@ -64,8 +67,15 @@ public class SnapshotHFileCleaner extends BaseHFileCleanerDelegate {
|
64 | 67 |
|
65 | 68 | @Override
|
66 | 69 | public Iterable<FileStatus> getDeletableFiles(Iterable<FileStatus> files) {
|
| 70 | + // The Iterable is lazy evaluated, so if we just pass this Iterable in, we will access the HFile |
| 71 | + // storage inside the snapshot lock, which could take a lot of time (for example, several |
| 72 | + // seconds), and block all other operations, especially other cleaners. |
| 73 | + // So here we convert it to List first, to force it evaluated before calling |
| 74 | + // getUnreferencedFiles, so we will not hold snapshot lock for a long time. |
| 75 | + List<FileStatus> filesList = |
| 76 | + StreamSupport.stream(files.spliterator(), false).collect(Collectors.toList()); |
67 | 77 | try {
|
68 |
| - return cache.getUnreferencedFiles(files, master.getSnapshotManager()); |
| 78 | + return cache.getUnreferencedFiles(filesList, master.getSnapshotManager()); |
69 | 79 | } catch (CorruptedSnapshotException cse) {
|
70 | 80 | LOG.debug("Corrupted in-progress snapshot file exception, ignored ", cse);
|
71 | 81 | } catch (IOException e) {
|
|
0 commit comments