|
@@ -61,7 +61,8 @@ abstract class CommandWithDestination extends FsCommand {
|
|
|
private boolean verifyChecksum = true;
|
|
|
private boolean writeChecksum = true;
|
|
|
private boolean lazyPersist = false;
|
|
|
-
|
|
|
+ private boolean direct = false;
|
|
|
+
|
|
|
/**
|
|
|
* The name of the raw xattr namespace. It would be nice to use
|
|
|
* XAttr.RAW.name() but we can't reference the hadoop-hdfs project.
|
|
@@ -94,7 +95,11 @@ abstract class CommandWithDestination extends FsCommand {
|
|
|
protected void setWriteChecksum(boolean flag) {
|
|
|
writeChecksum = flag;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ protected void setDirectWrite(boolean flag) {
|
|
|
+ direct = flag;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* If true, the last modified time, last access time,
|
|
|
* owner, group and permission information of the source
|
|
@@ -372,9 +377,11 @@ abstract class CommandWithDestination extends FsCommand {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Copies the stream contents to a temporary file. If the copy is
|
|
|
+ * If direct write is disabled ,copies the stream contents to a temporary
|
|
|
+ * file "<target>._COPYING_". If the copy is
|
|
|
* successful, the temporary file will be renamed to the real path,
|
|
|
* else the temporary file will be deleted.
|
|
|
+ * if direct write is enabled , then creation temporary file is skipped.
|
|
|
* @param in the input stream for the copy
|
|
|
* @param target where to store the contents of the stream
|
|
|
* @throws IOException if copy fails
|
|
@@ -386,10 +393,12 @@ abstract class CommandWithDestination extends FsCommand {
|
|
|
}
|
|
|
TargetFileSystem targetFs = new TargetFileSystem(target.fs);
|
|
|
try {
|
|
|
- PathData tempTarget = target.suffix("._COPYING_");
|
|
|
+ PathData tempTarget = direct ? target : target.suffix("._COPYING_");
|
|
|
targetFs.setWriteChecksum(writeChecksum);
|
|
|
- targetFs.writeStreamToFile(in, tempTarget, lazyPersist);
|
|
|
- targetFs.rename(tempTarget, target);
|
|
|
+ targetFs.writeStreamToFile(in, tempTarget, lazyPersist, direct);
|
|
|
+ if (!direct) {
|
|
|
+ targetFs.rename(tempTarget, target);
|
|
|
+ }
|
|
|
} finally {
|
|
|
targetFs.close(); // last ditch effort to ensure temp file is removed
|
|
|
}
|
|
@@ -459,10 +468,11 @@ abstract class CommandWithDestination extends FsCommand {
|
|
|
}
|
|
|
|
|
|
void writeStreamToFile(InputStream in, PathData target,
|
|
|
- boolean lazyPersist) throws IOException {
|
|
|
+ boolean lazyPersist, boolean direct)
|
|
|
+ throws IOException {
|
|
|
FSDataOutputStream out = null;
|
|
|
try {
|
|
|
- out = create(target, lazyPersist);
|
|
|
+ out = create(target, lazyPersist, direct);
|
|
|
IOUtils.copyBytes(in, out, getConf(), true);
|
|
|
} finally {
|
|
|
IOUtils.closeStream(out); // just in case copyBytes didn't
|
|
@@ -470,7 +480,8 @@ abstract class CommandWithDestination extends FsCommand {
|
|
|
}
|
|
|
|
|
|
// tag created files as temp files
|
|
|
- FSDataOutputStream create(PathData item, boolean lazyPersist)
|
|
|
+ FSDataOutputStream create(PathData item, boolean lazyPersist,
|
|
|
+ boolean direct)
|
|
|
throws IOException {
|
|
|
try {
|
|
|
if (lazyPersist) {
|
|
@@ -488,7 +499,9 @@ abstract class CommandWithDestination extends FsCommand {
|
|
|
return create(item.path, true);
|
|
|
}
|
|
|
} finally { // might have been created but stream was interrupted
|
|
|
- deleteOnExit(item.path);
|
|
|
+ if (!direct) {
|
|
|
+ deleteOnExit(item.path);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|