-
Notifications
You must be signed in to change notification settings - Fork 5.3k
/
BinaryFileSpiller.java
132 lines (119 loc) · 4.66 KB
/
BinaryFileSpiller.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
/*
* Licensed 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 com.facebook.presto.spiller;
import com.facebook.presto.block.PagesSerde;
import com.facebook.presto.spi.Page;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.block.BlockEncodingSerde;
import com.google.common.io.Closer;
import com.google.common.util.concurrent.ListeningExecutorService;
import io.airlift.concurrent.MoreFutures;
import io.airlift.slice.InputStreamSliceInput;
import io.airlift.slice.OutputStreamSliceOutput;
import io.airlift.slice.RuntimeIOException;
import io.airlift.slice.SliceOutput;
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.stream.IntStream;
import java.util.stream.Stream;
import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR;
import static com.facebook.presto.util.ImmutableCollectors.toImmutableList;
import static java.util.Objects.requireNonNull;
import static java.util.stream.Collectors.toList;
public class BinaryFileSpiller
implements Spiller
{
private final Path targetDirectory;
private final Closer closer = Closer.create();
private final BlockEncodingSerde blockEncodingSerde;
private int spillsCount;
private final ListeningExecutorService executor;
public BinaryFileSpiller(BlockEncodingSerde blockEncodingSerde, ListeningExecutorService executor, Path spillPath)
{
this.blockEncodingSerde = requireNonNull(blockEncodingSerde, "blockEncodingSerde is null");
this.executor = requireNonNull(executor, "executor is null");
try {
this.targetDirectory = Files.createTempDirectory(spillPath, "presto-spill");
}
catch (IOException e) {
throw new PrestoException(GENERIC_INTERNAL_ERROR, "Failed to create spill directory", e);
}
}
@Override
public CompletableFuture<?> spill(Iterator<Page> pageIterator)
{
Path spillPath = getPath(spillsCount++);
return MoreFutures.toCompletableFuture(executor.submit(
() -> writePages(pageIterator, spillPath)
));
}
private void writePages(Iterator<Page> pageIterator, Path spillPath)
{
try (SliceOutput output = new OutputStreamSliceOutput(new BufferedOutputStream(new FileOutputStream(spillPath.toFile())))) {
PagesSerde.writePages(blockEncodingSerde, output, pageIterator);
}
catch (RuntimeIOException | IOException e) {
throw new PrestoException(GENERIC_INTERNAL_ERROR, "Failed to spill pages", e);
}
}
@Override
public List<Iterator<Page>> getSpills()
{
return IntStream.range(0, spillsCount)
.mapToObj(i -> readPages(getPath(i)))
.collect(toImmutableList());
}
private Iterator<Page> readPages(Path spillPath)
{
try {
InputStream input = new BufferedInputStream(new FileInputStream(spillPath.toFile()));
closer.register(input);
return PagesSerde.readPages(blockEncodingSerde, new InputStreamSliceInput(input));
}
catch (IOException e) {
throw new PrestoException(GENERIC_INTERNAL_ERROR, "Failed to read spilled pages", e);
}
}
@Override
public void close()
{
try (Stream<Path> list = Files.list(targetDirectory)) {
closer.close();
for (Path path : list.collect(toList())) {
Files.delete(path);
}
Files.delete(targetDirectory);
}
catch (IOException e) {
throw new PrestoException(
GENERIC_INTERNAL_ERROR,
String.format("Failed to delete directory [%s]", targetDirectory),
e);
}
}
private Path getPath(int spillNumber)
{
return Paths.get(targetDirectory.toAbsolutePath().toString(), String.format("%d.bin", spillNumber));
}
}