HDFS-6125. Merge 1581242 from trunk

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1581243 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Suresh Srinivas 2014-03-25 05:56:53 +00:00
parent 45d6277ec2
commit 590df94253
55 changed files with 82 additions and 82 deletions

View File

@ -11,6 +11,8 @@ Release 2.5.0 - UNRELEASED
HDFS-6007. Update documentation about short-circuit local reads (iwasakims
via cmccabe)
HDFS-6125. Cleanup unnecessary cast in HDFS code base. (suresh)
OPTIMIZATIONS
BUG FIXES

View File

@ -203,7 +203,7 @@ public int hashCode() {
hash = hash * 31 + (int) ledgerId;
hash = hash * 31 + (int) firstTxId;
hash = hash * 31 + (int) lastTxId;
hash = hash * 31 + (int) dataLayoutVersion;
hash = hash * 31 + dataLayoutVersion;
return hash;
}

View File

@ -143,7 +143,7 @@ public boolean trashEnabled(String bpid) {
*/
public String getTrashDirectoryForBlockFile(String bpid, File blockFile) {
if (trashEnabledBpids.contains(bpid)) {
return ((BlockPoolSliceStorage) getBPStorage(bpid)).getTrashDirectory(blockFile);
return getBPStorage(bpid).getTrashDirectory(blockFile);
}
return null;
}

View File

@ -308,7 +308,7 @@ private static void getLiveNodeCount(String json, NamenodeStatus nn)
// Inner map of attribute name to value
Map<String, Object> innerMap = entry.getValue();
if (innerMap != null) {
if (((String) innerMap.get("adminState"))
if (innerMap.get("adminState")
.equals(AdminStates.DECOMMISSIONED.toString())) {
nn.liveDecomCount++;
}

View File

@ -2396,7 +2396,7 @@ public INodeMap getINodeMap() {
*/
public final void addToInodeMap(INode inode) {
if (inode instanceof INodeWithAdditionalFields) {
inodeMap.put((INodeWithAdditionalFields)inode);
inodeMap.put(inode);
}
}

View File

@ -3507,7 +3507,7 @@ static SetAclOp getInstance() {
@Override
void readFields(DataInputStream in, int logVersion) throws IOException {
AclEditLogProto p = AclEditLogProto.parseDelimitedFrom((DataInputStream)in);
AclEditLogProto p = AclEditLogProto.parseDelimitedFrom(in);
if (p == null) {
throw new IOException("Failed to read fields from SetAclOp");
}

View File

@ -3911,7 +3911,7 @@ public boolean isInSnapshot(BlockInfoUnderConstruction blockUC) {
assert hasReadLock();
final BlockCollection bc = blockUC.getBlockCollection();
if (bc == null || !(bc instanceof INodeFile)
|| !((INodeFile) bc).isUnderConstruction()) {
|| !bc.isUnderConstruction()) {
return false;
}

View File

@ -55,7 +55,7 @@ private URL createRedirectURL(UserGroupInformation ugi, DatanodeID host,
HttpServletRequest request, NameNode nn)
throws IOException {
final String hostname = host instanceof DatanodeInfo
? ((DatanodeInfo)host).getHostName() : host.getIpAddr();
? host.getHostName() : host.getIpAddr();
final String scheme = request.getScheme();
int port = host.getInfoPort();
if ("https".equals(scheme)) {

View File

@ -56,7 +56,7 @@ private URL createRedirectURL(String path, String encodedPath, HdfsFileStatus st
final DatanodeID host = pickSrcDatanode(blks, status, conf);
final String hostname;
if (host instanceof DatanodeInfo) {
hostname = ((DatanodeInfo)host).getHostName();
hostname = host.getHostName();
} else {
hostname = host.getIpAddr();
}

View File

@ -128,7 +128,7 @@ public synchronized ProxyInfo<T> getProxy() {
throw new RuntimeException(e);
}
}
return new ProxyInfo<T>((T)current.namenode, current.address.toString());
return new ProxyInfo<T>(current.namenode, current.address.toString());
}
@Override

View File

@ -414,7 +414,7 @@ private INodeReferenceSection.INodeReference.Builder buildINodeReference(
rb.setLastSnapshotId(((WithName) ref).getLastSnapshotId()).setName(
ByteString.copyFrom(ref.getLocalNameBytes()));
} else if (ref instanceof DstReference) {
rb.setDstSnapshotId(((DstReference) ref).getDstSnapshotId());
rb.setDstSnapshotId(ref.getDstSnapshotId());
}
return rb;
}

View File

@ -346,8 +346,7 @@ static private IOException getExceptionFromResponse(HttpURLConnection con) {
Class.forName(exceptionClass).asSubclass(Exception.class);
// we are interested in constructor with String arguments
java.lang.reflect.Constructor<? extends Exception> constructor =
(java.lang.reflect.Constructor<? extends Exception>)
ec.getConstructor (new Class[] {String.class});
ec.getConstructor (new Class[] {String.class});
// create an instance
e = (IOException) constructor.newInstance (exceptionMsg);

View File

@ -317,7 +317,7 @@ private static long getLong(Map<?, ?> m, String key, final long defaultValue) {
if (value == null) {
return defaultValue;
}
return (long) (Long) value;
return (Long) value;
}
private static String getString(Map<?, ?> m, String key,

View File

@ -62,7 +62,7 @@ public static void startUp () throws IOException {
conf = new HdfsConfiguration();
init(conf);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_NUM).build();
fs = (DistributedFileSystem)cluster.getFileSystem();
fs = cluster.getFileSystem();
}
@AfterClass

View File

@ -50,7 +50,7 @@ public void testBalancerBandwidth() throws Exception {
try {
cluster.waitActive();
DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
DistributedFileSystem fs = cluster.getFileSystem();
ArrayList<DataNode> datanodes = cluster.getDataNodes();
// Ensure value from the configuration is reflected in the datanodes.

View File

@ -54,7 +54,7 @@ public void testBlockMissingException() throws Exception {
try {
dfs = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES).build();
dfs.waitActive();
fileSys = (DistributedFileSystem)dfs.getFileSystem();
fileSys = dfs.getFileSystem();
Path file1 = new Path("/user/dhruba/raidtest/file1");
createOldFile(fileSys, file1, 1, numBlocks, blockSize);

View File

@ -77,7 +77,7 @@ public void startUpCluster() throws IOException {
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes)
.build();
cluster.waitActive();
dfs = (DistributedFileSystem) cluster.getFileSystem();
dfs = cluster.getFileSystem();
buffersize = conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096);
}

View File

@ -92,7 +92,7 @@ public void testDFSMkdirs() throws IOException {
public void testMkdir() throws IOException {
Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem();
DistributedFileSystem dfs = cluster.getFileSystem();
try {
// Create a dir in root dir, should succeed
assertTrue(dfs.mkdir(new Path("/mkdir-" + Time.now()),

View File

@ -657,8 +657,7 @@ private void textTest(Path root, Configuration conf) throws Exception {
// Test a simple codec. Extension based detection. We use
// Bzip2 cause its non-native.
CompressionCodec codec = (CompressionCodec)
ReflectionUtils.newInstance(BZip2Codec.class, conf);
CompressionCodec codec = ReflectionUtils.newInstance(BZip2Codec.class, conf);
String extension = codec.getDefaultExtension();
Path p = new Path(root, "file." + extension);
OutputStream fout = new DataOutputStream(codec.createOutputStream(
@ -785,7 +784,7 @@ static String createTree(FileSystem fs, String name) throws IOException {
public void testCount() throws Exception {
Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
DistributedFileSystem dfs = cluster.getFileSystem();
FsShell shell = new FsShell();
shell.setConf(conf);
@ -1488,7 +1487,7 @@ public String run(int exitcode, String... options) throws IOException {
try {
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).format(true)
.build();
dfs = (DistributedFileSystem)cluster.getFileSystem();
dfs = cluster.getFileSystem();
mkdir(dfs, root);
dfs.copyFromLocalFile(false, false, new Path(localf.getPath()), remotef);
@ -1517,7 +1516,7 @@ public String run(int exitcode, String... options) throws IOException {
// Start the cluster again, but do not reformat, so prior files remain.
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).format(false)
.build();
dfs = (DistributedFileSystem)cluster.getFileSystem();
dfs = cluster.getFileSystem();
assertEquals(null, runner.run(1));
String corruptedcontent = runner.run(0, "-ignoreCrc");
@ -1541,7 +1540,7 @@ public String run(int exitcode, String... options) throws IOException {
public void testLsr() throws Exception {
final Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
DistributedFileSystem dfs = cluster.getFileSystem();
try {
final String root = createTree(dfs, "lsr");

View File

@ -443,7 +443,7 @@ private void upgradeAndVerify(MiniDFSCluster.Builder bld)
.clusterId("testClusterId");
cluster = bld.build();
cluster.waitActive();
DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
DistributedFileSystem dfs = cluster.getFileSystem();
DFSClient dfsClient = dfs.dfs;
//Safemode will be off only after upgrade is complete. Wait for it.
while ( dfsClient.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_GET) ) {

View File

@ -156,7 +156,7 @@ private String checkFile(FileSystem fileSys, Path name, int repl,
assertTrue("Not HDFS:"+fileSys.getUri(),
fileSys instanceof DistributedFileSystem);
HdfsDataInputStream dis = (HdfsDataInputStream)
((DistributedFileSystem)fileSys).open(name);
fileSys.open(name);
Collection<LocatedBlock> dinfo = dis.getAllBlocks();
for (LocatedBlock blk : dinfo) { // for each block
int hasdown = 0;

View File

@ -479,7 +479,7 @@ public void testFileChecksum() throws Exception {
current.getShortUserName() + "x", new String[]{"user"});
try {
((DistributedFileSystem) hdfs).getFileChecksum(new Path(
hdfs.getFileChecksum(new Path(
"/test/TestNonExistingFile"));
fail("Expecting FileNotFoundException");
} catch (FileNotFoundException e) {
@ -490,7 +490,7 @@ public void testFileChecksum() throws Exception {
try {
Path path = new Path("/test/TestExistingDir/");
hdfs.mkdirs(path);
((DistributedFileSystem) hdfs).getFileChecksum(path);
hdfs.getFileChecksum(path);
fail("Expecting FileNotFoundException");
} catch (FileNotFoundException e) {
assertTrue("Not throwing the intended exception message", e.getMessage()

View File

@ -74,7 +74,7 @@ public static void setUp() throws java.lang.Exception {
conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 512);
buffersize = conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_NUM).build();
fs = (DistributedFileSystem)cluster.getFileSystem();
fs = cluster.getFileSystem();
}
@AfterClass

View File

@ -62,7 +62,7 @@ public void uncaughtException(Thread t, Throwable e) {
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_NUM).build();
try {
cluster.waitActive();
DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
DistributedFileSystem dfs = cluster.getFileSystem();
// create a new file.
TestFileCreation.createFile(dfs, new Path("/foo"), DATANODE_NUM);

View File

@ -44,7 +44,7 @@ public void testFileLengthWithHSyncAndClusterRestartWithOutDNsRegister()
HdfsDataInputStream in = null;
try {
Path path = new Path("/tmp/TestFileLengthOnClusterRestart", "test");
DistributedFileSystem dfs = (DistributedFileSystem) cluster
DistributedFileSystem dfs = cluster
.getFileSystem();
FSDataOutputStream out = dfs.create(path);
int fileLength = 1030;

View File

@ -104,7 +104,7 @@ public void hSyncUpdateLength_00() throws IOException {
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(
2).build();
DistributedFileSystem fileSystem =
(DistributedFileSystem)cluster.getFileSystem();
cluster.getFileSystem();
try {
Path path = new Path(fName);
@ -198,7 +198,7 @@ public static void doTheJob(Configuration conf, final String fileName,
.numDataNodes(replicas).build();
// Make sure we work with DFS in order to utilize all its functionality
DistributedFileSystem fileSystem =
(DistributedFileSystem)cluster.getFileSystem();
cluster.getFileSystem();
FSDataInputStream is;
try {
@ -278,7 +278,7 @@ public void testPipelineHeartbeat() throws Exception {
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_NUM).build();
try {
DistributedFileSystem fs = (DistributedFileSystem)cluster.getFileSystem();
DistributedFileSystem fs = cluster.getFileSystem();
byte[] fileContents = AppendTestUtil.initBuffer(fileLen);
@ -328,7 +328,7 @@ public void testHFlushInterrupted() throws Exception {
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(DATANODE_NUM).build();
try {
DistributedFileSystem fs = (DistributedFileSystem)cluster.getFileSystem();
DistributedFileSystem fs = cluster.getFileSystem();
// create a new file.
FSDataOutputStream stm = AppendTestUtil.createFile(fs, p, DATANODE_NUM);

View File

@ -89,7 +89,7 @@ public void testBlockSynchronization() throws Exception {
cluster.waitActive();
//create a file
DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
DistributedFileSystem dfs = cluster.getFileSystem();
String filestr = "/foo";
Path filepath = new Path(filestr);
DFSTestUtil.createFile(dfs, filepath, ORG_FILE_SIZE, REPLICATION_NUM, 0L);

View File

@ -109,7 +109,7 @@ public void testIsClusterUpAfterShutdown() throws Throwable {
conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, c4Path);
MiniDFSCluster cluster4 = new MiniDFSCluster.Builder(conf).build();
try {
DistributedFileSystem dfs = (DistributedFileSystem) cluster4.getFileSystem();
DistributedFileSystem dfs = cluster4.getFileSystem();
dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
cluster4.shutdown();
} finally {

View File

@ -62,8 +62,8 @@ public void testMissingBlocksAlert() throws IOException,
cluster.waitActive();
final BlockManager bm = cluster.getNamesystem().getBlockManager();
DistributedFileSystem dfs =
(DistributedFileSystem) cluster.getFileSystem();
DistributedFileSystem dfs =
cluster.getFileSystem();
// create a normal file
DFSTestUtil.createFile(dfs, new Path("/testMissingBlocksAlert/file1"),

View File

@ -63,7 +63,7 @@ public class TestPipelines {
@Before
public void startUpCluster() throws IOException {
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL_FACTOR).build();
fs = (DistributedFileSystem) cluster.getFileSystem();
fs = cluster.getFileSystem();
}
@After

View File

@ -122,7 +122,7 @@ public void testReplaceDatanodeOnFailure() throws Exception {
).racks(racks).numDataNodes(REPLICATION).build();
try {
final DistributedFileSystem fs = (DistributedFileSystem)cluster.getFileSystem();
final DistributedFileSystem fs = cluster.getFileSystem();
final Path dir = new Path(DIR);
final SlowWriter[] slowwriters = new SlowWriter[10];
@ -246,7 +246,7 @@ public void testAppend() throws Exception {
).numDataNodes(1).build();
try {
final DistributedFileSystem fs = (DistributedFileSystem)cluster.getFileSystem();
final DistributedFileSystem fs = cluster.getFileSystem();
final Path f = new Path(DIR, "testAppend");
{

View File

@ -102,7 +102,7 @@ public void testDelegationTokenRpc() throws Exception {
public Object run() throws Exception {
ClientProtocol proxy = null;
try {
proxy = (ClientProtocol) RPC.getProxy(ClientProtocol.class,
proxy = RPC.getProxy(ClientProtocol.class,
ClientProtocol.versionID, addr, conf);
proxy.getServerDefaults();
} finally {

View File

@ -184,9 +184,9 @@ public void testHeartbeatBlockRecovery() throws Exception {
DatanodeInfo[] recoveringNodes = recoveryCommand.getRecoveringBlocks()
.toArray(new BlockRecoveryCommand.RecoveringBlock[0])[0].getLocations();
assertEquals(3, recoveringNodes.length);
assertEquals(recoveringNodes[0], (DatanodeInfo)dd1);
assertEquals(recoveringNodes[1], (DatanodeInfo)dd2);
assertEquals(recoveringNodes[2], (DatanodeInfo)dd3);
assertEquals(recoveringNodes[0], dd1);
assertEquals(recoveringNodes[1], dd2);
assertEquals(recoveringNodes[2], dd3);
// Test with one stale node.
dd1.setLastUpdate(System.currentTimeMillis());
@ -206,8 +206,8 @@ public void testHeartbeatBlockRecovery() throws Exception {
.toArray(new BlockRecoveryCommand.RecoveringBlock[0])[0].getLocations();
assertEquals(2, recoveringNodes.length);
// dd2 is skipped.
assertEquals(recoveringNodes[0], (DatanodeInfo)dd1);
assertEquals(recoveringNodes[1], (DatanodeInfo)dd3);
assertEquals(recoveringNodes[0], dd1);
assertEquals(recoveringNodes[1], dd3);
// Test with all stale node.
dd1.setLastUpdate(System.currentTimeMillis() - 60 * 1000);
@ -228,9 +228,9 @@ public void testHeartbeatBlockRecovery() throws Exception {
// Only dd1 is included since it heart beated and hence its not stale
// when the list of recovery blocks is constructed.
assertEquals(3, recoveringNodes.length);
assertEquals(recoveringNodes[0], (DatanodeInfo)dd1);
assertEquals(recoveringNodes[1], (DatanodeInfo)dd2);
assertEquals(recoveringNodes[2], (DatanodeInfo)dd3);
assertEquals(recoveringNodes[0], dd1);
assertEquals(recoveringNodes[1], dd2);
assertEquals(recoveringNodes[2], dd3);
}
} finally {
namesystem.writeUnlock();

View File

@ -79,8 +79,8 @@ synchronized void fadvise(int offset, int len, int flags) {
LOG.debug("got fadvise(offset=" + offset + ", len=" + len +
",flags=" + flags + ")");
if (flags == NativeIO.POSIX.POSIX_FADV_DONTNEED) {
for (int i = 0; i < (int)len; i++) {
dropped[(int)(offset + i)] = true;
for (int i = 0; i < len; i++) {
dropped[(offset + i)] = true;
}
}
}
@ -165,7 +165,7 @@ static void createHdfsFile(FileSystem fs, Path p, long length,
}
byte buf[] = new byte[8196];
while (length > 0) {
int amt = (length > buf.length) ? (int)buf.length : (int)length;
int amt = (length > buf.length) ? buf.length : (int)length;
fos.write(buf, 0, amt);
length -= amt;
}
@ -189,7 +189,7 @@ static long readHdfsFile(FileSystem fs, Path p, long length,
}
byte buf[] = new byte[8196];
while (length > 0) {
int amt = (length > buf.length) ? (int)buf.length : (int)length;
int amt = (length > buf.length) ? buf.length : (int)length;
int ret = fis.read(buf, 0, amt);
if (ret == -1) {
return totalRead;

View File

@ -103,7 +103,7 @@ public void testReceivePacketMetrics() throws Exception {
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
try {
cluster.waitActive();
DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
DistributedFileSystem fs = cluster.getFileSystem();
Path testFile = new Path("/testFlushNanosMetric.txt");
FSDataOutputStream fout = fs.create(testFile);

View File

@ -78,7 +78,7 @@ public void testTransferRbw() throws Exception {
).numDataNodes(REPLICATION).build();
try {
cluster.waitActive();
final DistributedFileSystem fs = (DistributedFileSystem)cluster.getFileSystem();
final DistributedFileSystem fs = cluster.getFileSystem();
//create a file, write some data and leave it open.
final Path p = new Path("/foo");

View File

@ -126,7 +126,7 @@ public void shutdownCluster() throws IOException {
private CheckpointSignature runOperations() throws IOException {
LOG.info("Creating edits by performing fs operations");
// no check, if it's not it throws an exception which is what we want
DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem();
DistributedFileSystem dfs = cluster.getFileSystem();
DFSTestUtil.runOperations(cluster, dfs, cluster.getConfiguration(0),
dfs.getDefaultBlockSize(), 0);

View File

@ -56,7 +56,7 @@ public static void setUp() throws Exception {
Configuration conf = new HdfsConfiguration();
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
cluster.waitActive();
hdfs = (DistributedFileSystem)cluster.getFileSystem();
hdfs = cluster.getFileSystem();
}
@AfterClass

View File

@ -71,7 +71,7 @@ public void testSaveNamespace() throws IOException {
DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDatanodes).build();
cluster.waitActive();
fs = (DistributedFileSystem)(cluster.getFileSystem());
fs = cluster.getFileSystem();
FSNamesystem namesystem = cluster.getNamesystem();
String renewer = UserGroupInformation.getLoginUser().getUserName();
Token<DelegationTokenIdentifier> token1 = namesystem

View File

@ -372,7 +372,7 @@ public void testAclOther() throws IOException {
private void addAcl(INodeWithAdditionalFields inode, AclEntry... acl)
throws IOException {
AclStorage.updateINodeAcl((INodeWithAdditionalFields)inode,
AclStorage.updateINodeAcl(inode,
Arrays.asList(acl), Snapshot.CURRENT_STATE_ID);
}

View File

@ -464,7 +464,7 @@ public void checkSalvagedRemains() throws IOException {
len = (int)(in.getFileLength() % blockSize);
if (len == 0) len = blockBuffer.length;
}
IOUtils.readFully(in, blockBuffer, 0, (int)len);
IOUtils.readFully(in, blockBuffer, 0, len);
int startIdx = blockIdx * blockSize;
for (int i = 0; i < len; i++) {
if (initialContents[startIdx + i] != blockBuffer[i]) {
@ -851,7 +851,7 @@ public void testFsckMissingReplicas() throws IOException {
new MiniDFSCluster.Builder(conf).numDataNodes(NUM_REPLICAS).build();
assertNotNull("Failed Cluster Creation", cluster);
cluster.waitClusterUp();
dfs = (DistributedFileSystem) cluster.getFileSystem();
dfs = cluster.getFileSystem();
assertNotNull("Failed to get FileSystem", dfs);
// Create a file that will be intentionally under-replicated
@ -923,7 +923,7 @@ public void testFsckMisPlacedReplicas() throws IOException {
.racks(racks).build();
assertNotNull("Failed Cluster Creation", cluster);
cluster.waitClusterUp();
dfs = (DistributedFileSystem) cluster.getFileSystem();
dfs = cluster.getFileSystem();
assertNotNull("Failed to get FileSystem", dfs);
// Create a file that will be intentionally under-replicated
@ -946,7 +946,7 @@ public void testFsckMisPlacedReplicas() throws IOException {
PrintWriter out = new PrintWriter(result, true);
InetAddress remoteAddress = InetAddress.getLocalHost();
NamenodeFsck fsck = new NamenodeFsck(conf, namenode, nettop, pmap, out,
NUM_DN, (short)REPL_FACTOR, remoteAddress);
NUM_DN, REPL_FACTOR, remoteAddress);
// Run the fsck and check the Result
final HdfsFileStatus file =

View File

@ -70,7 +70,7 @@ public void startUpCluster() throws IOException {
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPL_FACTOR).build();
assertNotNull("Failed Cluster Creation", cluster);
cluster.waitClusterUp();
dfs = (DistributedFileSystem) cluster.getFileSystem();
dfs = cluster.getFileSystem();
assertNotNull("Failed to get FileSystem", dfs);
nn = cluster.getNameNodeRpc();
assertNotNull("Failed to get NameNode", nn);

View File

@ -361,7 +361,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable {
public Void answer(InvocationOnMock invocation) throws Throwable {
Object[] args = invocation.getArguments();
journalSet.selectInputStreams((Collection<EditLogInputStream>)args[0],
(long)((Long)args[1]), (boolean)((Boolean)args[2]));
(Long)args[1], (Boolean)args[2]);
return null;
}
}).when(mockLog).selectInputStreams(Mockito.anyCollection(),

View File

@ -609,7 +609,7 @@ public void testSaveNamespaceWithDanglingLease() throws Exception {
MiniDFSCluster cluster = new MiniDFSCluster.Builder(new Configuration())
.numDataNodes(1).build();
cluster.waitActive();
DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
DistributedFileSystem fs = cluster.getFileSystem();
try {
cluster.getNamesystem().leaseManager.addLease("me", "/non-existent");
fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);

View File

@ -191,9 +191,9 @@ public void testEditsForCancelOnTokenExpire() throws IOException,
Token<DelegationTokenIdentifier> token1 = fsn.getDelegationToken(renewer);
Token<DelegationTokenIdentifier> token2 = fsn.getDelegationToken(renewer);
DelegationTokenIdentifier ident1 =
(DelegationTokenIdentifier)token1.decodeIdentifier();
token1.decodeIdentifier();
DelegationTokenIdentifier ident2 =
(DelegationTokenIdentifier)token2.decodeIdentifier();
token2.decodeIdentifier();
// verify we got the tokens
verify(log, times(1)).logGetDelegationToken(eq(ident1), anyLong());

View File

@ -212,7 +212,7 @@ private void corruptFSImageMD5(boolean corruptAll) throws IOException {
private void corruptNameNodeFiles() throws IOException {
// now corrupt/delete the directrory
List<URI> nameDirs = (List<URI>)FSNamesystem.getNamespaceDirs(config);
List<URI> nameEditsDirs = (List<URI>)FSNamesystem.getNamespaceEditsDirs(config);
List<URI> nameEditsDirs = FSNamesystem.getNamespaceEditsDirs(config);
// get name dir and its length, then delete and recreate the directory
File dir = new File(nameDirs.get(0).getPath()); // has only one

View File

@ -954,7 +954,7 @@ void invoke() throws Exception {
boolean checkNamenodeBeforeReturn() throws Exception {
for (int i = 0; i < CHECKTIMES; i++) {
RemoteIterator<CachePoolEntry> iter = dfs.listCachePools();
if (iter.hasNext() && (long)iter.next().getInfo().getLimit() == 99) {
if (iter.hasNext() && iter.next().getInfo().getLimit() == 99) {
return true;
}
Thread.sleep(1000);

View File

@ -57,7 +57,7 @@ public void setUp() throws Exception {
cluster = new MiniDFSCluster.Builder(CONF).build();
cluster.waitActive();
cluster.getNameNode();
fs = (DistributedFileSystem) cluster.getFileSystem();
fs = cluster.getFileSystem();
}
@After

View File

@ -105,7 +105,7 @@ public void setUp() throws Exception {
cluster.waitActive();
namesystem = cluster.getNamesystem();
bm = namesystem.getBlockManager();
fs = (DistributedFileSystem) cluster.getFileSystem();
fs = cluster.getFileSystem();
}
@After

View File

@ -1592,7 +1592,7 @@ public void testRenameUndo_5() throws Exception {
INode barNode = fsdir.getINode4Write(bar.toString());
assertTrue(barNode.getClass() == INodeFile.class);
assertSame(fooNode, barNode.getParent());
List<DirectoryDiff> diffList = ((INodeDirectorySnapshottable) dir1Node)
List<DirectoryDiff> diffList = dir1Node
.getDiffs().asList();
assertEquals(1, diffList.size());
DirectoryDiff diff = diffList.get(0);
@ -1666,7 +1666,7 @@ public void testRenameUndo_6() throws Exception {
INode fooNode = childrenList.get(0);
assertTrue(fooNode.asDirectory().isWithSnapshot());
assertSame(dir1Node, fooNode.getParent());
List<DirectoryDiff> diffList = ((INodeDirectorySnapshottable) dir1Node)
List<DirectoryDiff> diffList = dir1Node
.getDiffs().asList();
assertEquals(1, diffList.size());
DirectoryDiff diff = diffList.get(0);

View File

@ -490,7 +490,7 @@ private Modification[] prepareModifications(TestDirectoryTree.Node[] nodes)
// ...
//
Modification create = new FileCreation(
node.fileList.get(node.nullFileIndex), hdfs, (int) BLOCKSIZE);
node.fileList.get(node.nullFileIndex), hdfs, BLOCKSIZE);
Modification delete = new FileDeletion(
node.fileList.get((node.nullFileIndex + 1) % node.fileList.size()),
hdfs);

View File

@ -421,7 +421,7 @@ public void testOther() {
assertEquals(NUM - 10, array2.length);
for (int i = 0; i < array2.length; i++) {
assertTrue(sub2.contains((Integer) array2[i]));
assertTrue(sub2.contains(array2[i]));
}
LOG.info("Test other - DONE");

View File

@ -358,7 +358,7 @@ public void testOther() {
Object[] array2 = set.toArray();
assertEquals(NUM, array2.length);
for (int i = 0; i < array2.length; i++) {
assertTrue(list.contains((Integer) array2[i]));
assertTrue(list.contains(array2[i]));
}
LOG.info("Test capacity - DONE");
}

View File

@ -103,7 +103,7 @@ public void testByteRange() throws IOException {
doReturn(new MockHttpURLConnection(ospy.getURL())).when(ospy)
.openConnection();
HftpFileSystem.RangeHeaderUrlOpener rspy = spy(
new HftpFileSystem.RangeHeaderUrlOpener(factory, (URL) null));
new HftpFileSystem.RangeHeaderUrlOpener(factory, null));
doReturn(new MockHttpURLConnection(rspy.getURL())).when(rspy)
.openConnection();
ByteRangeInputStream is = new HftpFileSystem.RangeHeaderInputStream(ospy, rspy);

View File

@ -124,8 +124,8 @@ public void testToDatanodeInfoWithName() throws Exception {
// The encoded result should contain name, ipAddr and xferPort.
Map<String, Object> r = JsonUtil.toJsonMap(di);
Assert.assertEquals(name, (String)r.get("name"));
Assert.assertEquals("127.0.0.1", (String)r.get("ipAddr"));
Assert.assertEquals(name, r.get("name"));
Assert.assertEquals("127.0.0.1", r.get("ipAddr"));
// In this test, it is Integer instead of Long since json was not actually
// involved in constructing the map.
Assert.assertEquals(1004, (int)(Integer)r.get("xferPort"));