Merge change r1558255 (HDFS-5704) and r1558675 (HDFS-5777) to branch-2: Change OP_UPDATE_BLOCKS with a new OP_ADD_BLOCK,and update the LayoutVersion
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1559224 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
7c7f33a178
commit
5736c7ad1b
|
@ -200,6 +200,8 @@ Release 2.4.0 - UNRELEASED
|
|||
InvalidEncryptionKeyException in fetchBlockByteRange (Liang Xie via Colin
|
||||
Patrick McCabe)
|
||||
|
||||
HDFS-5704. Change OP_UPDATE_BLOCKS with a new OP_ADD_BLOCK. (jing9)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HDFS-5239. Allow FSNamesystem lock fairness to be configurable (daryn)
|
||||
|
@ -275,6 +277,8 @@ Release 2.4.0 - UNRELEASED
|
|||
HDFS-5579. Under construction files make DataNode decommission take very long
|
||||
hours. (zhaoyunjiong via jing9)
|
||||
|
||||
HDFS-5777. Update LayoutVersion for the new editlog op OP_ADD_BLOCK. (jing9)
|
||||
|
||||
BREAKDOWN OF HDFS-2832 SUBTASKS AND RELATED JIRAS
|
||||
|
||||
HDFS-4985. Add storage type to the protocol and expose it in block report
|
||||
|
|
|
@ -107,8 +107,10 @@ public class LayoutVersion {
|
|||
"block IDs in the edits log and image files"),
|
||||
EDITLOG_SUPPORT_RETRYCACHE(-47, "Record ClientId and CallId in editlog to "
|
||||
+ "enable rebuilding retry cache in case of HA failover"),
|
||||
ADD_DATANODE_AND_STORAGE_UUIDS(-49, -47, "Replace StorageID with DatanodeUuid."
|
||||
+ " Use distinct StorageUuid per storage directory.", false);
|
||||
EDITLOG_ADD_BLOCK(-48, "Add new editlog that only records allocation of "
|
||||
+ "the new block instead of the entire block list"),
|
||||
ADD_DATANODE_AND_STORAGE_UUIDS(-49, "Replace StorageID with DatanodeUuid."
|
||||
+ " Use distinct StorageUuid per storage directory.");
|
||||
|
||||
final int lv;
|
||||
final int ancestorLV;
|
||||
|
|
|
@ -414,6 +414,25 @@ public class FSDirectory implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Persist the new block (the last block of the given file).
|
||||
*/
|
||||
void persistNewBlock(String path, INodeFileUnderConstruction file) {
|
||||
waitForReady();
|
||||
|
||||
writeLock();
|
||||
try {
|
||||
fsImage.getEditLog().logAddBlock(path, file);
|
||||
} finally {
|
||||
writeUnlock();
|
||||
}
|
||||
if (NameNode.stateChangeLog.isDebugEnabled()) {
|
||||
NameNode.stateChangeLog.debug("DIR* FSDirectory.persistNewBlock: "
|
||||
+ path + " with new block " + file.getLastBlock().toString()
|
||||
+ ", current total block count is " + file.getBlocks().length);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Close file.
|
||||
*/
|
||||
|
|
|
@ -39,10 +39,13 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
|
|||
|
||||
import static org.apache.hadoop.util.ExitUtil.terminate;
|
||||
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
|
||||
import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
|
||||
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddBlockOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllocateBlockIdOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CloseOp;
|
||||
|
@ -68,11 +71,9 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
|
|||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateBlocksOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllocateBlockIdOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampV1Op;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampV2Op;
|
||||
import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
|
||||
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
|
||||
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
|
||||
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
|
||||
|
@ -704,7 +705,17 @@ public class FSEditLog implements LogsPurgeable {
|
|||
logEdit(op);
|
||||
}
|
||||
|
||||
public void logUpdateBlocks(String path, INodeFileUnderConstruction file,
|
||||
public void logAddBlock(String path, INodeFileUnderConstruction file) {
|
||||
BlockInfo[] blocks = file.getBlocks();
|
||||
Preconditions.checkState(blocks != null && blocks.length > 0);
|
||||
BlockInfo pBlock = blocks.length > 1 ? blocks[blocks.length - 2] : null;
|
||||
BlockInfo lastBlock = blocks[blocks.length - 1];
|
||||
AddBlockOp op = AddBlockOp.getInstance(cache.get()).setPath(path)
|
||||
.setPenultimateBlock(pBlock).setLastBlock(lastBlock);
|
||||
logEdit(op);
|
||||
}
|
||||
|
||||
public void logUpdateBlocks(String path, INodeFileUnderConstruction file,
|
||||
boolean toLogRpcIds) {
|
||||
UpdateBlocksOp op = UpdateBlocksOp.getInstance(cache.get())
|
||||
.setPath(path)
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.namenode;
|
|||
|
||||
import static org.apache.hadoop.util.Time.now;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FilterInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
@ -41,7 +40,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
|
||||
import org.apache.hadoop.hdfs.server.common.Storage;
|
||||
import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream.LogHeaderCorruptException;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddBlockOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllocateBlockIdOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AllowSnapshotOp;
|
||||
|
@ -81,6 +80,7 @@ import org.apache.hadoop.hdfs.util.ChunkedArrayList;
|
|||
import org.apache.hadoop.hdfs.util.Holder;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Evolving
|
||||
|
@ -407,7 +407,18 @@ public class FSEditLogLoader {
|
|||
}
|
||||
break;
|
||||
}
|
||||
|
||||
case OP_ADD_BLOCK: {
|
||||
AddBlockOp addBlockOp = (AddBlockOp) op;
|
||||
String path = addBlockOp.getPath();
|
||||
if (FSNamesystem.LOG.isDebugEnabled()) {
|
||||
FSNamesystem.LOG.debug(op.opCode + ": " + path +
|
||||
" new block id : " + addBlockOp.getLastBlock().getBlockId());
|
||||
}
|
||||
INodeFile oldFile = INodeFile.valueOf(fsDir.getINode(path), path);
|
||||
// add the new block to the INodeFile
|
||||
addNewBlock(fsDir, addBlockOp, oldFile);
|
||||
break;
|
||||
}
|
||||
case OP_SET_REPLICATION: {
|
||||
SetReplicationOp setReplicationOp = (SetReplicationOp)op;
|
||||
short replication = fsNamesys.getBlockManager().adjustReplication(
|
||||
|
@ -660,6 +671,45 @@ public class FSEditLogLoader {
|
|||
return sb.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new block into the given INodeFile
|
||||
*/
|
||||
private void addNewBlock(FSDirectory fsDir, AddBlockOp op, INodeFile file)
|
||||
throws IOException {
|
||||
BlockInfo[] oldBlocks = file.getBlocks();
|
||||
Block pBlock = op.getPenultimateBlock();
|
||||
Block newBlock= op.getLastBlock();
|
||||
|
||||
if (pBlock != null) { // the penultimate block is not null
|
||||
Preconditions.checkState(oldBlocks != null && oldBlocks.length > 0);
|
||||
// compare pBlock with the last block of oldBlocks
|
||||
Block oldLastBlock = oldBlocks[oldBlocks.length - 1];
|
||||
if (oldLastBlock.getBlockId() != pBlock.getBlockId()
|
||||
|| oldLastBlock.getGenerationStamp() != pBlock.getGenerationStamp()) {
|
||||
throw new IOException(
|
||||
"Mismatched block IDs or generation stamps for the old last block of file "
|
||||
+ op.getPath() + ", the old last block is " + oldLastBlock
|
||||
+ ", and the block read from editlog is " + pBlock);
|
||||
}
|
||||
|
||||
oldLastBlock.setNumBytes(pBlock.getNumBytes());
|
||||
if (oldLastBlock instanceof BlockInfoUnderConstruction) {
|
||||
fsNamesys.getBlockManager().forceCompleteBlock(
|
||||
(INodeFileUnderConstruction) file,
|
||||
(BlockInfoUnderConstruction) oldLastBlock);
|
||||
fsNamesys.getBlockManager().processQueuedMessagesForBlock(pBlock);
|
||||
}
|
||||
} else { // the penultimate block is null
|
||||
Preconditions.checkState(oldBlocks == null || oldBlocks.length == 0);
|
||||
}
|
||||
// add the new block
|
||||
BlockInfo newBI = new BlockInfoUnderConstruction(
|
||||
newBlock, file.getBlockReplication());
|
||||
fsNamesys.getBlockManager().addBlockCollection(newBI, file);
|
||||
file.addBlock(newBI);
|
||||
fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
|
||||
}
|
||||
|
||||
/**
|
||||
* Update in-memory data structures with new block information.
|
||||
* @throws IOException
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
package org.apache.hadoop.hdfs.server.namenode;
|
||||
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_BLOCK;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ALLOCATE_BLOCK_ID;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ALLOW_SNAPSHOT;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_CANCEL_DELEGATION_TOKEN;
|
||||
|
@ -149,6 +150,7 @@ public abstract class FSEditLogOp {
|
|||
inst.put(OP_RENAME_SNAPSHOT, new RenameSnapshotOp());
|
||||
inst.put(OP_SET_GENSTAMP_V2, new SetGenstampV2Op());
|
||||
inst.put(OP_ALLOCATE_BLOCK_ID, new AllocateBlockIdOp());
|
||||
inst.put(OP_ADD_BLOCK, new AddBlockOp());
|
||||
}
|
||||
|
||||
public FSEditLogOp get(FSEditLogOpCodes opcode) {
|
||||
|
@ -584,6 +586,108 @@ public abstract class FSEditLogOp {
|
|||
}
|
||||
}
|
||||
|
||||
static class AddBlockOp extends FSEditLogOp {
|
||||
private String path;
|
||||
private Block penultimateBlock;
|
||||
private Block lastBlock;
|
||||
|
||||
private AddBlockOp() {
|
||||
super(OP_ADD_BLOCK);
|
||||
}
|
||||
|
||||
static AddBlockOp getInstance(OpInstanceCache cache) {
|
||||
return (AddBlockOp) cache.get(OP_ADD_BLOCK);
|
||||
}
|
||||
|
||||
AddBlockOp setPath(String path) {
|
||||
this.path = path;
|
||||
return this;
|
||||
}
|
||||
|
||||
public String getPath() {
|
||||
return path;
|
||||
}
|
||||
|
||||
AddBlockOp setPenultimateBlock(Block pBlock) {
|
||||
this.penultimateBlock = pBlock;
|
||||
return this;
|
||||
}
|
||||
|
||||
Block getPenultimateBlock() {
|
||||
return penultimateBlock;
|
||||
}
|
||||
|
||||
AddBlockOp setLastBlock(Block lastBlock) {
|
||||
this.lastBlock = lastBlock;
|
||||
return this;
|
||||
}
|
||||
|
||||
Block getLastBlock() {
|
||||
return lastBlock;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeFields(DataOutputStream out) throws IOException {
|
||||
FSImageSerialization.writeString(path, out);
|
||||
int size = penultimateBlock != null ? 2 : 1;
|
||||
Block[] blocks = new Block[size];
|
||||
if (penultimateBlock != null) {
|
||||
blocks[0] = penultimateBlock;
|
||||
}
|
||||
blocks[size - 1] = lastBlock;
|
||||
FSImageSerialization.writeCompactBlockArray(blocks, out);
|
||||
// clientId and callId
|
||||
writeRpcIds(rpcClientId, rpcCallId, out);
|
||||
}
|
||||
|
||||
@Override
|
||||
void readFields(DataInputStream in, int logVersion) throws IOException {
|
||||
path = FSImageSerialization.readString(in);
|
||||
Block[] blocks = FSImageSerialization.readCompactBlockArray(in,
|
||||
logVersion);
|
||||
Preconditions.checkState(blocks.length == 2 || blocks.length == 1);
|
||||
penultimateBlock = blocks.length == 1 ? null : blocks[0];
|
||||
lastBlock = blocks[blocks.length - 1];
|
||||
readRpcIds(in, logVersion);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append("AddBlockOp [path=")
|
||||
.append(path)
|
||||
.append(", penultimateBlock=")
|
||||
.append(penultimateBlock == null ? "NULL" : penultimateBlock)
|
||||
.append(", lastBlock=")
|
||||
.append(lastBlock);
|
||||
appendRpcIdsToString(sb, rpcClientId, rpcCallId);
|
||||
sb.append("]");
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void toXml(ContentHandler contentHandler) throws SAXException {
|
||||
XMLUtils.addSaxString(contentHandler, "PATH", path);
|
||||
if (penultimateBlock != null) {
|
||||
FSEditLogOp.blockToXml(contentHandler, penultimateBlock);
|
||||
}
|
||||
FSEditLogOp.blockToXml(contentHandler, lastBlock);
|
||||
appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
|
||||
}
|
||||
|
||||
@Override
|
||||
void fromXml(Stanza st) throws InvalidXmlException {
|
||||
this.path = st.getValue("PATH");
|
||||
List<Stanza> blocks = st.getChildren("BLOCK");
|
||||
int size = blocks.size();
|
||||
Preconditions.checkState(size == 1 || size == 2);
|
||||
this.penultimateBlock = size == 2 ?
|
||||
FSEditLogOp.blockFromXml(blocks.get(0)) : null;
|
||||
this.lastBlock = FSEditLogOp.blockFromXml(blocks.get(size - 1));
|
||||
readRpcIdsFromXml(st);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@literal @AtMostOnce} for {@link ClientProtocol#updatePipeline}, but
|
||||
* {@literal @Idempotent} for some other ops.
|
||||
|
|
|
@ -60,6 +60,7 @@ public enum FSEditLogOpCodes {
|
|||
OP_DISALLOW_SNAPSHOT ((byte) 30),
|
||||
OP_SET_GENSTAMP_V2 ((byte) 31),
|
||||
OP_ALLOCATE_BLOCK_ID ((byte) 32),
|
||||
OP_ADD_BLOCK ((byte) 33),
|
||||
|
||||
// Note that fromByte(..) depends on OP_INVALID being at the last position.
|
||||
OP_INVALID ((byte) -1);
|
||||
|
|
|
@ -2584,7 +2584,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|||
newBlock = createNewBlock();
|
||||
saveAllocatedBlock(src, inodesInPath, newBlock, targets);
|
||||
|
||||
dir.persistBlocks(src, pendingFile, false);
|
||||
dir.persistNewBlock(src, pendingFile);
|
||||
offset = pendingFile.computeFileSize();
|
||||
} finally {
|
||||
writeUnlock();
|
||||
|
|
|
@ -97,29 +97,30 @@ public class TestFileAppendRestart {
|
|||
|
||||
counts = FSImageTestUtil.countEditLogOpTypes(editLog);
|
||||
// OP_ADD to create file
|
||||
// OP_UPDATE_BLOCKS for first block
|
||||
// OP_ADD_BLOCK for first block
|
||||
// OP_CLOSE to close file
|
||||
// OP_ADD to reopen file
|
||||
// OP_UPDATE_BLOCKS for second block
|
||||
// OP_ADD_BLOCK for second block
|
||||
// OP_CLOSE to close file
|
||||
assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_ADD).held);
|
||||
assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_UPDATE_BLOCKS).held);
|
||||
assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_ADD_BLOCK).held);
|
||||
assertEquals(2, (int)counts.get(FSEditLogOpCodes.OP_CLOSE).held);
|
||||
|
||||
Path p2 = new Path("/not-block-boundaries");
|
||||
writeAndAppend(fs, p2, BLOCK_SIZE/2, BLOCK_SIZE);
|
||||
counts = FSImageTestUtil.countEditLogOpTypes(editLog);
|
||||
// OP_ADD to create file
|
||||
// OP_UPDATE_BLOCKS for first block
|
||||
// OP_ADD_BLOCK for first block
|
||||
// OP_CLOSE to close file
|
||||
// OP_ADD to re-establish the lease
|
||||
// OP_UPDATE_BLOCKS from the updatePipeline call (increments genstamp of last block)
|
||||
// OP_UPDATE_BLOCKS at the start of the second block
|
||||
// OP_ADD_BLOCK at the start of the second block
|
||||
// OP_CLOSE to close file
|
||||
// Total: 2 OP_ADDs, 3 OP_UPDATE_BLOCKS, and 2 OP_CLOSEs in addition
|
||||
// to the ones above
|
||||
// Total: 2 OP_ADDs, 1 OP_UPDATE_BLOCKS, 2 OP_ADD_BLOCKs, and 2 OP_CLOSEs
|
||||
// in addition to the ones above
|
||||
assertEquals(2+2, (int)counts.get(FSEditLogOpCodes.OP_ADD).held);
|
||||
assertEquals(2+3, (int)counts.get(FSEditLogOpCodes.OP_UPDATE_BLOCKS).held);
|
||||
assertEquals(1, (int)counts.get(FSEditLogOpCodes.OP_UPDATE_BLOCKS).held);
|
||||
assertEquals(2+2, (int)counts.get(FSEditLogOpCodes.OP_ADD_BLOCK).held);
|
||||
assertEquals(2+2, (int)counts.get(FSEditLogOpCodes.OP_CLOSE).held);
|
||||
|
||||
cluster.restartNameNode();
|
||||
|
@ -176,13 +177,14 @@ public class TestFileAppendRestart {
|
|||
}
|
||||
|
||||
/**
|
||||
* Test to append to the file, when one of datanode in the existing pipeline is down.
|
||||
* @throws Exception
|
||||
* Test to append to the file, when one of datanode in the existing pipeline
|
||||
* is down.
|
||||
*/
|
||||
@Test
|
||||
public void testAppendWithPipelineRecovery() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
MiniDFSCluster cluster = null;
|
||||
FSDataOutputStream out = null;
|
||||
try {
|
||||
cluster = new MiniDFSCluster.Builder(conf).manageDataDfsDirs(true)
|
||||
.manageNameDfsDirs(true).numDataNodes(4)
|
||||
|
@ -192,11 +194,20 @@ public class TestFileAppendRestart {
|
|||
|
||||
DistributedFileSystem fs = cluster.getFileSystem();
|
||||
Path path = new Path("/test1");
|
||||
DFSTestUtil.createFile(fs, path, 1024, (short) 3, 1l);
|
||||
|
||||
out = fs.create(path, true, BLOCK_SIZE, (short) 3, BLOCK_SIZE);
|
||||
AppendTestUtil.write(out, 0, 1024);
|
||||
out.close();
|
||||
|
||||
cluster.stopDataNode(3);
|
||||
DFSTestUtil.appendFile(fs, path, "hello");
|
||||
out = fs.append(path);
|
||||
AppendTestUtil.write(out, 1024, 1024);
|
||||
out.close();
|
||||
|
||||
cluster.restartNameNode(true);
|
||||
AppendTestUtil.check(fs, path, 2048);
|
||||
} finally {
|
||||
IOUtils.closeStream(out);
|
||||
if (null != cluster) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
|
|
@ -0,0 +1,157 @@
|
|||
/**
|
||||
* 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 static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.EnumSet;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DFSOutputStream;
|
||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
|
||||
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
||||
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* Test AddBlockOp is written and read correctly
|
||||
*/
|
||||
public class TestAddBlock {
|
||||
private static final short REPLICATION = 3;
|
||||
private static final int BLOCKSIZE = 1024;
|
||||
|
||||
private MiniDFSCluster cluster;
|
||||
private Configuration conf;
|
||||
|
||||
@Before
|
||||
public void setup() throws IOException {
|
||||
conf = new Configuration();
|
||||
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
|
||||
.build();
|
||||
cluster.waitActive();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test adding new blocks. Restart the NameNode in the test to make sure the
|
||||
* AddBlockOp in the editlog is applied correctly.
|
||||
*/
|
||||
@Test
|
||||
public void testAddBlock() throws Exception {
|
||||
DistributedFileSystem fs = cluster.getFileSystem();
|
||||
|
||||
final Path file1 = new Path("/file1");
|
||||
final Path file2 = new Path("/file2");
|
||||
final Path file3 = new Path("/file3");
|
||||
final Path file4 = new Path("/file4");
|
||||
|
||||
DFSTestUtil.createFile(fs, file1, BLOCKSIZE - 1, REPLICATION, 0L);
|
||||
DFSTestUtil.createFile(fs, file2, BLOCKSIZE, REPLICATION, 0L);
|
||||
DFSTestUtil.createFile(fs, file3, BLOCKSIZE * 2 - 1, REPLICATION, 0L);
|
||||
DFSTestUtil.createFile(fs, file4, BLOCKSIZE * 2, REPLICATION, 0L);
|
||||
|
||||
// restart NameNode
|
||||
cluster.restartNameNode(true);
|
||||
FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
|
||||
|
||||
// check file1
|
||||
INodeFile file1Node = fsdir.getINode4Write(file1.toString()).asFile();
|
||||
BlockInfo[] file1Blocks = file1Node.getBlocks();
|
||||
assertEquals(1, file1Blocks.length);
|
||||
assertEquals(BLOCKSIZE - 1, file1Blocks[0].getNumBytes());
|
||||
assertEquals(BlockUCState.COMPLETE, file1Blocks[0].getBlockUCState());
|
||||
|
||||
// check file2
|
||||
INodeFile file2Node = fsdir.getINode4Write(file2.toString()).asFile();
|
||||
BlockInfo[] file2Blocks = file2Node.getBlocks();
|
||||
assertEquals(1, file2Blocks.length);
|
||||
assertEquals(BLOCKSIZE, file2Blocks[0].getNumBytes());
|
||||
assertEquals(BlockUCState.COMPLETE, file2Blocks[0].getBlockUCState());
|
||||
|
||||
// check file3
|
||||
INodeFile file3Node = fsdir.getINode4Write(file3.toString()).asFile();
|
||||
BlockInfo[] file3Blocks = file3Node.getBlocks();
|
||||
assertEquals(2, file3Blocks.length);
|
||||
assertEquals(BLOCKSIZE, file3Blocks[0].getNumBytes());
|
||||
assertEquals(BlockUCState.COMPLETE, file3Blocks[0].getBlockUCState());
|
||||
assertEquals(BLOCKSIZE - 1, file3Blocks[1].getNumBytes());
|
||||
assertEquals(BlockUCState.COMPLETE, file3Blocks[1].getBlockUCState());
|
||||
|
||||
// check file4
|
||||
INodeFile file4Node = fsdir.getINode4Write(file4.toString()).asFile();
|
||||
BlockInfo[] file4Blocks = file4Node.getBlocks();
|
||||
assertEquals(2, file4Blocks.length);
|
||||
assertEquals(BLOCKSIZE, file4Blocks[0].getNumBytes());
|
||||
assertEquals(BlockUCState.COMPLETE, file4Blocks[0].getBlockUCState());
|
||||
assertEquals(BLOCKSIZE, file4Blocks[1].getNumBytes());
|
||||
assertEquals(BlockUCState.COMPLETE, file4Blocks[1].getBlockUCState());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test adding new blocks but without closing the corresponding the file
|
||||
*/
|
||||
@Test
|
||||
public void testAddBlockUC() throws Exception {
|
||||
DistributedFileSystem fs = cluster.getFileSystem();
|
||||
final Path file1 = new Path("/file1");
|
||||
DFSTestUtil.createFile(fs, file1, BLOCKSIZE - 1, REPLICATION, 0L);
|
||||
|
||||
FSDataOutputStream out = null;
|
||||
try {
|
||||
// append files without closing the streams
|
||||
out = fs.append(file1);
|
||||
String appendContent = "appending-content";
|
||||
out.writeBytes(appendContent);
|
||||
((DFSOutputStream) out.getWrappedStream()).hsync(
|
||||
EnumSet.of(SyncFlag.UPDATE_LENGTH));
|
||||
|
||||
// restart NN
|
||||
cluster.restartNameNode(true);
|
||||
FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
|
||||
|
||||
INodeFile fileNode = fsdir.getINode4Write(file1.toString()).asFile();
|
||||
BlockInfo[] fileBlocks = fileNode.getBlocks();
|
||||
assertEquals(2, fileBlocks.length);
|
||||
assertEquals(BLOCKSIZE, fileBlocks[0].getNumBytes());
|
||||
assertEquals(BlockUCState.COMPLETE, fileBlocks[0].getBlockUCState());
|
||||
assertEquals(appendContent.length() - 1, fileBlocks[1].getNumBytes());
|
||||
assertEquals(BlockUCState.UNDER_CONSTRUCTION,
|
||||
fileBlocks[1].getBlockUCState());
|
||||
} finally {
|
||||
if (out != null) {
|
||||
out.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -21,7 +21,6 @@ package org.apache.hadoop.hdfs.tools.offlineEditsViewer;
|
|||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
|
|
Binary file not shown.
|
@ -13,8 +13,8 @@
|
|||
<TXID>2</TXID>
|
||||
<DELEGATION_KEY>
|
||||
<KEY_ID>1</KEY_ID>
|
||||
<EXPIRY_DATE>1389816437379</EXPIRY_DATE>
|
||||
<KEY>8ef8a1007d96eef6</KEY>
|
||||
<EXPIRY_DATE>1390675042623</EXPIRY_DATE>
|
||||
<KEY>3d747279912f2511</KEY>
|
||||
</DELEGATION_KEY>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
|
@ -24,8 +24,8 @@
|
|||
<TXID>3</TXID>
|
||||
<DELEGATION_KEY>
|
||||
<KEY_ID>2</KEY_ID>
|
||||
<EXPIRY_DATE>1389816437382</EXPIRY_DATE>
|
||||
<KEY>811b959590557873</KEY>
|
||||
<EXPIRY_DATE>1390675042667</EXPIRY_DATE>
|
||||
<KEY>cc4a64b98b6c8b80</KEY>
|
||||
</DELEGATION_KEY>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
|
@ -37,18 +37,18 @@
|
|||
<INODEID>16386</INODEID>
|
||||
<PATH>/file_create</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1389125238605</MTIME>
|
||||
<ATIME>1389125238605</ATIME>
|
||||
<MTIME>1389983882397</MTIME>
|
||||
<ATIME>1389983882397</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-451107822_1</CLIENT_NAME>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-36724706_1</CLIENT_NAME>
|
||||
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>aagarwal</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
<RPC_CLIENTID>61e49ae1-d682-491e-bd15-134c11f912e9</RPC_CLIENTID>
|
||||
<RPC_CALLID>8</RPC_CALLID>
|
||||
<RPC_CLIENTID>6ea2f8e1-8436-477e-b691-6daf7146bf79</RPC_CLIENTID>
|
||||
<RPC_CALLID>22</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
|
@ -59,13 +59,13 @@
|
|||
<INODEID>0</INODEID>
|
||||
<PATH>/file_create</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1389125238661</MTIME>
|
||||
<ATIME>1389125238605</ATIME>
|
||||
<MTIME>1389983882685</MTIME>
|
||||
<ATIME>1389983882397</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME></CLIENT_NAME>
|
||||
<CLIENT_MACHINE></CLIENT_MACHINE>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>aagarwal</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
|
@ -78,9 +78,9 @@
|
|||
<LENGTH>0</LENGTH>
|
||||
<SRC>/file_create</SRC>
|
||||
<DST>/file_moved</DST>
|
||||
<TIMESTAMP>1389125238673</TIMESTAMP>
|
||||
<RPC_CLIENTID>61e49ae1-d682-491e-bd15-134c11f912e9</RPC_CLIENTID>
|
||||
<RPC_CALLID>10</RPC_CALLID>
|
||||
<TIMESTAMP>1389983882713</TIMESTAMP>
|
||||
<RPC_CLIENTID>6ea2f8e1-8436-477e-b691-6daf7146bf79</RPC_CLIENTID>
|
||||
<RPC_CALLID>25</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
|
@ -89,9 +89,9 @@
|
|||
<TXID>7</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<PATH>/file_moved</PATH>
|
||||
<TIMESTAMP>1389125238696</TIMESTAMP>
|
||||
<RPC_CLIENTID>61e49ae1-d682-491e-bd15-134c11f912e9</RPC_CLIENTID>
|
||||
<RPC_CALLID>11</RPC_CALLID>
|
||||
<TIMESTAMP>1389983882722</TIMESTAMP>
|
||||
<RPC_CLIENTID>6ea2f8e1-8436-477e-b691-6daf7146bf79</RPC_CLIENTID>
|
||||
<RPC_CALLID>26</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
|
@ -101,9 +101,9 @@
|
|||
<LENGTH>0</LENGTH>
|
||||
<INODEID>16387</INODEID>
|
||||
<PATH>/directory_mkdir</PATH>
|
||||
<TIMESTAMP>1389125238703</TIMESTAMP>
|
||||
<TIMESTAMP>1389983882732</TIMESTAMP>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>aagarwal</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>493</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
|
@ -136,8 +136,8 @@
|
|||
<TXID>12</TXID>
|
||||
<SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
|
||||
<SNAPSHOTNAME>snapshot1</SNAPSHOTNAME>
|
||||
<RPC_CLIENTID>61e49ae1-d682-491e-bd15-134c11f912e9</RPC_CLIENTID>
|
||||
<RPC_CALLID>16</RPC_CALLID>
|
||||
<RPC_CLIENTID>6ea2f8e1-8436-477e-b691-6daf7146bf79</RPC_CLIENTID>
|
||||
<RPC_CALLID>31</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
|
@ -147,8 +147,8 @@
|
|||
<SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
|
||||
<SNAPSHOTOLDNAME>snapshot1</SNAPSHOTOLDNAME>
|
||||
<SNAPSHOTNEWNAME>snapshot2</SNAPSHOTNEWNAME>
|
||||
<RPC_CLIENTID>61e49ae1-d682-491e-bd15-134c11f912e9</RPC_CLIENTID>
|
||||
<RPC_CALLID>17</RPC_CALLID>
|
||||
<RPC_CLIENTID>6ea2f8e1-8436-477e-b691-6daf7146bf79</RPC_CLIENTID>
|
||||
<RPC_CALLID>32</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
|
@ -157,8 +157,8 @@
|
|||
<TXID>14</TXID>
|
||||
<SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
|
||||
<SNAPSHOTNAME>snapshot2</SNAPSHOTNAME>
|
||||
<RPC_CLIENTID>61e49ae1-d682-491e-bd15-134c11f912e9</RPC_CLIENTID>
|
||||
<RPC_CALLID>18</RPC_CALLID>
|
||||
<RPC_CLIENTID>6ea2f8e1-8436-477e-b691-6daf7146bf79</RPC_CLIENTID>
|
||||
<RPC_CALLID>33</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
|
@ -169,18 +169,18 @@
|
|||
<INODEID>16388</INODEID>
|
||||
<PATH>/file_create</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1389125238750</MTIME>
|
||||
<ATIME>1389125238750</ATIME>
|
||||
<MTIME>1389983883326</MTIME>
|
||||
<ATIME>1389983883326</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-451107822_1</CLIENT_NAME>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-36724706_1</CLIENT_NAME>
|
||||
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>aagarwal</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
<RPC_CLIENTID>61e49ae1-d682-491e-bd15-134c11f912e9</RPC_CLIENTID>
|
||||
<RPC_CALLID>19</RPC_CALLID>
|
||||
<RPC_CLIENTID>6ea2f8e1-8436-477e-b691-6daf7146bf79</RPC_CLIENTID>
|
||||
<RPC_CALLID>34</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
|
@ -191,13 +191,13 @@
|
|||
<INODEID>0</INODEID>
|
||||
<PATH>/file_create</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1389125238752</MTIME>
|
||||
<ATIME>1389125238750</ATIME>
|
||||
<MTIME>1389983883328</MTIME>
|
||||
<ATIME>1389983883326</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME></CLIENT_NAME>
|
||||
<CLIENT_MACHINE></CLIENT_MACHINE>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>aagarwal</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
|
@ -253,10 +253,10 @@
|
|||
<LENGTH>0</LENGTH>
|
||||
<SRC>/file_create</SRC>
|
||||
<DST>/file_moved</DST>
|
||||
<TIMESTAMP>1389125238773</TIMESTAMP>
|
||||
<TIMESTAMP>1389983883464</TIMESTAMP>
|
||||
<OPTIONS>NONE</OPTIONS>
|
||||
<RPC_CLIENTID>61e49ae1-d682-491e-bd15-134c11f912e9</RPC_CLIENTID>
|
||||
<RPC_CALLID>26</RPC_CALLID>
|
||||
<RPC_CLIENTID>6ea2f8e1-8436-477e-b691-6daf7146bf79</RPC_CLIENTID>
|
||||
<RPC_CALLID>41</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
|
@ -267,18 +267,18 @@
|
|||
<INODEID>16389</INODEID>
|
||||
<PATH>/file_concat_target</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1389125238802</MTIME>
|
||||
<ATIME>1389125238802</ATIME>
|
||||
<MTIME>1389983883506</MTIME>
|
||||
<ATIME>1389983883506</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-451107822_1</CLIENT_NAME>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-36724706_1</CLIENT_NAME>
|
||||
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>aagarwal</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
<RPC_CLIENTID>61e49ae1-d682-491e-bd15-134c11f912e9</RPC_CLIENTID>
|
||||
<RPC_CALLID>28</RPC_CALLID>
|
||||
<RPC_CLIENTID>6ea2f8e1-8436-477e-b691-6daf7146bf79</RPC_CLIENTID>
|
||||
<RPC_CALLID>43</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
|
@ -296,7 +296,7 @@
|
|||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_UPDATE_BLOCKS</OPCODE>
|
||||
<OPCODE>OP_ADD_BLOCK</OPCODE>
|
||||
<DATA>
|
||||
<TXID>26</TXID>
|
||||
<PATH>/file_concat_target</PATH>
|
||||
|
@ -324,7 +324,7 @@
|
|||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_UPDATE_BLOCKS</OPCODE>
|
||||
<OPCODE>OP_ADD_BLOCK</OPCODE>
|
||||
<DATA>
|
||||
<TXID>29</TXID>
|
||||
<PATH>/file_concat_target</PATH>
|
||||
|
@ -357,15 +357,10 @@
|
|||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_UPDATE_BLOCKS</OPCODE>
|
||||
<OPCODE>OP_ADD_BLOCK</OPCODE>
|
||||
<DATA>
|
||||
<TXID>32</TXID>
|
||||
<PATH>/file_concat_target</PATH>
|
||||
<BLOCK>
|
||||
<BLOCK_ID>1073741825</BLOCK_ID>
|
||||
<NUM_BYTES>512</NUM_BYTES>
|
||||
<GENSTAMP>1001</GENSTAMP>
|
||||
</BLOCK>
|
||||
<BLOCK>
|
||||
<BLOCK_ID>1073741826</BLOCK_ID>
|
||||
<NUM_BYTES>512</NUM_BYTES>
|
||||
|
@ -388,8 +383,8 @@
|
|||
<INODEID>0</INODEID>
|
||||
<PATH>/file_concat_target</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1389125239192</MTIME>
|
||||
<ATIME>1389125238802</ATIME>
|
||||
<MTIME>1389983884500</MTIME>
|
||||
<ATIME>1389983883506</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME></CLIENT_NAME>
|
||||
<CLIENT_MACHINE></CLIENT_MACHINE>
|
||||
|
@ -409,7 +404,7 @@
|
|||
<GENSTAMP>1003</GENSTAMP>
|
||||
</BLOCK>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>aagarwal</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
|
@ -423,18 +418,18 @@
|
|||
<INODEID>16390</INODEID>
|
||||
<PATH>/file_concat_0</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1389125239195</MTIME>
|
||||
<ATIME>1389125239195</ATIME>
|
||||
<MTIME>1389983884503</MTIME>
|
||||
<ATIME>1389983884503</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-451107822_1</CLIENT_NAME>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-36724706_1</CLIENT_NAME>
|
||||
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>aagarwal</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
<RPC_CLIENTID>61e49ae1-d682-491e-bd15-134c11f912e9</RPC_CLIENTID>
|
||||
<RPC_CALLID>41</RPC_CALLID>
|
||||
<RPC_CLIENTID>6ea2f8e1-8436-477e-b691-6daf7146bf79</RPC_CLIENTID>
|
||||
<RPC_CALLID>56</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
|
@ -452,7 +447,7 @@
|
|||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_UPDATE_BLOCKS</OPCODE>
|
||||
<OPCODE>OP_ADD_BLOCK</OPCODE>
|
||||
<DATA>
|
||||
<TXID>37</TXID>
|
||||
<PATH>/file_concat_0</PATH>
|
||||
|
@ -480,7 +475,7 @@
|
|||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_UPDATE_BLOCKS</OPCODE>
|
||||
<OPCODE>OP_ADD_BLOCK</OPCODE>
|
||||
<DATA>
|
||||
<TXID>40</TXID>
|
||||
<PATH>/file_concat_0</PATH>
|
||||
|
@ -513,15 +508,10 @@
|
|||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_UPDATE_BLOCKS</OPCODE>
|
||||
<OPCODE>OP_ADD_BLOCK</OPCODE>
|
||||
<DATA>
|
||||
<TXID>43</TXID>
|
||||
<PATH>/file_concat_0</PATH>
|
||||
<BLOCK>
|
||||
<BLOCK_ID>1073741828</BLOCK_ID>
|
||||
<NUM_BYTES>512</NUM_BYTES>
|
||||
<GENSTAMP>1004</GENSTAMP>
|
||||
</BLOCK>
|
||||
<BLOCK>
|
||||
<BLOCK_ID>1073741829</BLOCK_ID>
|
||||
<NUM_BYTES>512</NUM_BYTES>
|
||||
|
@ -544,8 +534,8 @@
|
|||
<INODEID>0</INODEID>
|
||||
<PATH>/file_concat_0</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1389125239221</MTIME>
|
||||
<ATIME>1389125239195</ATIME>
|
||||
<MTIME>1389983884530</MTIME>
|
||||
<ATIME>1389983884503</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME></CLIENT_NAME>
|
||||
<CLIENT_MACHINE></CLIENT_MACHINE>
|
||||
|
@ -565,7 +555,7 @@
|
|||
<GENSTAMP>1006</GENSTAMP>
|
||||
</BLOCK>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>aagarwal</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
|
@ -579,18 +569,18 @@
|
|||
<INODEID>16391</INODEID>
|
||||
<PATH>/file_concat_1</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1389125239223</MTIME>
|
||||
<ATIME>1389125239223</ATIME>
|
||||
<MTIME>1389983884533</MTIME>
|
||||
<ATIME>1389983884533</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-451107822_1</CLIENT_NAME>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-36724706_1</CLIENT_NAME>
|
||||
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>aagarwal</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
<RPC_CLIENTID>61e49ae1-d682-491e-bd15-134c11f912e9</RPC_CLIENTID>
|
||||
<RPC_CALLID>53</RPC_CALLID>
|
||||
<RPC_CLIENTID>6ea2f8e1-8436-477e-b691-6daf7146bf79</RPC_CLIENTID>
|
||||
<RPC_CALLID>68</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
|
@ -608,7 +598,7 @@
|
|||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_UPDATE_BLOCKS</OPCODE>
|
||||
<OPCODE>OP_ADD_BLOCK</OPCODE>
|
||||
<DATA>
|
||||
<TXID>48</TXID>
|
||||
<PATH>/file_concat_1</PATH>
|
||||
|
@ -636,7 +626,7 @@
|
|||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_UPDATE_BLOCKS</OPCODE>
|
||||
<OPCODE>OP_ADD_BLOCK</OPCODE>
|
||||
<DATA>
|
||||
<TXID>51</TXID>
|
||||
<PATH>/file_concat_1</PATH>
|
||||
|
@ -669,15 +659,10 @@
|
|||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_UPDATE_BLOCKS</OPCODE>
|
||||
<OPCODE>OP_ADD_BLOCK</OPCODE>
|
||||
<DATA>
|
||||
<TXID>54</TXID>
|
||||
<PATH>/file_concat_1</PATH>
|
||||
<BLOCK>
|
||||
<BLOCK_ID>1073741831</BLOCK_ID>
|
||||
<NUM_BYTES>512</NUM_BYTES>
|
||||
<GENSTAMP>1007</GENSTAMP>
|
||||
</BLOCK>
|
||||
<BLOCK>
|
||||
<BLOCK_ID>1073741832</BLOCK_ID>
|
||||
<NUM_BYTES>512</NUM_BYTES>
|
||||
|
@ -700,8 +685,8 @@
|
|||
<INODEID>0</INODEID>
|
||||
<PATH>/file_concat_1</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1389125239250</MTIME>
|
||||
<ATIME>1389125239223</ATIME>
|
||||
<MTIME>1389983884560</MTIME>
|
||||
<ATIME>1389983884533</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME></CLIENT_NAME>
|
||||
<CLIENT_MACHINE></CLIENT_MACHINE>
|
||||
|
@ -721,7 +706,7 @@
|
|||
<GENSTAMP>1009</GENSTAMP>
|
||||
</BLOCK>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>aagarwal</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
|
@ -733,13 +718,13 @@
|
|||
<TXID>56</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<TRG>/file_concat_target</TRG>
|
||||
<TIMESTAMP>1389125239253</TIMESTAMP>
|
||||
<TIMESTAMP>1389983884587</TIMESTAMP>
|
||||
<SOURCES>
|
||||
<SOURCE1>/file_concat_0</SOURCE1>
|
||||
<SOURCE2>/file_concat_1</SOURCE2>
|
||||
</SOURCES>
|
||||
<RPC_CLIENTID>61e49ae1-d682-491e-bd15-134c11f912e9</RPC_CLIENTID>
|
||||
<RPC_CALLID>64</RPC_CALLID>
|
||||
<RPC_CLIENTID>6ea2f8e1-8436-477e-b691-6daf7146bf79</RPC_CLIENTID>
|
||||
<RPC_CALLID>79</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
|
@ -750,15 +735,15 @@
|
|||
<INODEID>16392</INODEID>
|
||||
<PATH>/file_symlink</PATH>
|
||||
<VALUE>/file_concat_target</VALUE>
|
||||
<MTIME>1389125239299</MTIME>
|
||||
<ATIME>1389125239299</ATIME>
|
||||
<MTIME>1389983884610</MTIME>
|
||||
<ATIME>1389983884610</ATIME>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>aagarwal</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>511</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
<RPC_CLIENTID>61e49ae1-d682-491e-bd15-134c11f912e9</RPC_CLIENTID>
|
||||
<RPC_CALLID>65</RPC_CALLID>
|
||||
<RPC_CLIENTID>6ea2f8e1-8436-477e-b691-6daf7146bf79</RPC_CLIENTID>
|
||||
<RPC_CALLID>80</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
|
@ -769,18 +754,18 @@
|
|||
<INODEID>16393</INODEID>
|
||||
<PATH>/hard-lease-recovery-test</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1389125239332</MTIME>
|
||||
<ATIME>1389125239332</ATIME>
|
||||
<MTIME>1389983884634</MTIME>
|
||||
<ATIME>1389983884634</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-451107822_1</CLIENT_NAME>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-36724706_1</CLIENT_NAME>
|
||||
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>aagarwal</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
<RPC_CLIENTID>61e49ae1-d682-491e-bd15-134c11f912e9</RPC_CLIENTID>
|
||||
<RPC_CALLID>66</RPC_CALLID>
|
||||
<RPC_CLIENTID>6ea2f8e1-8436-477e-b691-6daf7146bf79</RPC_CLIENTID>
|
||||
<RPC_CALLID>81</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
|
@ -798,7 +783,7 @@
|
|||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_UPDATE_BLOCKS</OPCODE>
|
||||
<OPCODE>OP_ADD_BLOCK</OPCODE>
|
||||
<DATA>
|
||||
<TXID>61</TXID>
|
||||
<PATH>/hard-lease-recovery-test</PATH>
|
||||
|
@ -836,7 +821,7 @@
|
|||
<OPCODE>OP_REASSIGN_LEASE</OPCODE>
|
||||
<DATA>
|
||||
<TXID>64</TXID>
|
||||
<LEASEHOLDER>DFSClient_NONMAPREDUCE_-451107822_1</LEASEHOLDER>
|
||||
<LEASEHOLDER>DFSClient_NONMAPREDUCE_-36724706_1</LEASEHOLDER>
|
||||
<PATH>/hard-lease-recovery-test</PATH>
|
||||
<NEWHOLDER>HDFS_NameNode</NEWHOLDER>
|
||||
</DATA>
|
||||
|
@ -849,8 +834,8 @@
|
|||
<INODEID>0</INODEID>
|
||||
<PATH>/hard-lease-recovery-test</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1389125241496</MTIME>
|
||||
<ATIME>1389125239332</ATIME>
|
||||
<MTIME>1389983888641</MTIME>
|
||||
<ATIME>1389983884634</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME></CLIENT_NAME>
|
||||
<CLIENT_MACHINE></CLIENT_MACHINE>
|
||||
|
@ -860,7 +845,7 @@
|
|||
<GENSTAMP>1011</GENSTAMP>
|
||||
</BLOCK>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>aagarwal</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
|
|
Loading…
Reference in New Issue