HADOOP-14384. Reduce the visibility of FileSystem.newFSDataOutputStreamBuilder before the API becomes stable. (lei)

This commit is contained in:
Lei Xu 2017-05-09 10:27:17 -07:00
parent 543aac9f28
commit 149b903be3
3 changed files with 12 additions and 4 deletions

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.fs;
import com.google.common.base.Preconditions;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.Options.ChecksumOpt;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.util.Progressable;
@ -29,6 +31,8 @@
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
/** Base of specific file system FSDataOutputStreamBuilder. */
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class FSDataOutputStreamBuilder {
private Path path = null;
private FsPermission permission = null;
@ -41,7 +45,7 @@ public class FSDataOutputStreamBuilder{
private final FileSystem fs;
public FSDataOutputStreamBuilder(FileSystem fileSystem, Path p) {
protected FSDataOutputStreamBuilder(FileSystem fileSystem, Path p) {
fs = fileSystem;
path = p;
}

View File

@ -4144,8 +4144,12 @@ public static GlobalStorageStatistics getGlobalStorageStatistics() {
* Create a new FSDataOutputStreamBuilder for the file with path.
* @param path file path
* @return a FSDataOutputStreamBuilder object to build the file
*
* HADOOP-14384. Temporarily reduce the visibility of method before the
* builder interface becomes stable.
*/
public FSDataOutputStreamBuilder newFSDataOutputStreamBuilder(Path path) {
@InterfaceAudience.Private
protected FSDataOutputStreamBuilder newFSDataOutputStreamBuilder(Path path) {
return new FSDataOutputStreamBuilder(this, path);
}
}

View File

@ -667,7 +667,7 @@ public Collection<FileStatus> getTrashRoots(boolean allUsers) {
}
@Override
public FSDataOutputStreamBuilder newFSDataOutputStreamBuilder(Path path) {
protected FSDataOutputStreamBuilder newFSDataOutputStreamBuilder(Path path) {
return fs.newFSDataOutputStreamBuilder(path);
}
}