HDFS-4610. Use common utils FileUtil#setReadable/Writable/Executable & FileUtil#canRead/Write/Execute. Contributed by Ivan Mitic.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1477385 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Suresh Srinivas 2013-04-29 22:26:53 +00:00
parent 30b7a9c49f
commit fb30240b87
13 changed files with 53 additions and 43 deletions

View File

@ -332,6 +332,9 @@ Trunk (Unreleased)
HDFS-4734. HDFS Tests that use ShellCommandFencer are broken on Windows.
(Arpit Agarwal via suresh)
HDFS-4610. Use common utils FileUtil#setReadable/Writable/Executable &
FileUtil#canRead/Write/Execute. (Ivan Mitic via suresh)
BREAKDOWN OF HDFS-347 SUBTASKS AND RELATED JIRAS
HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.

View File

@ -448,7 +448,7 @@ public abstract class Storage extends StorageInfo {
LOG.warn(rootPath + "is not a directory");
return StorageState.NON_EXISTENT;
}
if (!root.canWrite()) {
if (!FileUtil.canWrite(root)) {
LOG.warn("Cannot access storage directory " + rootPath);
return StorageState.NON_EXISTENT;
}

View File

@ -33,6 +33,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@ -128,7 +129,7 @@ class FSImagePreTransactionalStorageInspector extends FSImageStorageInspector {
static long readCheckpointTime(StorageDirectory sd) throws IOException {
File timeFile = NNStorage.getStorageFile(sd, NameNodeFile.TIME);
long timeStamp = 0L;
if (timeFile.exists() && timeFile.canRead()) {
if (timeFile.exists() && FileUtil.canRead(timeFile)) {
DataInputStream in = new DataInputStream(new FileInputStream(timeFile));
try {
timeStamp = in.readLong();

View File

@ -34,6 +34,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
@ -230,8 +231,8 @@ public class NNStorage extends Storage implements Closeable,
File root = sd.getRoot();
LOG.info("currently disabled dir " + root.getAbsolutePath() +
"; type="+sd.getStorageDirType()
+ ";canwrite="+root.canWrite());
if(root.exists() && root.canWrite()) {
+ ";canwrite="+FileUtil.canWrite(root));
if(root.exists() && FileUtil.canWrite(root)) {
LOG.info("restoring dir " + sd.getRoot().getAbsolutePath());
this.addStorageDir(sd); // restore
this.removedStorageDirs.remove(sd);
@ -505,7 +506,7 @@ public class NNStorage extends Storage implements Closeable,
dirIterator(NameNodeDirType.IMAGE); it.hasNext();) {
sd = it.next();
File fsImage = getStorageFile(sd, NameNodeFile.IMAGE, txid);
if(sd.getRoot().canRead() && fsImage.exists())
if(FileUtil.canRead(sd.getRoot()) && fsImage.exists())
return fsImage;
}
return null;
@ -722,7 +723,7 @@ public class NNStorage extends Storage implements Closeable,
private File findFile(NameNodeDirType dirType, String name) {
for (StorageDirectory sd : dirIterable(dirType)) {
File candidate = new File(sd.getCurrentDir(), name);
if (sd.getCurrentDir().canRead() &&
if (FileUtil.canRead(sd.getCurrentDir()) &&
candidate.exists()) {
return candidate;
}

View File

@ -33,6 +33,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.http.HttpConfig;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.util.Time;
@ -102,7 +103,7 @@ public class TransferFsImage {
assert !dstFiles.isEmpty() : "No checkpoint targets.";
for (File f : dstFiles) {
if (f.exists() && f.canRead()) {
if (f.exists() && FileUtil.canRead(f)) {
LOG.info("Skipping download of remote edit log " +
log + " since it already is stored locally at " + f);
return;

View File

@ -681,9 +681,9 @@ public class MiniDFSCluster {
sb.append("\tabsolute:").append(path.getAbsolutePath()).append("\n");
sb.append("\tpermissions: ");
sb.append(path.isDirectory() ? "d": "-");
sb.append(path.canRead() ? "r" : "-");
sb.append(path.canWrite() ? "w" : "-");
sb.append(path.canExecute() ? "x" : "-");
sb.append(FileUtil.canRead(path) ? "r" : "-");
sb.append(FileUtil.canWrite(path) ? "w" : "-");
sb.append(FileUtil.canExecute(path) ? "x" : "-");
sb.append("\n");
path = path.getParentFile();
}

View File

@ -31,6 +31,7 @@ import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.BlockReader;
import org.apache.hadoop.hdfs.BlockReaderFactory;
@ -91,10 +92,10 @@ public class TestDataNodeVolumeFailure {
@After
public void tearDown() throws Exception {
if(data_fail != null) {
data_fail.setWritable(true);
FileUtil.setWritable(data_fail, true);
}
if(failedDir != null) {
failedDir.setWritable(true);
FileUtil.setWritable(failedDir, true);
}
if(cluster != null) {
cluster.shutdown();

View File

@ -31,6 +31,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
@ -88,8 +89,8 @@ public class TestDataNodeVolumeFailureReporting {
@After
public void tearDown() throws Exception {
for (int i = 0; i < 3; i++) {
new File(dataDir, "data"+(2*i+1)).setExecutable(true);
new File(dataDir, "data"+(2*i+2)).setExecutable(true);
FileUtil.setExecutable(new File(dataDir, "data"+(2*i+1)), true);
FileUtil.setExecutable(new File(dataDir, "data"+(2*i+2)), true);
}
cluster.shutdown();
}
@ -131,8 +132,8 @@ public class TestDataNodeVolumeFailureReporting {
* fail. The client does not retry failed nodes even though
* perhaps they could succeed because just a single volume failed.
*/
assertTrue("Couldn't chmod local vol", dn1Vol1.setExecutable(false));
assertTrue("Couldn't chmod local vol", dn2Vol1.setExecutable(false));
assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1, false));
assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, false));
/*
* Create file1 and wait for 3 replicas (ie all DNs can still
@ -168,7 +169,7 @@ public class TestDataNodeVolumeFailureReporting {
* Now fail a volume on the third datanode. We should be able to get
* three replicas since we've already identified the other failures.
*/
assertTrue("Couldn't chmod local vol", dn3Vol1.setExecutable(false));
assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn3Vol1, false));
Path file2 = new Path("/test2");
DFSTestUtil.createFile(fs, file2, 1024, (short)3, 1L);
DFSTestUtil.waitReplication(fs, file2, (short)3);
@ -200,7 +201,7 @@ public class TestDataNodeVolumeFailureReporting {
* and that it's no longer up. Only wait for two replicas since
* we'll never get a third.
*/
assertTrue("Couldn't chmod local vol", dn3Vol2.setExecutable(false));
assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn3Vol2, false));
Path file3 = new Path("/test3");
DFSTestUtil.createFile(fs, file3, 1024, (short)3, 1L);
DFSTestUtil.waitReplication(fs, file3, (short)2);
@ -222,10 +223,10 @@ public class TestDataNodeVolumeFailureReporting {
* restart, so file creation should be able to succeed after
* restoring the data directories and restarting the datanodes.
*/
assertTrue("Couldn't chmod local vol", dn1Vol1.setExecutable(true));
assertTrue("Couldn't chmod local vol", dn2Vol1.setExecutable(true));
assertTrue("Couldn't chmod local vol", dn3Vol1.setExecutable(true));
assertTrue("Couldn't chmod local vol", dn3Vol2.setExecutable(true));
assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1, true));
assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, true));
assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn3Vol1, true));
assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn3Vol2, true));
cluster.restartDataNodes();
cluster.waitActive();
Path file4 = new Path("/test4");
@ -261,8 +262,8 @@ public class TestDataNodeVolumeFailureReporting {
// third healthy so one node in the pipeline will not fail).
File dn1Vol1 = new File(dataDir, "data"+(2*0+1));
File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
assertTrue("Couldn't chmod local vol", dn1Vol1.setExecutable(false));
assertTrue("Couldn't chmod local vol", dn2Vol1.setExecutable(false));
assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1, false));
assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, false));
Path file1 = new Path("/test1");
DFSTestUtil.createFile(fs, file1, 1024, (short)2, 1L);

View File

@ -77,8 +77,8 @@ public class TestDataNodeVolumeFailureToleration {
@After
public void tearDown() throws Exception {
for (int i = 0; i < 3; i++) {
new File(dataDir, "data"+(2*i+1)).setExecutable(true);
new File(dataDir, "data"+(2*i+2)).setExecutable(true);
FileUtil.setExecutable(new File(dataDir, "data"+(2*i+1)), true);
FileUtil.setExecutable(new File(dataDir, "data"+(2*i+2)), true);
}
cluster.shutdown();
}
@ -152,7 +152,7 @@ public class TestDataNodeVolumeFailureToleration {
// Fail a volume on the 2nd DN
File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
assertTrue("Couldn't chmod local vol", dn2Vol1.setExecutable(false));
assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, false));
// Should only get two replicas (the first DN and the 3rd)
Path file1 = new Path("/test1");
@ -165,7 +165,7 @@ public class TestDataNodeVolumeFailureToleration {
// If we restore the volume we should still only be able to get
// two replicas since the DN is still considered dead.
assertTrue("Couldn't chmod local vol", dn2Vol1.setExecutable(true));
assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, true));
Path file2 = new Path("/test2");
DFSTestUtil.createFile(fs, file2, 1024, (short)3, 1L);
DFSTestUtil.waitReplication(fs, file2, (short)2);

View File

@ -27,6 +27,7 @@ import java.net.Socket;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSConfigKeys;
@ -106,8 +107,8 @@ public class TestDiskError {
}
} finally {
// restore its old permission
dir1.setWritable(true);
dir2.setWritable(true);
FileUtil.setWritable(dir1, true);
FileUtil.setWritable(dir2, true);
}
}

View File

@ -157,7 +157,7 @@ public class TestCheckpoint {
try {
// Simulate the mount going read-only
dir.setWritable(false);
FileUtil.setWritable(dir, false);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
.format(false).build();
fail("NN should have failed to start with " + dir + " set unreadable");
@ -167,7 +167,7 @@ public class TestCheckpoint {
} finally {
cleanup(cluster);
cluster = null;
dir.setWritable(true);
FileUtil.setWritable(dir, true);
}
}
}
@ -1825,7 +1825,7 @@ public class TestCheckpoint {
StorageDirectory sd1 = storage.getStorageDir(1);
currentDir = sd0.getCurrentDir();
currentDir.setExecutable(false);
FileUtil.setExecutable(currentDir, false);
// Upload checkpoint when NN has a bad storage dir. This should
// succeed and create the checkpoint in the good dir.
@ -1835,7 +1835,7 @@ public class TestCheckpoint {
new File(sd1.getCurrentDir(), NNStorage.getImageFileName(2)));
// Restore the good dir
currentDir.setExecutable(true);
FileUtil.setExecutable(currentDir, true);
nn.restoreFailedStorage("true");
nn.rollEditLog();
@ -1846,7 +1846,7 @@ public class TestCheckpoint {
assertParallelFilesInvariant(cluster, ImmutableList.of(secondary));
} finally {
if (currentDir != null) {
currentDir.setExecutable(true);
FileUtil.setExecutable(currentDir, true);
}
cleanup(secondary);
secondary = null;
@ -1896,7 +1896,7 @@ public class TestCheckpoint {
StorageDirectory sd0 = storage.getStorageDir(0);
assertEquals(NameNodeDirType.IMAGE, sd0.getStorageDirType());
currentDir = sd0.getCurrentDir();
currentDir.setExecutable(false);
FileUtil.setExecutable(currentDir, false);
// Try to upload checkpoint -- this should fail since there are no
// valid storage dirs
@ -1909,7 +1909,7 @@ public class TestCheckpoint {
}
// Restore the good dir
currentDir.setExecutable(true);
FileUtil.setExecutable(currentDir, true);
nn.restoreFailedStorage("true");
nn.rollEditLog();
@ -1920,7 +1920,7 @@ public class TestCheckpoint {
assertParallelFilesInvariant(cluster, ImmutableList.of(secondary));
} finally {
if (currentDir != null) {
currentDir.setExecutable(true);
FileUtil.setExecutable(currentDir, true);
}
cleanup(secondary);
secondary = null;

View File

@ -881,14 +881,14 @@ public class TestEditLog {
logDir.mkdirs();
FSEditLog log = FSImageTestUtil.createStandaloneEditLog(logDir);
try {
logDir.setWritable(false);
FileUtil.setWritable(logDir, false);
log.openForWrite();
fail("Did no throw exception on only having a bad dir");
} catch (IOException ioe) {
GenericTestUtils.assertExceptionContains(
"too few journals successfully started", ioe);
} finally {
logDir.setWritable(true);
FileUtil.setWritable(logDir, true);
log.close();
}
}

View File

@ -28,6 +28,7 @@ import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
@ -106,10 +107,10 @@ public class TestNNStorageRetentionFunctional {
getInProgressEditsFileName(5));
LOG.info("Failing first storage dir by chmodding it");
sd0.setExecutable(false);
FileUtil.setExecutable(sd0, false);
doSaveNamespace(nn);
LOG.info("Restoring accessibility of first storage dir");
sd0.setExecutable(true);
FileUtil.setExecutable(sd0, true);
LOG.info("nothing should have been purged in first storage dir");
assertGlobEquals(cd0, "fsimage_\\d*",
@ -138,7 +139,7 @@ public class TestNNStorageRetentionFunctional {
assertGlobEquals(cd0, "edits_.*",
getInProgressEditsFileName(9));
} finally {
sd0.setExecutable(true);
FileUtil.setExecutable(sd0, true);
LOG.info("Shutting down...");
if (cluster != null) {