|
@@ -33,6 +33,9 @@ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.net.unix.DomainSocket;
|
|
|
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
|
|
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
|
|
|
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Retry;
|
|
|
+
|
|
|
import org.junit.Assume;
|
|
|
import org.junit.Test;
|
|
|
|
|
@@ -136,4 +139,42 @@ public class TestDFSInputStream {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Test(timeout=60000)
|
|
|
+ public void testOpenInfo() throws IOException {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.setInt(Retry.TIMES_GET_LAST_BLOCK_LENGTH_KEY, 0);
|
|
|
+ MiniDFSCluster cluster =
|
|
|
+ new MiniDFSCluster.Builder(conf).build();
|
|
|
+ cluster.waitActive();
|
|
|
+ try {
|
|
|
+ DistributedFileSystem fs = cluster.getFileSystem();
|
|
|
+
|
|
|
+ int chunkSize = 512;
|
|
|
+ Random r = new Random(12345L);
|
|
|
+ byte[] data = new byte[chunkSize];
|
|
|
+ r.nextBytes(data);
|
|
|
+
|
|
|
+ Path file = new Path("/testfile");
|
|
|
+ try(FSDataOutputStream fout = fs.create(file)) {
|
|
|
+ fout.write(data);
|
|
|
+ }
|
|
|
+
|
|
|
+ DfsClientConf dcconf = new DfsClientConf(conf);
|
|
|
+ int retryTimesForGetLastBlockLength =
|
|
|
+ dcconf.getRetryTimesForGetLastBlockLength();
|
|
|
+ assertEquals(0, retryTimesForGetLastBlockLength);
|
|
|
+
|
|
|
+ try(DFSInputStream fin = fs.dfs.open("/testfile")) {
|
|
|
+ long flen = fin.getFileLength();
|
|
|
+ assertEquals(chunkSize, flen);
|
|
|
+
|
|
|
+ long lastBlockBeingWrittenLength =
|
|
|
+ fin.getlastBlockBeingWrittenLengthForTesting();
|
|
|
+ assertEquals(0, lastBlockBeingWrittenLength);
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|