HDFS-9289. Make DataStreamer#block thread safe and verify genStamp in commitBlock. Contributed by Chang Li.

Change-Id: I0591b1f9c6a0a8b368408b195151de2e0aa440c4
This commit is contained in:
Zhe Zhang 2015-11-03 14:23:02 -08:00
parent 246ab44b5e
commit 4f29f5d192
6 changed files with 175 additions and 2 deletions

View File

@ -331,7 +331,7 @@ synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) {
} }
private volatile boolean streamerClosed = false; private volatile boolean streamerClosed = false;
private ExtendedBlock block; // its length is number of bytes acked private volatile ExtendedBlock block; // its length is number of bytes acked
private Token<BlockTokenIdentifier> accessToken; private Token<BlockTokenIdentifier> accessToken;
private DataOutputStream blockStream; private DataOutputStream blockStream;
private DataInputStream blockReplyStream; private DataInputStream blockReplyStream;

View File

@ -1382,6 +1382,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

View File

@ -409,7 +409,7 @@ void commitBlock(Block block) throws IOException {
} }
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());
} }

View File

@ -621,6 +621,10 @@ private static boolean commitBlock(final BlockInfo block,
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;
} }

View File

@ -127,12 +127,16 @@
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
import org.apache.hadoop.hdfs.server.namenode.FSEditLog; import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
import org.apache.hadoop.hdfs.server.namenode.LeaseManager; import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider; import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
import org.apache.hadoop.hdfs.tools.DFSAdmin; import org.apache.hadoop.hdfs.tools.DFSAdmin;
import org.apache.hadoop.hdfs.tools.JMXGet; import org.apache.hadoop.hdfs.tools.JMXGet;
import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.IOUtils;
@ -1883,4 +1887,66 @@ public Boolean get() {
} }
}, 1000, Integer.MAX_VALUE); }, 1000, Integer.MAX_VALUE);
} }
public static StorageReceivedDeletedBlocks[] makeReportForReceivedBlock(
Block block, BlockStatus blockStatus, DatanodeStorage storage) {
ReceivedDeletedBlockInfo[] receivedBlocks = new ReceivedDeletedBlockInfo[1];
receivedBlocks[0] = new ReceivedDeletedBlockInfo(block, blockStatus, null);
StorageReceivedDeletedBlocks[] reports = new StorageReceivedDeletedBlocks[1];
reports[0] = new StorageReceivedDeletedBlocks(storage, receivedBlocks);
return reports;
}
/**
* Adds a block to a file.
* This method only manipulates NameNode
* states of the file and the block without injecting data to DataNode.
* It does mimic block reports.
* You should disable periodical heartbeat before use this.
* @param dataNodes List DataNodes to host the block
* @param previous Previous block in the file
* @param len block size
* @return The added block
*/
public static Block addBlockToFile(
List<DataNode> dataNodes, DistributedFileSystem fs, FSNamesystem ns,
String file, INodeFile fileNode,
String clientName, ExtendedBlock previous, int len)
throws Exception {
fs.getClient().namenode.addBlock(file, clientName, previous, null,
fileNode.getId(), null);
final BlockInfo lastBlock = fileNode.getLastBlock();
final int groupSize = fileNode.getPreferredBlockReplication();
assert dataNodes.size() >= groupSize;
// 1. RECEIVING_BLOCK IBR
for (int i = 0; i < groupSize; i++) {
DataNode dn = dataNodes.get(i);
final Block block = new Block(lastBlock.getBlockId() + i, 0,
lastBlock.getGenerationStamp());
DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
StorageReceivedDeletedBlocks[] reports = DFSTestUtil
.makeReportForReceivedBlock(block,
ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, storage);
for (StorageReceivedDeletedBlocks report : reports) {
ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
}
}
// 2. RECEIVED_BLOCK IBR
for (int i = 0; i < groupSize; i++) {
DataNode dn = dataNodes.get(i);
final Block block = new Block(lastBlock.getBlockId() + i,
len, lastBlock.getGenerationStamp());
DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
StorageReceivedDeletedBlocks[] reports = DFSTestUtil
.makeReportForReceivedBlock(block,
ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
for (StorageReceivedDeletedBlocks report : reports) {
ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
}
}
lastBlock.setNumBytes(len);
return lastBlock;
}
} }

View File

@ -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(cluster.getDataNodes(),
dfs, cluster.getNamesystem(), file.toString(), fileNode,
dfs.getClient().getClientName(), previous, 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);
}
}
}