|
@@ -48,12 +48,24 @@ final class HdfsBlobContainer extends AbstractBlobContainer {
|
|
|
private final Path path;
|
|
|
private final int bufferSize;
|
|
|
|
|
|
- HdfsBlobContainer(BlobPath blobPath, HdfsBlobStore store, Path path, int bufferSize, HdfsSecurityContext hdfsSecurityContext) {
|
|
|
+ private final Short replicationFactor;
|
|
|
+ private final Options.CreateOpts[] createOpts;
|
|
|
+
|
|
|
+ HdfsBlobContainer(
|
|
|
+ BlobPath blobPath,
|
|
|
+ HdfsBlobStore store,
|
|
|
+ Path path,
|
|
|
+ int bufferSize,
|
|
|
+ HdfsSecurityContext hdfsSecurityContext,
|
|
|
+ Short replicationFactor
|
|
|
+ ) {
|
|
|
super(blobPath);
|
|
|
this.store = store;
|
|
|
this.securityContext = hdfsSecurityContext;
|
|
|
this.path = path;
|
|
|
this.bufferSize = bufferSize;
|
|
|
+ this.replicationFactor = replicationFactor;
|
|
|
+ this.createOpts = replicationFactor == null ? new CreateOpts[0] : new CreateOpts[] { CreateOpts.repFac(replicationFactor) };
|
|
|
}
|
|
|
|
|
|
// TODO: See if we can get precise result reporting.
|
|
@@ -174,7 +186,11 @@ final class HdfsBlobContainer extends AbstractBlobContainer {
|
|
|
store.execute((Operation<Void>) fileContext -> {
|
|
|
try {
|
|
|
try (
|
|
|
- FSDataOutputStream stream = fileContext.create(tempBlobPath, EnumSet.of(CreateFlag.CREATE, CreateFlag.SYNC_BLOCK))
|
|
|
+ FSDataOutputStream stream = fileContext.create(
|
|
|
+ tempBlobPath,
|
|
|
+ EnumSet.of(CreateFlag.CREATE, CreateFlag.SYNC_BLOCK),
|
|
|
+ createOpts
|
|
|
+ )
|
|
|
) {
|
|
|
writer.accept(stream);
|
|
|
}
|
|
@@ -191,7 +207,7 @@ final class HdfsBlobContainer extends AbstractBlobContainer {
|
|
|
? EnumSet.of(CreateFlag.CREATE, CreateFlag.SYNC_BLOCK)
|
|
|
: EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE, CreateFlag.SYNC_BLOCK);
|
|
|
store.execute((Operation<Void>) fileContext -> {
|
|
|
- try (FSDataOutputStream stream = fileContext.create(blob, flags)) {
|
|
|
+ try (FSDataOutputStream stream = fileContext.create(blob, flags, createOpts)) {
|
|
|
writer.accept(stream);
|
|
|
} catch (org.apache.hadoop.fs.FileAlreadyExistsException faee) {
|
|
|
throw new FileAlreadyExistsException(blob.toString(), null, faee.getMessage());
|
|
@@ -219,7 +235,7 @@ final class HdfsBlobContainer extends AbstractBlobContainer {
|
|
|
|
|
|
private void writeToPath(BytesReference bytes, Path blobPath, FileContext fileContext, EnumSet<CreateFlag> createFlags)
|
|
|
throws IOException {
|
|
|
- try (FSDataOutputStream stream = fileContext.create(blobPath, createFlags)) {
|
|
|
+ try (FSDataOutputStream stream = fileContext.create(blobPath, createFlags, createOpts)) {
|
|
|
bytes.writeTo(stream);
|
|
|
}
|
|
|
}
|
|
@@ -232,7 +248,9 @@ final class HdfsBlobContainer extends AbstractBlobContainer {
|
|
|
EnumSet<CreateFlag> createFlags
|
|
|
) throws IOException {
|
|
|
final byte[] buffer = new byte[blobSize < bufferSize ? Math.toIntExact(blobSize) : bufferSize];
|
|
|
- try (FSDataOutputStream stream = fileContext.create(blobPath, createFlags, CreateOpts.bufferSize(buffer.length))) {
|
|
|
+
|
|
|
+ Options.CreateOpts[] createOptsWithBufferSize = addOptionToArray(createOpts, CreateOpts.bufferSize(buffer.length));
|
|
|
+ try (FSDataOutputStream stream = fileContext.create(blobPath, createFlags, createOptsWithBufferSize)) {
|
|
|
int bytesRead;
|
|
|
while ((bytesRead = inputStream.read(buffer)) != -1) {
|
|
|
stream.write(buffer, 0, bytesRead);
|
|
@@ -271,7 +289,10 @@ final class HdfsBlobContainer extends AbstractBlobContainer {
|
|
|
for (FileStatus file : files) {
|
|
|
if (file.isDirectory()) {
|
|
|
final String name = file.getPath().getName();
|
|
|
- map.put(name, new HdfsBlobContainer(path().add(name), store, new Path(path, name), bufferSize, securityContext));
|
|
|
+ map.put(
|
|
|
+ name,
|
|
|
+ new HdfsBlobContainer(path().add(name), store, new Path(path, name), bufferSize, securityContext, replicationFactor)
|
|
|
+ );
|
|
|
}
|
|
|
}
|
|
|
return Collections.unmodifiableMap(map);
|
|
@@ -328,4 +349,15 @@ final class HdfsBlobContainer extends AbstractBlobContainer {
|
|
|
) {
|
|
|
listener.onFailure(new UnsupportedOperationException("HDFS repositories do not support this operation"));
|
|
|
}
|
|
|
+
|
|
|
+ private static CreateOpts[] addOptionToArray(final CreateOpts[] opts, final CreateOpts opt) {
|
|
|
+ if (opts == null) {
|
|
|
+ return new CreateOpts[] { opt };
|
|
|
+ }
|
|
|
+ CreateOpts[] newOpts = new CreateOpts[opts.length + 1];
|
|
|
+ System.arraycopy(opts, 0, newOpts, 0, opts.length);
|
|
|
+ newOpts[opts.length] = opt;
|
|
|
+
|
|
|
+ return newOpts;
|
|
|
+ }
|
|
|
}
|