-
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-15413. add dfs.client.read.striped.datanode.max.attempts to fix read ecfile timeout #5829
Open
Neilxzn
wants to merge
25
commits into
apache:trunk
Choose a base branch
from
Neilxzn:fix_HDFS-15413
base: trunk
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
+272
−30
Open
Changes from all commits
Commits
Show all changes
25 commits
Select commit
Hold shift + click to select a range
5366ce9
add dfs.client.read.striped.datanode.max.attempts to fix read ec issue
e471564
fix checkstyle
ff4b113
add unit test
db52c34
add testReadFileWithAttempt for unit test
bb46dbd
fix chekstyle.
abac7c8
add dfs.client.read.striped.datanode.max.attempts to fix read ec issue
3646123
fix checkstyle
0a0b5e3
add unit test
bfc97a8
add testReadFileWithAttempt for unit test
ff3de47
fix chekstyle.
5099ee3
fix throw Exception
07a944c
Merge branch 'fix_HDFS-15413' of https://github.com/Neilxzn/hadoop in…
40dd653
add dfs.client.read.striped.datanode.max.attempts to fix read ec issue
a663760
fix checkstyle
302adf6
add unit test
35b5dc4
add testReadFileWithAttempt for unit test
59cd633
fix chekstyle.
56ee12c
fix throw Exception
cc61a87
fix checkstyle
5af4ee4
add unit test
7e4992b
Merge branch 'fix_HDFS-15413' of https://github.com/Neilxzn/hadoop in…
5d597a1
add offsetInBlock to avoid reading duplicate data from datanode.
381f5ca
fix check style && log.
c5afc65
fix tests
d698a94
fix checkstyle
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
185 changes: 185 additions & 0 deletions
185
...adoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStreamWithTimeout.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,185 @@ | ||
/** | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* <p> | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* <p> | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.apache.hadoop.hdfs; | ||
|
||
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
import org.apache.hadoop.conf.Configuration; | ||
import org.apache.hadoop.fs.Path; | ||
import org.apache.hadoop.hdfs.protocol.Block; | ||
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; | ||
import org.apache.hadoop.hdfs.protocol.LocatedBlock; | ||
import org.apache.hadoop.hdfs.protocol.LocatedBlocks; | ||
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; | ||
import org.apache.hadoop.hdfs.server.datanode.DataNode; | ||
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; | ||
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; | ||
import org.apache.hadoop.io.erasurecode.CodecUtil; | ||
import org.apache.hadoop.io.erasurecode.ErasureCodeNative; | ||
import org.apache.hadoop.io.erasurecode.rawcoder.NativeRSRawErasureCoderFactory; | ||
import org.apache.hadoop.test.GenericTestUtils; | ||
import org.junit.After; | ||
import org.junit.Assert; | ||
import org.junit.Before; | ||
import org.junit.Rule; | ||
import org.junit.Test; | ||
import org.junit.rules.Timeout; | ||
|
||
import java.io.IOException; | ||
import java.util.Arrays; | ||
|
||
public class TestDFSStripedInputStreamWithTimeout { | ||
|
||
public static final Logger LOG = | ||
LoggerFactory.getLogger(TestDFSStripedInputStreamWithTimeout.class); | ||
|
||
private MiniDFSCluster cluster; | ||
private Configuration conf = new Configuration(); | ||
private DistributedFileSystem fs; | ||
private final Path dirPath = new Path("/striped"); | ||
private Path filePath = new Path(dirPath, "file"); | ||
private ErasureCodingPolicy ecPolicy; | ||
private short dataBlocks; | ||
private short parityBlocks; | ||
private int cellSize; | ||
private final int stripesPerBlock = 2; | ||
private int blockSize; | ||
private int blockGroupSize; | ||
|
||
@Rule | ||
public Timeout globalTimeout = new Timeout(300000); | ||
|
||
public ErasureCodingPolicy getEcPolicy() { | ||
return StripedFileTestUtil.getDefaultECPolicy(); | ||
} | ||
|
||
@Before | ||
public void setup() throws IOException { | ||
/* | ||
* Initialize erasure coding policy. | ||
*/ | ||
ecPolicy = getEcPolicy(); | ||
dataBlocks = (short) ecPolicy.getNumDataUnits(); | ||
parityBlocks = (short) ecPolicy.getNumParityUnits(); | ||
cellSize = ecPolicy.getCellSize(); | ||
blockSize = stripesPerBlock * cellSize; | ||
blockGroupSize = dataBlocks * blockSize; | ||
System.out.println("EC policy = " + ecPolicy); | ||
|
||
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); | ||
Neilxzn marked this conversation as resolved.
Show resolved
Hide resolved
|
||
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); | ||
conf.setInt(DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY, 500); | ||
|
||
if (ErasureCodeNative.isNativeCodeLoaded()) { | ||
conf.set( | ||
CodecUtil.IO_ERASURECODE_CODEC_RS_RAWCODERS_KEY, | ||
NativeRSRawErasureCoderFactory.CODER_NAME); | ||
} | ||
conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, | ||
GenericTestUtils.getRandomizedTempPath()); | ||
SimulatedFSDataset.setFactory(conf); | ||
startUp(); | ||
} | ||
|
||
private void startUp() throws IOException { | ||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes( | ||
dataBlocks + parityBlocks).build(); | ||
cluster.waitActive(); | ||
for (DataNode dn : cluster.getDataNodes()) { | ||
DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true); | ||
} | ||
fs = cluster.getFileSystem(); | ||
fs.enableErasureCodingPolicy(getEcPolicy().getName()); | ||
fs.mkdirs(dirPath); | ||
fs.getClient() | ||
.setErasureCodingPolicy(dirPath.toString(), ecPolicy.getName()); | ||
} | ||
|
||
@After | ||
public void tearDown() { | ||
if (cluster != null) { | ||
cluster.shutdown(); | ||
cluster = null; | ||
} | ||
} | ||
|
||
@Test | ||
public void testPreadTimeout() throws Exception { | ||
final int numBlocks = 2; | ||
DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks, | ||
stripesPerBlock, false, ecPolicy); | ||
final int fileSize = numBlocks * blockGroupSize; | ||
|
||
LocatedBlocks lbs = fs.getClient().namenode. | ||
getBlockLocations(filePath.toString(), 0, fileSize); | ||
for (LocatedBlock lb : lbs.getLocatedBlocks()) { | ||
assert lb instanceof LocatedStripedBlock; | ||
LocatedStripedBlock bg = (LocatedStripedBlock) (lb); | ||
for (int i = 0; i < dataBlocks; i++) { | ||
Block blk = new Block(bg.getBlock().getBlockId() + i, | ||
stripesPerBlock * cellSize, | ||
bg.getBlock().getGenerationStamp()); | ||
blk.setGenerationStamp(bg.getBlock().getGenerationStamp()); | ||
cluster.injectBlocks(i, Arrays.asList(blk), | ||
bg.getBlock().getBlockPoolId()); | ||
} | ||
} | ||
try { | ||
testReadFileWithAttempt(1); | ||
Assert.fail("It Should fail to read striped time out with 1 attempt . "); | ||
} catch (Exception e) { | ||
Assert.assertTrue( | ||
"Throw IOException error message with 4 missing blocks. ", | ||
e.getMessage().contains("4 missing blocks")); | ||
} | ||
|
||
try { | ||
testReadFileWithAttempt(3); | ||
} catch (Exception e) { | ||
Assert.fail("It Should successfully read striped file with 3 attempts. "); | ||
} | ||
} | ||
|
||
private void testReadFileWithAttempt(int attempt) throws Exception { | ||
// set dfs client config | ||
cluster.getConfiguration(0) | ||
.setInt(HdfsClientConfigKeys.StripedRead.DATANODE_MAX_ATTEMPTS, | ||
attempt); | ||
cluster.getConfiguration(0) | ||
.setInt(HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 1000); | ||
DistributedFileSystem newFs = | ||
(DistributedFileSystem) cluster.getNewFileSystemInstance(0); | ||
try(DFSStripedInputStream in = new DFSStripedInputStream(newFs.getClient(), | ||
filePath.toString(), false, ecPolicy, null)){ | ||
int bufLen = 1024 * 100; | ||
byte[] buf = new byte[bufLen]; | ||
int readTotal = 0; | ||
in.seek(readTotal); | ||
int nread = in.read(buf, 0, bufLen); | ||
// Simulated time-consuming processing operations, such as UDF. | ||
// And datanodes close connect because of socket timeout. | ||
cluster.dataNodes.forEach(dn -> dn.getDatanode().closeDataXceiverServer()); | ||
in.seek(nread); | ||
// StripeRange 6MB | ||
bufLen = 1024 * 1024 * 6; | ||
buf = new byte[bufLen]; | ||
in.read(buf, 0, bufLen); | ||
} | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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 may have been different in the initial implementation, but on the first pass through the
while (true)
,curAttemtps
will already be 1 so there will be no retries by default I think?