/
FileSystemDataset.java
328 lines (271 loc) · 9.96 KB
/
FileSystemDataset.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
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
/**
* Copyright 2013 Cloudera Inc.
*
* 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.cloudera.data.filesystem;
import com.cloudera.data.Dataset;
import com.cloudera.data.DatasetDescriptor;
import com.cloudera.data.DatasetException;
import com.cloudera.data.DatasetReader;
import com.cloudera.data.DatasetWriter;
import com.cloudera.data.FieldPartitioner;
import com.cloudera.data.Formats;
import com.cloudera.data.PartitionKey;
import com.cloudera.data.PartitionStrategy;
import com.cloudera.data.impl.Accessor;
import com.google.common.base.Joiner;
import com.google.common.base.Objects;
import com.google.common.base.Preconditions;
import com.google.common.base.Splitter;
import com.google.common.base.Supplier;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import java.io.IOException;
import java.util.List;
import javax.annotation.Nullable;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
class FileSystemDataset implements Dataset {
private static final Logger logger = LoggerFactory
.getLogger(FileSystemDataset.class);
private final FileSystem fileSystem;
private final Path directory;
private final String name;
private final DatasetDescriptor descriptor;
private final PartitionKey partitionKey;
private final PartitionStrategy partitionStrategy;
private final Schema schema;
FileSystemDataset(FileSystem fileSystem, Path directory, String name,
DatasetDescriptor descriptor, @Nullable PartitionKey partitionKey) {
this.fileSystem = fileSystem;
this.directory = directory;
this.name = name;
this.descriptor = descriptor;
this.partitionKey = partitionKey;
this.partitionStrategy =
descriptor.isPartitioned() ? descriptor.getPartitionStrategy() : null;
this.schema = descriptor.getSchema();
}
@Override
public String getName() {
return name;
}
@Override
public DatasetDescriptor getDescriptor() {
return descriptor;
}
PartitionKey getPartitionKey() {
return partitionKey;
}
FileSystem getFileSystem() {
return fileSystem;
}
Path getDirectory() {
return directory;
}
@Override
public <E> DatasetWriter<E> getWriter() {
logger.debug("Getting writer to dataset:{}", this);
DatasetWriter<E> writer;
if (descriptor.isPartitioned()) {
writer = new PartitionedDatasetWriter<E>(this);
} else {
Path dataFile = new Path(directory, uniqueFilename());
if (Formats.PARQUET.equals(descriptor.getFormat())) {
writer = new ParquetFileSystemDatasetWriter<E>(fileSystem, dataFile, schema);
} else {
writer = new FileSystemDatasetWriter.Builder<E>().fileSystem(fileSystem)
.path(dataFile).schema(schema).get();
}
}
return writer;
}
@Override
public <E> DatasetReader<E> getReader() {
logger.debug("Getting reader for dataset:{}", this);
List<Path> paths = Lists.newArrayList();
try {
accumulateDatafilePaths(directory, paths);
} catch (IOException e) {
throw new DatasetException("Unable to retrieve data file list for directory " + directory, e);
}
return new MultiFileDatasetReader<E>(fileSystem, paths, descriptor);
}
@Override
@Nullable
public Dataset getPartition(PartitionKey key, boolean allowCreate) {
Preconditions.checkState(descriptor.isPartitioned(),
"Attempt to get a partition on a non-partitioned dataset (name:%s)",
name);
logger.debug("Loading partition for key {}, allowCreate:{}", new Object[] {
key, allowCreate });
Path partitionDirectory = toDirectoryName(directory, key);
try {
if (!fileSystem.exists(partitionDirectory)) {
if (allowCreate) {
fileSystem.mkdirs(partitionDirectory);
} else {
return null;
}
}
} catch (IOException e) {
throw new DatasetException("Unable to locate or create dataset partition directory " + partitionDirectory, e);
}
int partitionDepth = key.getLength();
PartitionStrategy subpartitionStrategy = Accessor.getDefault()
.getSubpartitionStrategy(partitionStrategy, partitionDepth);
return new FileSystemDataset.Builder()
.name(name)
.fileSystem(fileSystem)
.descriptor(
new DatasetDescriptor.Builder().schema(schema).format(descriptor.getFormat())
.partitionStrategy(subpartitionStrategy).get())
.directory(partitionDirectory).partitionKey(key).get();
}
@Override
public void dropPartition(PartitionKey key) {
Preconditions.checkState(descriptor.isPartitioned(),
"Attempt to drop a partition on a non-partitioned dataset (name:%s)",
name);
Preconditions.checkArgument(key != null, "Partition key may not be null");
logger.debug("Dropping partition with key:{} dataset:{}", key, name);
Path partitionDirectory = toDirectoryName(directory, key);
try {
if (!fileSystem.delete(partitionDirectory, true)) {
throw new DatasetException("Partition directory " + partitionDirectory
+ " for key " + key + " does not exist");
}
} catch (IOException e) {
throw new DatasetException("Unable to locate or drop dataset partition directory " + partitionDirectory, e);
}
}
@Override
public Iterable<Dataset> getPartitions() {
Preconditions.checkState(descriptor.isPartitioned(),
"Attempt to get partitions on a non-partitioned dataset (name:%s)",
name);
List<Dataset> partitions = Lists.newArrayList();
FileStatus[] fileStatuses;
try {
fileStatuses = fileSystem.listStatus(directory,
PathFilters.notHidden());
} catch (IOException e) {
throw new DatasetException("Unable to list partition directory for directory " + directory, e);
}
for (FileStatus stat : fileStatuses) {
Path p = stat.getPath();
PartitionKey key = fromDirectoryName(p);
Builder builder = new FileSystemDataset.Builder()
.name(name)
.fileSystem(fileSystem)
.descriptor(
new DatasetDescriptor.Builder()
.schema(schema).format(descriptor.getFormat())
.partitionStrategy(
Accessor.getDefault().getSubpartitionStrategy(
partitionStrategy, 1)).get()).directory(p)
.partitionKey(key);
partitions.add(builder.get());
}
return partitions;
}
@Override
public String toString() {
return Objects.toStringHelper(this).add("name", name)
.add("descriptor", descriptor).add("directory", directory)
.add("dataDirectory", directory).add("partitionKey", partitionKey)
.toString();
}
private String uniqueFilename() {
// FIXME: This file name is not guaranteed to be truly unique.
return Joiner.on('-').join(System.currentTimeMillis(),
Thread.currentThread().getId() + "." + descriptor.getFormat().getExtension());
}
void accumulateDatafilePaths(Path directory, List<Path> paths)
throws IOException {
for (FileStatus status : fileSystem.listStatus(directory,
PathFilters.notHidden())) {
if (status.isDirectory()) {
accumulateDatafilePaths(status.getPath(), paths);
} else {
paths.add(status.getPath());
}
}
}
private Path toDirectoryName(Path dir, PartitionKey key) {
Path result = dir;
for (int i = 0; i < key.getLength(); i++) {
FieldPartitioner fp = partitionStrategy.getFieldPartitioners().get(i);
String fieldName = fp.getName();
String fieldValue = fp.valueToString(key.get(i));
result = new Path(result, fieldName + "=" + fieldValue);
}
return result;
}
private PartitionKey fromDirectoryName(Path dir) {
List<Object> values = Lists.newArrayList();
if (partitionKey != null) {
values.addAll(partitionKey.getValues());
}
String stringValue = Iterables.get(Splitter.on('=').split(dir.getName()), 1);
Object value = partitionStrategy.getFieldPartitioners().get(0)
.valueFromString(stringValue);
values.add(value);
return Accessor.getDefault().newPartitionKey(values.toArray());
}
public static class Builder implements Supplier<FileSystemDataset> {
private FileSystem fileSystem;
private Path directory;
private String name;
private DatasetDescriptor descriptor;
private PartitionKey partitionKey;
public Builder fileSystem(FileSystem fileSystem) {
this.fileSystem = fileSystem;
return this;
}
public Builder name(String name) {
this.name = name;
return this;
}
public Builder directory(Path directory) {
this.directory = directory;
return this;
}
public Builder descriptor(DatasetDescriptor descriptor) {
this.descriptor = descriptor;
return this;
}
Builder partitionKey(@Nullable PartitionKey partitionKey) {
this.partitionKey = partitionKey;
return this;
}
@Override
public FileSystemDataset get() {
Preconditions.checkState(this.name != null, "No dataset name defined");
Preconditions.checkState(this.descriptor != null,
"No dataset descriptor defined");
Preconditions.checkState(this.directory != null,
"No dataset directory defined");
Preconditions
.checkState(this.fileSystem != null, "No filesystem defined");
Path absoluteDirectory = fileSystem.makeQualified(directory);
return new FileSystemDataset(fileSystem, absoluteDirectory, name, descriptor,
partitionKey);
}
}
}