HDFS-12318. Fix IOException condition for openInfo in DFSInputStream. Contributed by legend.
This commit is contained in:
parent
652dd434d9
commit
3123895db0
@ -182,6 +182,11 @@ void addToDeadNodes(DatanodeInfo dnInfo) {
|
|||||||
openInfo(false);
|
openInfo(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public long getlastBlockBeingWrittenLengthForTesting() {
|
||||||
|
return lastBlockBeingWrittenLength;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Grab the open-file info from namenode
|
* Grab the open-file info from namenode
|
||||||
* @param refreshLocatedBlocks whether to re-fetch locatedblocks
|
* @param refreshLocatedBlocks whether to re-fetch locatedblocks
|
||||||
@ -209,7 +214,8 @@ void openInfo(boolean refreshLocatedBlocks) throws IOException {
|
|||||||
}
|
}
|
||||||
retriesForLastBlockLength--;
|
retriesForLastBlockLength--;
|
||||||
}
|
}
|
||||||
if (retriesForLastBlockLength == 0) {
|
if (lastBlockBeingWrittenLength == -1
|
||||||
|
&& retriesForLastBlockLength == 0) {
|
||||||
throw new IOException("Could not obtain the last block locations.");
|
throw new IOException("Could not obtain the last block locations.");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -33,6 +33,9 @@
|
|||||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||||
import org.apache.hadoop.net.unix.DomainSocket;
|
import org.apache.hadoop.net.unix.DomainSocket;
|
||||||
import org.apache.hadoop.net.unix.TemporarySocketDirectory;
|
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.Assume;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
@ -135,4 +138,42 @@ public void testSeekToNewSource() throws IOException {
|
|||||||
cluster.shutdown();
|
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();
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user