HDFS-9289. Make DataStreamer#block thread safe and verify genStamp in commitBlock. Contributed by Chang Li.
Change-Id: If5ce1b2d212bb0726bce52ad12a3de401bcec02d
This commit is contained in:
parent
7e2829662b
commit
dac0463a4e
|
@ -351,7 +351,7 @@ class DataStreamer extends Daemon {
|
||||||
}
|
}
|
||||||
|
|
||||||
private volatile boolean streamerClosed = false;
|
private volatile boolean streamerClosed = false;
|
||||||
protected ExtendedBlock block; // its length is number of bytes acked
|
protected volatile ExtendedBlock block; // its length is number of bytes acked
|
||||||
protected Token<BlockTokenIdentifier> accessToken;
|
protected Token<BlockTokenIdentifier> accessToken;
|
||||||
private DataOutputStream blockStream;
|
private DataOutputStream blockStream;
|
||||||
private DataInputStream blockReplyStream;
|
private DataInputStream blockReplyStream;
|
||||||
|
|
|
@ -2225,6 +2225,9 @@ Release 2.8.0 - UNRELEASED
|
||||||
HDFS-9362. TestAuditLogger#testAuditLoggerWithCallContext assumes Unix line
|
HDFS-9362. TestAuditLogger#testAuditLoggerWithCallContext assumes Unix line
|
||||||
endings, fails on Windows. (cnauroth)
|
endings, fails on Windows. (cnauroth)
|
||||||
|
|
||||||
|
HDFS-9289. Make DataStreamer#block thread safe and verify genStamp in
|
||||||
|
commitBlock. (Chang Li via zhz)
|
||||||
|
|
||||||
Release 2.7.2 - UNRELEASED
|
Release 2.7.2 - UNRELEASED
|
||||||
|
|
||||||
INCOMPATIBLE CHANGES
|
INCOMPATIBLE CHANGES
|
||||||
|
|
|
@ -411,7 +411,7 @@ public abstract class BlockInfo extends Block
|
||||||
}
|
}
|
||||||
Preconditions.checkState(!isComplete());
|
Preconditions.checkState(!isComplete());
|
||||||
uc.commit();
|
uc.commit();
|
||||||
this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp());
|
this.setNumBytes(block.getNumBytes());
|
||||||
// Sort out invalid replicas.
|
// Sort out invalid replicas.
|
||||||
setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
|
setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
|
||||||
}
|
}
|
||||||
|
|
|
@ -653,6 +653,10 @@ public class BlockManager implements BlockStatsMXBean {
|
||||||
assert block.getNumBytes() <= commitBlock.getNumBytes() :
|
assert block.getNumBytes() <= commitBlock.getNumBytes() :
|
||||||
"commitBlock length is less than the stored one "
|
"commitBlock length is less than the stored one "
|
||||||
+ commitBlock.getNumBytes() + " vs. " + block.getNumBytes();
|
+ commitBlock.getNumBytes() + " vs. " + block.getNumBytes();
|
||||||
|
if(block.getGenerationStamp() != commitBlock.getGenerationStamp()) {
|
||||||
|
throw new IOException("Commit block with mismatching GS. NN has " +
|
||||||
|
block + ", client submits " + commitBlock);
|
||||||
|
}
|
||||||
block.commitBlock(commitBlock);
|
block.commitBlock(commitBlock);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
|
@ -1917,9 +1917,9 @@ public class DFSTestUtil {
|
||||||
|
|
||||||
ExtendedBlock previous = null;
|
ExtendedBlock previous = null;
|
||||||
for (int i = 0; i < numBlocks; i++) {
|
for (int i = 0; i < numBlocks; i++) {
|
||||||
Block newBlock = addStripedBlockToFile(cluster.getDataNodes(), dfs, ns,
|
Block newBlock = addBlockToFile(true, cluster.getDataNodes(), dfs, ns,
|
||||||
file.toString(), fileNode, dfs.getClient().getClientName(),
|
file.toString(), fileNode, dfs.getClient().getClientName(),
|
||||||
previous, numStripesPerBlk);
|
previous, numStripesPerBlk, 0);
|
||||||
previous = new ExtendedBlock(ns.getBlockPoolId(), newBlock);
|
previous = new ExtendedBlock(ns.getBlockPoolId(), newBlock);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1931,18 +1931,22 @@ public class DFSTestUtil {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Adds a striped block group to a file. This method only manipulates NameNode
|
* Adds a block or a striped block group to a file.
|
||||||
|
* This method only manipulates NameNode
|
||||||
* states of the file and the block without injecting data to DataNode.
|
* states of the file and the block without injecting data to DataNode.
|
||||||
* It does mimic block reports.
|
* It does mimic block reports.
|
||||||
* You should disable periodical heartbeat before use this.
|
* You should disable periodical heartbeat before use this.
|
||||||
|
* @param isStripedBlock a boolean tell if the block added a striped block
|
||||||
* @param dataNodes List DataNodes to host the striped block group
|
* @param dataNodes List DataNodes to host the striped block group
|
||||||
* @param previous Previous block in the file
|
* @param previous Previous block in the file
|
||||||
* @param numStripes Number of stripes in each block group
|
* @param numStripes Number of stripes in each block group
|
||||||
* @return The added block group
|
* @param len block size for a non striped block added
|
||||||
|
* @return The added block or block group
|
||||||
*/
|
*/
|
||||||
public static Block addStripedBlockToFile(List<DataNode> dataNodes,
|
public static Block addBlockToFile(boolean isStripedBlock,
|
||||||
DistributedFileSystem fs, FSNamesystem ns, String file, INodeFile fileNode,
|
List<DataNode> dataNodes, DistributedFileSystem fs, FSNamesystem ns,
|
||||||
String clientName, ExtendedBlock previous, int numStripes)
|
String file, INodeFile fileNode,
|
||||||
|
String clientName, ExtendedBlock previous, int numStripes, int len)
|
||||||
throws Exception {
|
throws Exception {
|
||||||
fs.getClient().namenode.addBlock(file, clientName, previous, null,
|
fs.getClient().namenode.addBlock(file, clientName, previous, null,
|
||||||
fileNode.getId(), null);
|
fileNode.getId(), null);
|
||||||
|
@ -1965,10 +1969,12 @@ public class DFSTestUtil {
|
||||||
}
|
}
|
||||||
|
|
||||||
// 2. RECEIVED_BLOCK IBR
|
// 2. RECEIVED_BLOCK IBR
|
||||||
|
long blockSize = isStripedBlock ?
|
||||||
|
numStripes * BLOCK_STRIPED_CELL_SIZE : len;
|
||||||
for (int i = 0; i < groupSize; i++) {
|
for (int i = 0; i < groupSize; i++) {
|
||||||
DataNode dn = dataNodes.get(i);
|
DataNode dn = dataNodes.get(i);
|
||||||
final Block block = new Block(lastBlock.getBlockId() + i,
|
final Block block = new Block(lastBlock.getBlockId() + i,
|
||||||
numStripes * BLOCK_STRIPED_CELL_SIZE, lastBlock.getGenerationStamp());
|
blockSize, lastBlock.getGenerationStamp());
|
||||||
DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
|
DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
|
||||||
StorageReceivedDeletedBlocks[] reports = DFSTestUtil
|
StorageReceivedDeletedBlocks[] reports = DFSTestUtil
|
||||||
.makeReportForReceivedBlock(block,
|
.makeReportForReceivedBlock(block,
|
||||||
|
@ -1977,8 +1983,9 @@ public class DFSTestUtil {
|
||||||
ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
|
ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
long bytes = isStripedBlock ?
|
||||||
lastBlock.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS);
|
numStripes * BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS : len;
|
||||||
|
lastBlock.setNumBytes(bytes);
|
||||||
return lastBlock;
|
return lastBlock;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,100 @@
|
||||||
|
/**
|
||||||
|
* 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
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* 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.server.namenode;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.fs.StorageType;
|
||||||
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||||
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||||
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.Block;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||||
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||||
|
import org.apache.hadoop.io.IOUtils;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
|
||||||
|
public class TestCommitBlockWithInvalidGenStamp {
|
||||||
|
private static final int BLOCK_SIZE = 1024;
|
||||||
|
private MiniDFSCluster cluster;
|
||||||
|
private FSDirectory dir;
|
||||||
|
private DistributedFileSystem dfs;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws IOException {
|
||||||
|
final Configuration conf = new Configuration();
|
||||||
|
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
|
||||||
|
cluster = new MiniDFSCluster.Builder(conf).build();
|
||||||
|
cluster.waitActive();
|
||||||
|
|
||||||
|
dir = cluster.getNamesystem().getFSDirectory();
|
||||||
|
dfs = cluster.getFileSystem();
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void tearDown() {
|
||||||
|
if (cluster != null) {
|
||||||
|
cluster.shutdown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCommitWithInvalidGenStamp() throws Exception {
|
||||||
|
final Path file = new Path("/file");
|
||||||
|
FSDataOutputStream out = null;
|
||||||
|
|
||||||
|
try {
|
||||||
|
out = dfs.create(file, (short) 1);
|
||||||
|
INodeFile fileNode = dir.getINode4Write(file.toString()).asFile();
|
||||||
|
ExtendedBlock previous = null;
|
||||||
|
|
||||||
|
Block newBlock = DFSTestUtil.addBlockToFile(false, cluster.getDataNodes(),
|
||||||
|
dfs, cluster.getNamesystem(), file.toString(), fileNode,
|
||||||
|
dfs.getClient().getClientName(), previous, 0, 100);
|
||||||
|
Block newBlockClone = new Block(newBlock);
|
||||||
|
previous = new ExtendedBlock(cluster.getNamesystem().getBlockPoolId(),
|
||||||
|
newBlockClone);
|
||||||
|
|
||||||
|
previous.setGenerationStamp(123);
|
||||||
|
try{
|
||||||
|
dfs.getClient().getNamenode().complete(file.toString(),
|
||||||
|
dfs.getClient().getClientName(), previous, fileNode.getId());
|
||||||
|
Assert.fail("should throw exception because invalid genStamp");
|
||||||
|
} catch (IOException e) {
|
||||||
|
Assert.assertTrue(e.toString().contains(
|
||||||
|
"Commit block with mismatching GS. NN has " +
|
||||||
|
newBlock + ", client submits " + newBlockClone));
|
||||||
|
}
|
||||||
|
previous = new ExtendedBlock(cluster.getNamesystem().getBlockPoolId(),
|
||||||
|
newBlock);
|
||||||
|
boolean complete = dfs.getClient().getNamenode().complete(file.toString(),
|
||||||
|
dfs.getClient().getClientName(), previous, fileNode.getId());
|
||||||
|
Assert.assertTrue("should complete successfully", complete);
|
||||||
|
} finally {
|
||||||
|
IOUtils.cleanup(null, out);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -90,9 +90,9 @@ public class TestQuotaWithStripedBlocks {
|
||||||
INodeFile fileNode = dir.getINode4Write(file.toString()).asFile();
|
INodeFile fileNode = dir.getINode4Write(file.toString()).asFile();
|
||||||
ExtendedBlock previous = null;
|
ExtendedBlock previous = null;
|
||||||
// Create striped blocks which have a cell in each block.
|
// Create striped blocks which have a cell in each block.
|
||||||
Block newBlock = DFSTestUtil.addStripedBlockToFile(cluster.getDataNodes(),
|
Block newBlock = DFSTestUtil.addBlockToFile(true, cluster.getDataNodes(),
|
||||||
dfs, cluster.getNamesystem(), file.toString(), fileNode,
|
dfs, cluster.getNamesystem(), file.toString(), fileNode,
|
||||||
dfs.getClient().getClientName(), previous, 1);
|
dfs.getClient().getClientName(), previous, 1, 0);
|
||||||
previous = new ExtendedBlock(cluster.getNamesystem().getBlockPoolId(),
|
previous = new ExtendedBlock(cluster.getNamesystem().getBlockPoolId(),
|
||||||
newBlock);
|
newBlock);
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue