HDFS-8463. Calling DFSInputStream.seekToNewSource just after stream creation causes NullPointerException. Contributed by Masatake Iwasaki.
This commit is contained in:
parent
ebd797c48f
commit
ade6d9a61e
@ -846,6 +846,9 @@ Release 2.8.0 - UNRELEASED
|
||||
HDFS-3716. Purger should remove stale fsimage ckpt files
|
||||
(J.Andreina via vinayakumarb)
|
||||
|
||||
HDFS-8463. Calling DFSInputStream.seekToNewSource just after stream creation
|
||||
causes NullPointerException (Masatake Iwasaki via kihwal)
|
||||
|
||||
Release 2.7.1 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -1533,6 +1533,9 @@ private boolean seekToBlockSource(long targetPos)
|
||||
*/
|
||||
@Override
|
||||
public synchronized boolean seekToNewSource(long targetPos) throws IOException {
|
||||
if (currentNode == null) {
|
||||
return seekToBlockSource(targetPos);
|
||||
}
|
||||
boolean markedDead = deadNodes.containsKey(currentNode);
|
||||
addToDeadNodes(currentNode);
|
||||
DatanodeInfo oldNode = currentNode;
|
||||
|
@ -18,6 +18,8 @@
|
||||
package org.apache.hadoop.hdfs;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.hamcrest.CoreMatchers.equalTo;
|
||||
|
||||
import java.io.File;
|
||||
@ -28,6 +30,7 @@
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
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.junit.Assume;
|
||||
@ -111,4 +114,26 @@ public void testSkipWithLocalBlockReader() throws IOException {
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=60000)
|
||||
public void testSeekToNewSource() throws IOException {
|
||||
Configuration conf = new Configuration();
|
||||
MiniDFSCluster cluster =
|
||||
new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
|
||||
DistributedFileSystem fs = cluster.getFileSystem();
|
||||
Path path = new Path("/testfile");
|
||||
DFSTestUtil.createFile(fs, path, 1024, (short) 3, 0);
|
||||
DFSInputStream fin = fs.dfs.open("/testfile");
|
||||
try {
|
||||
fin.seekToNewSource(100);
|
||||
assertEquals(100, fin.getPos());
|
||||
DatanodeInfo firstNode = fin.getCurrentDatanode();
|
||||
assertNotNull(firstNode);
|
||||
fin.seekToNewSource(100);
|
||||
assertEquals(100, fin.getPos());
|
||||
assertFalse(firstNode.equals(fin.getCurrentDatanode()));
|
||||
} finally {
|
||||
fin.close();
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user