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
This commit is contained in:
parent
09f383254c
commit
3a61d25457
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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++;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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)) {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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()),
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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) ) {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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"),
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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");
|
||||
|
||||
{
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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 =
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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(),
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -105,7 +105,7 @@ public class TestNameNodeMetrics {
|
|||
cluster.waitActive();
|
||||
namesystem = cluster.getNamesystem();
|
||||
bm = namesystem.getBlockManager();
|
||||
fs = (DistributedFileSystem) cluster.getFileSystem();
|
||||
fs = cluster.getFileSystem();
|
||||
}
|
||||
|
||||
@After
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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"));
|
||||
|
|
Loading…
Reference in New Issue