Prechádzať zdrojové kódy

HDFS-6125. Cleanup unnecessary cast in HDFS code base. Contributed by Suresh Srinivas.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1581242 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 11 rokov pred
rodič
commit
3a61d25457
56 zmenil súbory, kde vykonal 90 pridanie a 90 odobranie
  1. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/EditLogLedgerMetadata.java
  3. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  4. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java
  5. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  6. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  7. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  8. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
  9. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
  10. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java
  11. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
  12. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
  13. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  14. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/FileAppendTest4.java
  15. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBalancerBandwidth.java
  16. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockMissingException.java
  17. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java
  18. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSMkdirs.java
  19. 5 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
  20. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
  21. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
  22. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
  23. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java
  24. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationEmpty.java
  25. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileLengthOnClusterRestart.java
  26. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHFlush.java
  27. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
  28. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java
  29. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMissingBlocksAlert.java
  30. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPipelines.java
  31. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplaceDatanodeOnFailure.java
  32. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestClientProtocolWithDelegationToken.java
  33. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
  34. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestCachingStrategy.java
  35. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java
  36. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTransferRbw.java
  37. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
  38. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
  39. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java
  40. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
  41. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
  42. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHDFSConcat.java
  43. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java
  44. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
  45. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
  46. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecurityTokenEditLog.java
  47. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
  48. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
  49. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNNMetricFilesInGetListingOps.java
  50. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
  51. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
  52. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java
  53. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestLightWeightHashSet.java
  54. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestLightWeightLinkedSet.java
  55. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestByteRangeInputStream.java
  56. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -256,6 +256,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 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/EditLogLedgerMetadata.java

@@ -203,7 +203,7 @@ public class EditLogLedgerMetadata {
     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;
   }
     

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

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

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java

@@ -308,7 +308,7 @@ class ClusterJspHelper {
         // 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++;
           }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

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

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -3505,7 +3505,7 @@ public abstract class FSEditLogOp {
 
     @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");
       }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -3923,7 +3923,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     assert hasReadLock();
     final BlockCollection bc = blockUC.getBlockCollection();
     if (bc == null || !(bc instanceof INodeFile)
-        || !((INodeFile) bc).isUnderConstruction()) {
+        || !bc.isUnderConstruction()) {
       return false;
     }
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java

@@ -55,7 +55,7 @@ public class FileChecksumServlets {
         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)) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java

@@ -56,7 +56,7 @@ public class FileDataServlet extends DfsServlet {
     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();
     }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/ConfiguredFailoverProxyProvider.java

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

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java

@@ -414,7 +414,7 @@ public class FSImageFormatPBSnapshot {
         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;
     }

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java

@@ -349,8 +349,7 @@ public class DelegationTokenFetcher {
          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);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

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

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/FileAppendTest4.java

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

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBalancerBandwidth.java

@@ -50,7 +50,7 @@ public class TestBalancerBandwidth {
     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.

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockMissingException.java

@@ -54,7 +54,7 @@ public class TestBlockMissingException {
     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);
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java

@@ -77,7 +77,7 @@ public class TestClientReportBadBlock {
     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);
   }
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSMkdirs.java

@@ -92,7 +92,7 @@ public class TestDFSMkdirs {
   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()),

+ 5 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java

@@ -657,8 +657,7 @@ public class TestDFSShell {
 
       // 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(
@@ -801,7 +800,7 @@ public class TestDFSShell {
   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);
 
@@ -1504,7 +1503,7 @@ public class TestDFSShell {
     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);
@@ -1533,7 +1532,7 @@ public class TestDFSShell {
       // 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");
@@ -1557,7 +1556,7 @@ public class TestDFSShell {
   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");

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java

@@ -443,7 +443,7 @@ public class TestDFSUpgradeFromImage {
         .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) ) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java

@@ -155,7 +155,7 @@ public class TestDecommission {
     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;

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java

@@ -513,7 +513,7 @@ public class TestDistributedFileSystem {
         current.getShortUserName() + "x", new String[]{"user"});
     
     try {
-      ((DistributedFileSystem) hdfs).getFileChecksum(new Path(
+      hdfs.getFileChecksum(new Path(
           "/test/TestNonExistingFile"));
       fail("Expecting FileNotFoundException");
     } catch (FileNotFoundException e) {
@@ -524,7 +524,7 @@ public class TestDistributedFileSystem {
     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()

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java

@@ -74,7 +74,7 @@ public class TestFileAppend3  {
     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

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationEmpty.java

@@ -62,7 +62,7 @@ public class TestFileCreationEmpty {
     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);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileLengthOnClusterRestart.java

@@ -44,7 +44,7 @@ public class TestFileLengthOnClusterRestart {
     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;

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHFlush.java

@@ -104,7 +104,7 @@ public class TestHFlush {
     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 class TestHFlush {
                                                .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 class TestHFlush {
     
     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 class TestHFlush {
 
     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);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java

@@ -89,7 +89,7 @@ public class TestLeaseRecovery {
       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);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java

@@ -109,7 +109,7 @@ public class TestMiniDFSCluster {
     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 {

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMissingBlocksAlert.java

@@ -62,8 +62,8 @@ public class TestMissingBlocksAlert {
       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"), 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPipelines.java

@@ -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

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplaceDatanodeOnFailure.java

@@ -121,7 +121,7 @@ public class TestReplaceDatanodeOnFailure {
         ).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];
@@ -245,7 +245,7 @@ public class TestReplaceDatanodeOnFailure {
         ).numDataNodes(1).build();
 
     try {
-      final DistributedFileSystem fs = (DistributedFileSystem)cluster.getFileSystem();
+      final DistributedFileSystem fs = cluster.getFileSystem();
       final Path f = new Path(DIR, "testAppend");
       
       {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestClientProtocolWithDelegationToken.java

@@ -102,7 +102,7 @@ public class TestClientProtocolWithDelegationToken {
       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 {

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java

@@ -184,9 +184,9 @@ public class TestHeartbeatHandling {
           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 class TestHeartbeatHandling {
               .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 class TestHeartbeatHandling {
           // 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();

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestCachingStrategy.java

@@ -79,8 +79,8 @@ public class TestCachingStrategy {
       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 @@ public class TestCachingStrategy {
       }
       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 @@ public class TestCachingStrategy {
       }
       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;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java

@@ -103,7 +103,7 @@ public class TestDataNodeMetrics {
     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);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTransferRbw.java

@@ -78,7 +78,7 @@ public class TestTransferRbw {
         ).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");

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java

@@ -126,7 +126,7 @@ public class OfflineEditsViewerHelper {
   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);
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java

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

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java

@@ -71,7 +71,7 @@ public class TestCheckPointForSecurityTokens {
           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

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java

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

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

@@ -464,7 +464,7 @@ public class TestFsck {
             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 class TestFsck {
           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 class TestFsck {
           .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 class TestFsck {
       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 = 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHDFSConcat.java

@@ -70,7 +70,7 @@ public class TestHDFSConcat {
     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);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java

@@ -361,7 +361,7 @@ public class TestNNStorageRetentionManager {
         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(),

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java

@@ -175,7 +175,7 @@ public class TestNameEditsConfigs {
     try {
       assertTrue(!fileSys.exists(file1));
       DFSTestUtil.createFile(fileSys, file1, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
-          (short) replication, SEED);
+          replication, SEED);
       checkFile(fileSys, file1, replication);
       secondary.doCheckpoint();
     } finally {
@@ -213,7 +213,7 @@ public class TestNameEditsConfigs {
       checkFile(fileSys, file1, replication);
       cleanupFile(fileSys, file1);
       DFSTestUtil.createFile(fileSys, file2, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
-          (short) replication, SEED);
+          replication, SEED);
       checkFile(fileSys, file2, replication);
       secondary.doCheckpoint();
     } finally {
@@ -250,7 +250,7 @@ public class TestNameEditsConfigs {
       checkFile(fileSys, file2, replication);
       cleanupFile(fileSys, file2);
       DFSTestUtil.createFile(fileSys, file3, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
-          (short) replication, SEED);
+          replication, SEED);
       checkFile(fileSys, file3, replication);
       secondary.doCheckpoint();
     } finally {
@@ -444,7 +444,7 @@ public class TestNameEditsConfigs {
 
       assertTrue(!fileSys.exists(file1));
       DFSTestUtil.createFile(fileSys, file1, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
-          (short) replication, SEED);
+          replication, SEED);
       checkFile(fileSys, file1, replication);
     } finally  {
       fileSys.close();
@@ -483,7 +483,7 @@ public class TestNameEditsConfigs {
       checkFile(fileSys, file1, replication);
       cleanupFile(fileSys, file1);
       DFSTestUtil.createFile(fileSys, file2, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
-          (short) replication, SEED);
+          replication, SEED);
       checkFile(fileSys, file2, replication);
     } finally {
       fileSys.close();
@@ -511,7 +511,7 @@ public class TestNameEditsConfigs {
       checkFile(fileSys, file2, replication);
       cleanupFile(fileSys, file2);
       DFSTestUtil.createFile(fileSys, file3, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
-          (short) replication, SEED);
+          replication, SEED);
       checkFile(fileSys, file3, replication);
     } finally {
       fileSys.close();
@@ -566,7 +566,7 @@ public class TestNameEditsConfigs {
       checkFile(fileSys, file3, replication);
       cleanupFile(fileSys, file3);
       DFSTestUtil.createFile(fileSys, file3, FILE_SIZE, FILE_SIZE, BLOCK_SIZE,
-          (short) replication, SEED);
+          replication, SEED);
       checkFile(fileSys, file3, replication);
     } finally {
       fileSys.close();

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java

@@ -586,7 +586,7 @@ public class TestSaveNamespace {
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(new Configuration())
         .numDataNodes(1).build();
     cluster.waitActive();
-    DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
+    DistributedFileSystem fs = cluster.getFileSystem();
     OutputStream out = null;
     try {
       fs.mkdirs(new Path("/test-target"));
@@ -609,7 +609,7 @@ public class TestSaveNamespace {
     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);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecurityTokenEditLog.java

@@ -191,9 +191,9 @@ public class TestSecurityTokenEditLog {
       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());

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java

@@ -197,7 +197,7 @@ public class TestStartup {
   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

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java

@@ -954,7 +954,7 @@ public class TestRetryCacheWithHA {
     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);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNNMetricFilesInGetListingOps.java

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

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java

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

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java

@@ -1592,7 +1592,7 @@ public class TestRenameWithSnapshots {
     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 class TestRenameWithSnapshots {
     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);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java

@@ -490,7 +490,7 @@ public class TestSnapshot {
       // ...
       //
       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);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestLightWeightHashSet.java

@@ -421,7 +421,7 @@ public class TestLightWeightHashSet{
     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");

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestLightWeightLinkedSet.java

@@ -367,7 +367,7 @@ public class TestLightWeightLinkedSet {
     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");
   }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestByteRangeInputStream.java

@@ -103,7 +103,7 @@ public static class MockHttpURLConnection extends HttpURLConnection {
     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);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java

@@ -124,8 +124,8 @@ public class TestJsonUtil {
 
     // 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"));