-
Notifications
You must be signed in to change notification settings - Fork 8.9k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
HDFS-17455. Fix Client throw IndexOutOfBoundsException in DFSInputStream#fetchBlockAt #6710
Changes from 2 commits
a638efd
2c9a6fb
79945e9
cc6f05f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,6 +17,8 @@ | |
*/ | ||
package org.apache.hadoop.hdfs; | ||
|
||
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; | ||
import static org.apache.hadoop.fs.CreateFlag.CREATE; | ||
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_READ_USE_CACHE_PRIORITY; | ||
import static org.junit.Assert.assertEquals; | ||
import static org.junit.Assert.assertFalse; | ||
|
@@ -29,26 +31,37 @@ | |
import java.io.File; | ||
import java.io.IOException; | ||
import java.util.ArrayList; | ||
import java.util.EnumSet; | ||
import java.util.List; | ||
import java.util.Random; | ||
import java.util.concurrent.TimeoutException; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
|
||
import org.apache.hadoop.conf.Configuration; | ||
import org.apache.hadoop.fs.CreateFlag; | ||
import org.apache.hadoop.fs.FSDataOutputStream; | ||
import org.apache.hadoop.fs.Path; | ||
import org.apache.hadoop.fs.StorageType; | ||
import org.apache.hadoop.fs.permission.FsPermission; | ||
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; | ||
import org.apache.hadoop.hdfs.protocol.DatanodeID; | ||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; | ||
import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage; | ||
import org.apache.hadoop.hdfs.protocol.LocatedBlock; | ||
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException; | ||
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; | ||
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.apache.hadoop.test.GenericTestUtils; | ||
import org.apache.log4j.Level; | ||
import org.junit.Assume; | ||
import org.junit.Test; | ||
import org.mockito.Mockito; | ||
import org.mockito.invocation.InvocationOnMock; | ||
import org.mockito.stubbing.Answer; | ||
|
||
public class TestDFSInputStream { | ||
private void testSkipInner(MiniDFSCluster cluster) throws IOException { | ||
|
@@ -287,4 +300,74 @@ public void testReadWithoutPreferredCachingReplica() throws IOException { | |
cluster.shutdown(); | ||
} | ||
} | ||
|
||
@Test | ||
public void testCreateBlockReaderWhenInvalidBlockTokenException() throws | ||
IOException, InterruptedException, TimeoutException { | ||
GenericTestUtils.setLogLevel(DFSClient.LOG, Level.DEBUG); | ||
GenericTestUtils.LogCapturer logs = | ||
GenericTestUtils.LogCapturer.captureLogs(DFSClient.LOG); | ||
Configuration conf = new Configuration(); | ||
DFSClientFaultInjector oldFaultInjector = DFSClientFaultInjector.get(); | ||
try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build()) { | ||
cluster.waitActive(); | ||
DistributedFileSystem fs = cluster.getFileSystem(); | ||
String file = "/testfile"; | ||
Path path = new Path(file); | ||
long fileLen = 1024 * 64; | ||
EnumSet<CreateFlag> createFlags = EnumSet.of(CREATE); | ||
FSDataOutputStream out = fs.create(path, FsPermission.getFileDefault(), createFlags, | ||
fs.getConf().getInt(IO_FILE_BUFFER_SIZE_KEY, 4096), (short) 3, | ||
fs.getDefaultBlockSize(path), null); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What's the default block size? 256MB? If so, please hardcode it. |
||
int bufferLen = 1024; | ||
byte[] toWrite = new byte[bufferLen]; | ||
Random rb = new Random(0); | ||
long bytesToWrite = fileLen; | ||
while (bytesToWrite > 0) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. These logic is unnecessary if you just want to write 64KB data. 2KB is enough, right? I see you just seek to 1024. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. here will write 5KB data, ensure that the test can seek to 1024 and read 1KB of data. |
||
rb.nextBytes(toWrite); | ||
int bytesToWriteNext = (bufferLen < bytesToWrite) ? bufferLen : (int) bytesToWrite; | ||
out.write(toWrite, 0, bytesToWriteNext); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please add some comments to show that you just want to create a file which only contains one UC block. |
||
bytesToWrite -= bytesToWriteNext; | ||
} | ||
|
||
GenericTestUtils.waitFor(() -> { | ||
try { | ||
return fs.getFileBlockLocations(path, 0, fileLen).length == 1; | ||
} catch (IOException e) { | ||
return false; | ||
} | ||
}, 100, 10000); | ||
|
||
// Set up the InjectionHandler. | ||
DFSClientFaultInjector.set(Mockito.mock(DFSClientFaultInjector.class)); | ||
DFSClientFaultInjector injector = DFSClientFaultInjector.get(); | ||
final AtomicInteger count = new AtomicInteger(0); | ||
Mockito.doAnswer(new Answer<Void>() { | ||
@Override | ||
public Void answer(InvocationOnMock invocation) throws Throwable { | ||
// Mock access token was invalid when connecting to first datanode | ||
// throw InvalidBlockTokenException. | ||
if (count.getAndIncrement() == 0) { | ||
throw new InvalidBlockTokenException("Mock InvalidBlockTokenException"); | ||
} | ||
return null; | ||
} | ||
}).when(injector).failCreateBlockReader(); | ||
|
||
try (DFSInputStream in = new DFSInputStream(fs.getClient(), file, | ||
false, null)) { | ||
int bufLen = 1024; | ||
byte[] buf = new byte[bufLen]; | ||
//Seek the offset to 1024. | ||
in.seek(1024); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. add some comments to show that the offset should be in (0, fileSize). |
||
int read = in.read(buf, 0, bufLen); | ||
assertEquals(1024, read); | ||
} | ||
|
||
assertTrue(logs.getOutput().contains("Could not find target position 1")); | ||
logs.clearOutput(); | ||
} finally { | ||
DFSClientFaultInjector.set(oldFaultInjector); | ||
} | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this out should be closed in the
finally
logic, right?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1