|
@@ -25,6 +25,7 @@ import java.util.zip.CRC32;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
|
+import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.util.Progressable;
|
|
import org.apache.hadoop.util.Progressable;
|
|
import org.apache.hadoop.util.StringUtils;
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
|
|
|
@@ -341,28 +342,22 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Opens an FSDataOutputStream at the indicated Path with write-progress
|
|
|
|
- * reporting.
|
|
|
|
- * @param f the file name to open
|
|
|
|
- * @param overwrite if a file with this name already exists, then if true,
|
|
|
|
- * the file will be overwritten, and if false an error will be thrown.
|
|
|
|
- * @param bufferSize the size of the buffer to be used.
|
|
|
|
- * @param replication required block replication for the file.
|
|
|
|
- */
|
|
|
|
|
|
+ /** {@inheritDoc} */
|
|
@Override
|
|
@Override
|
|
- public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize,
|
|
|
|
- short replication, long blockSize, Progressable progress)
|
|
|
|
- throws IOException {
|
|
|
|
|
|
+ public FSDataOutputStream create(Path f, FsPermission permission,
|
|
|
|
+ boolean overwrite, int bufferSize, short replication, long blockSize,
|
|
|
|
+ Progressable progress) throws IOException {
|
|
Path parent = f.getParent();
|
|
Path parent = f.getParent();
|
|
if (parent != null && !mkdirs(parent)) {
|
|
if (parent != null && !mkdirs(parent)) {
|
|
throw new IOException("Mkdirs failed to create " + parent);
|
|
throw new IOException("Mkdirs failed to create " + parent);
|
|
}
|
|
}
|
|
- return new FSDataOutputStream
|
|
|
|
- (new ChecksumFSOutputSummer
|
|
|
|
- (this, f, overwrite, bufferSize, replication,
|
|
|
|
- blockSize, progress),
|
|
|
|
- null);
|
|
|
|
|
|
+ final FSDataOutputStream out = new FSDataOutputStream(
|
|
|
|
+ new ChecksumFSOutputSummer(this, f, overwrite, bufferSize, replication,
|
|
|
|
+ blockSize, progress), null);
|
|
|
|
+ if (permission != null) {
|
|
|
|
+ setPermission(f, permission);
|
|
|
|
+ }
|
|
|
|
+ return out;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|