HDFS-5704. Change OP_UPDATE_BLOCKS with a new OP_ADD_BLOCK. Contributed by Jing Zhao.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1558255 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
d13c2eeca6
commit
578dae9ef3
|
@ -927,6 +927,8 @@ Release 2.3.0 - UNRELEASED
|
|||
HDFS-5677. Need error checking for HA cluster configuration.
|
||||
(Vincent Sheffer via cos)
|
||||
|
||||
HDFS-5704. Change OP_UPDATE_BLOCKS with a new OP_ADD_BLOCK. (jing9)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
|
|
@ -417,6 +417,26 @@ public class FSDirectory implements Closeable {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Persist the new block (the last block of the given file).
|
||||
*/
|
||||
void persistNewBlock(String path, INodeFile file) {
|
||||
Preconditions.checkArgument(file.isUnderConstruction());
|
||||
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.
|
||||
*/
|
||||
|
|
|
@ -36,17 +36,18 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.Options;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
|
||||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
||||
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.AddCacheDirectiveInfoOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCachePoolOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ModifyCacheDirectiveInfoOp;
|
||||
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;
|
||||
|
@ -59,11 +60,12 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DisallowSnapshotOp;
|
|||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.GetDelegationTokenOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.LogSegmentOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ModifyCacheDirectiveInfoOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ModifyCachePoolOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.OpInstanceCache;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCachePoolOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCacheDirectiveInfoOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RemoveCachePoolOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameSnapshotOp;
|
||||
|
@ -709,6 +711,17 @@ public class FSEditLog implements LogsPurgeable {
|
|||
logEdit(op);
|
||||
}
|
||||
|
||||
public void logAddBlock(String path, INodeFile file) {
|
||||
Preconditions.checkArgument(file.isUnderConstruction());
|
||||
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, INodeFile file, boolean toLogRpcIds) {
|
||||
Preconditions.checkArgument(file.isUnderConstruction());
|
||||
UpdateBlocksOp op = UpdateBlocksOp.getInstance(cache.get())
|
||||
|
|
|
@ -40,6 +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.FSEditLogOp.AddBlockOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCacheDirectiveInfoOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCachePoolOp;
|
||||
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
|
||||
|
@ -411,7 +412,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(
|
||||
|
@ -713,6 +725,44 @@ 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(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_ADD_CACHE_DIRECTIVE;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_CACHE_POOL;
|
||||
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ALLOCATE_BLOCK_ID;
|
||||
|
@ -166,6 +167,7 @@ public abstract class FSEditLogOp {
|
|||
inst.put(OP_ADD_CACHE_POOL, new AddCachePoolOp());
|
||||
inst.put(OP_MODIFY_CACHE_POOL, new ModifyCachePoolOp());
|
||||
inst.put(OP_REMOVE_CACHE_POOL, new RemoveCachePoolOp());
|
||||
inst.put(OP_ADD_BLOCK, new AddBlockOp());
|
||||
}
|
||||
|
||||
public FSEditLogOp get(FSEditLogOpCodes opcode) {
|
||||
|
@ -600,6 +602,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.
|
||||
|
|
|
@ -66,6 +66,7 @@ public enum FSEditLogOpCodes {
|
|||
OP_MODIFY_CACHE_POOL ((byte) 36),
|
||||
OP_REMOVE_CACHE_POOL ((byte) 37),
|
||||
OP_MODIFY_CACHE_DIRECTIVE ((byte) 38),
|
||||
OP_ADD_BLOCK ((byte) 39),
|
||||
|
||||
// Note that fromByte(..) depends on OP_INVALID being at the last position.
|
||||
OP_INVALID ((byte) -1);
|
||||
|
|
|
@ -2629,7 +2629,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>1389121087930</EXPIRY_DATE>
|
||||
<KEY>d48b4b3e6a43707b</KEY>
|
||||
<EXPIRY_DATE>1389736494300</EXPIRY_DATE>
|
||||
<KEY>d1a0861e6b9e394e</KEY>
|
||||
</DELEGATION_KEY>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
|
@ -24,8 +24,8 @@
|
|||
<TXID>3</TXID>
|
||||
<DELEGATION_KEY>
|
||||
<KEY_ID>2</KEY_ID>
|
||||
<EXPIRY_DATE>1389121087937</EXPIRY_DATE>
|
||||
<KEY>62b6fae6bff918a9</KEY>
|
||||
<EXPIRY_DATE>1389736494302</EXPIRY_DATE>
|
||||
<KEY>8239b8f0ed7e6ce6</KEY>
|
||||
</DELEGATION_KEY>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
|
@ -35,19 +35,19 @@
|
|||
<TXID>4</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>16386</INODEID>
|
||||
<PATH>/file_create_u\0001;F431</PATH>
|
||||
<PATH>/file_create</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1388429889312</MTIME>
|
||||
<ATIME>1388429889312</ATIME>
|
||||
<MTIME>1389045295288</MTIME>
|
||||
<ATIME>1389045295288</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-1396063717_1</CLIENT_NAME>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-216163116_1</CLIENT_NAME>
|
||||
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>andrew</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>7</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
|
@ -57,15 +57,15 @@
|
|||
<TXID>5</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>0</INODEID>
|
||||
<PATH>/file_create_u\0001;F431</PATH>
|
||||
<PATH>/file_create</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1388429889328</MTIME>
|
||||
<ATIME>1388429889312</ATIME>
|
||||
<MTIME>1389045295307</MTIME>
|
||||
<ATIME>1389045295288</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME></CLIENT_NAME>
|
||||
<CLIENT_MACHINE></CLIENT_MACHINE>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>andrew</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
|
@ -76,10 +76,10 @@
|
|||
<DATA>
|
||||
<TXID>6</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<SRC>/file_create_u\0001;F431</SRC>
|
||||
<SRC>/file_create</SRC>
|
||||
<DST>/file_moved</DST>
|
||||
<TIMESTAMP>1388429889336</TIMESTAMP>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<TIMESTAMP>1389045295311</TIMESTAMP>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>9</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
|
@ -89,8 +89,8 @@
|
|||
<TXID>7</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<PATH>/file_moved</PATH>
|
||||
<TIMESTAMP>1388429889346</TIMESTAMP>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<TIMESTAMP>1389045295318</TIMESTAMP>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>10</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
|
@ -101,9 +101,9 @@
|
|||
<LENGTH>0</LENGTH>
|
||||
<INODEID>16387</INODEID>
|
||||
<PATH>/directory_mkdir</PATH>
|
||||
<TIMESTAMP>1388429889357</TIMESTAMP>
|
||||
<TIMESTAMP>1389045295326</TIMESTAMP>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>andrew</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>493</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
|
@ -136,7 +136,7 @@
|
|||
<TXID>12</TXID>
|
||||
<SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
|
||||
<SNAPSHOTNAME>snapshot1</SNAPSHOTNAME>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>15</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
|
@ -147,7 +147,7 @@
|
|||
<SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
|
||||
<SNAPSHOTOLDNAME>snapshot1</SNAPSHOTOLDNAME>
|
||||
<SNAPSHOTNEWNAME>snapshot2</SNAPSHOTNEWNAME>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>16</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
|
@ -157,7 +157,7 @@
|
|||
<TXID>14</TXID>
|
||||
<SNAPSHOTROOT>/directory_mkdir</SNAPSHOTROOT>
|
||||
<SNAPSHOTNAME>snapshot2</SNAPSHOTNAME>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>17</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
|
@ -167,19 +167,19 @@
|
|||
<TXID>15</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>16388</INODEID>
|
||||
<PATH>/file_create_u\0001;F431</PATH>
|
||||
<PATH>/file_create</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1388429889412</MTIME>
|
||||
<ATIME>1388429889412</ATIME>
|
||||
<MTIME>1389045295354</MTIME>
|
||||
<ATIME>1389045295354</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-1396063717_1</CLIENT_NAME>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-216163116_1</CLIENT_NAME>
|
||||
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>andrew</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>18</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
|
@ -189,15 +189,15 @@
|
|||
<TXID>16</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>0</INODEID>
|
||||
<PATH>/file_create_u\0001;F431</PATH>
|
||||
<PATH>/file_create</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1388429889420</MTIME>
|
||||
<ATIME>1388429889412</ATIME>
|
||||
<MTIME>1389045295357</MTIME>
|
||||
<ATIME>1389045295354</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME></CLIENT_NAME>
|
||||
<CLIENT_MACHINE></CLIENT_MACHINE>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>andrew</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
|
@ -207,7 +207,7 @@
|
|||
<OPCODE>OP_SET_REPLICATION</OPCODE>
|
||||
<DATA>
|
||||
<TXID>17</TXID>
|
||||
<PATH>/file_create_u\0001;F431</PATH>
|
||||
<PATH>/file_create</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
|
@ -215,7 +215,7 @@
|
|||
<OPCODE>OP_SET_PERMISSIONS</OPCODE>
|
||||
<DATA>
|
||||
<TXID>18</TXID>
|
||||
<SRC>/file_create_u\0001;F431</SRC>
|
||||
<SRC>/file_create</SRC>
|
||||
<MODE>511</MODE>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
|
@ -223,7 +223,7 @@
|
|||
<OPCODE>OP_SET_OWNER</OPCODE>
|
||||
<DATA>
|
||||
<TXID>19</TXID>
|
||||
<SRC>/file_create_u\0001;F431</SRC>
|
||||
<SRC>/file_create</SRC>
|
||||
<USERNAME>newOwner</USERNAME>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
|
@ -232,7 +232,7 @@
|
|||
<DATA>
|
||||
<TXID>20</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<PATH>/file_create_u\0001;F431</PATH>
|
||||
<PATH>/file_create</PATH>
|
||||
<MTIME>1285195527000</MTIME>
|
||||
<ATIME>1285195527000</ATIME>
|
||||
</DATA>
|
||||
|
@ -251,11 +251,11 @@
|
|||
<DATA>
|
||||
<TXID>22</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<SRC>/file_create_u\0001;F431</SRC>
|
||||
<SRC>/file_create</SRC>
|
||||
<DST>/file_moved</DST>
|
||||
<TIMESTAMP>1388429889495</TIMESTAMP>
|
||||
<TIMESTAMP>1389045295378</TIMESTAMP>
|
||||
<OPTIONS>NONE</OPTIONS>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>25</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
|
@ -267,17 +267,17 @@
|
|||
<INODEID>16389</INODEID>
|
||||
<PATH>/file_concat_target</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1388429889511</MTIME>
|
||||
<ATIME>1388429889511</ATIME>
|
||||
<MTIME>1389045295383</MTIME>
|
||||
<ATIME>1389045295383</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-1396063717_1</CLIENT_NAME>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-216163116_1</CLIENT_NAME>
|
||||
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>andrew</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>27</RPC_CALLID>
|
||||
</DATA>
|
||||
</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>1388429889812</MTIME>
|
||||
<ATIME>1388429889511</ATIME>
|
||||
<MTIME>1389045295484</MTIME>
|
||||
<ATIME>1389045295383</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME></CLIENT_NAME>
|
||||
<CLIENT_MACHINE></CLIENT_MACHINE>
|
||||
|
@ -409,7 +404,7 @@
|
|||
<GENSTAMP>1003</GENSTAMP>
|
||||
</BLOCK>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>andrew</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
|
@ -423,17 +418,17 @@
|
|||
<INODEID>16390</INODEID>
|
||||
<PATH>/file_concat_0</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1388429889825</MTIME>
|
||||
<ATIME>1388429889825</ATIME>
|
||||
<MTIME>1389045295486</MTIME>
|
||||
<ATIME>1389045295486</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-1396063717_1</CLIENT_NAME>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-216163116_1</CLIENT_NAME>
|
||||
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>andrew</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>40</RPC_CALLID>
|
||||
</DATA>
|
||||
</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>1388429889909</MTIME>
|
||||
<ATIME>1388429889825</ATIME>
|
||||
<MTIME>1389045295507</MTIME>
|
||||
<ATIME>1389045295486</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME></CLIENT_NAME>
|
||||
<CLIENT_MACHINE></CLIENT_MACHINE>
|
||||
|
@ -565,7 +555,7 @@
|
|||
<GENSTAMP>1006</GENSTAMP>
|
||||
</BLOCK>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>andrew</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
|
@ -579,17 +569,17 @@
|
|||
<INODEID>16391</INODEID>
|
||||
<PATH>/file_concat_1</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1388429889920</MTIME>
|
||||
<ATIME>1388429889920</ATIME>
|
||||
<MTIME>1389045295509</MTIME>
|
||||
<ATIME>1389045295509</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-1396063717_1</CLIENT_NAME>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-216163116_1</CLIENT_NAME>
|
||||
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>andrew</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>52</RPC_CALLID>
|
||||
</DATA>
|
||||
</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>1388429890016</MTIME>
|
||||
<ATIME>1388429889920</ATIME>
|
||||
<MTIME>1389045295532</MTIME>
|
||||
<ATIME>1389045295509</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME></CLIENT_NAME>
|
||||
<CLIENT_MACHINE></CLIENT_MACHINE>
|
||||
|
@ -721,7 +706,7 @@
|
|||
<GENSTAMP>1009</GENSTAMP>
|
||||
</BLOCK>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>andrew</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
|
@ -733,12 +718,12 @@
|
|||
<TXID>56</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<TRG>/file_concat_target</TRG>
|
||||
<TIMESTAMP>1388429890031</TIMESTAMP>
|
||||
<TIMESTAMP>1389045295535</TIMESTAMP>
|
||||
<SOURCES>
|
||||
<SOURCE1>/file_concat_0</SOURCE1>
|
||||
<SOURCE2>/file_concat_1</SOURCE2>
|
||||
</SOURCES>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>63</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
|
@ -750,175 +735,57 @@
|
|||
<INODEID>16392</INODEID>
|
||||
<PATH>/file_symlink</PATH>
|
||||
<VALUE>/file_concat_target</VALUE>
|
||||
<MTIME>1388429890046</MTIME>
|
||||
<ATIME>1388429890046</ATIME>
|
||||
<MTIME>1389045295540</MTIME>
|
||||
<ATIME>1389045295540</ATIME>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>andrew</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>511</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>64</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_GET_DELEGATION_TOKEN</OPCODE>
|
||||
<DATA>
|
||||
<TXID>58</TXID>
|
||||
<DELEGATION_TOKEN_IDENTIFIER>
|
||||
<KIND>HDFS_DELEGATION_TOKEN</KIND>
|
||||
<SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
|
||||
<OWNER>andrew</OWNER>
|
||||
<RENEWER>JobTracker</RENEWER>
|
||||
<REALUSER></REALUSER>
|
||||
<ISSUE_DATE>1388429890059</ISSUE_DATE>
|
||||
<MAX_DATE>1389034690059</MAX_DATE>
|
||||
<MASTER_KEY_ID>2</MASTER_KEY_ID>
|
||||
</DELEGATION_TOKEN_IDENTIFIER>
|
||||
<EXPIRY_TIME>1388516290059</EXPIRY_TIME>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_RENEW_DELEGATION_TOKEN</OPCODE>
|
||||
<DATA>
|
||||
<TXID>59</TXID>
|
||||
<DELEGATION_TOKEN_IDENTIFIER>
|
||||
<KIND>HDFS_DELEGATION_TOKEN</KIND>
|
||||
<SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
|
||||
<OWNER>andrew</OWNER>
|
||||
<RENEWER>JobTracker</RENEWER>
|
||||
<REALUSER></REALUSER>
|
||||
<ISSUE_DATE>1388429890059</ISSUE_DATE>
|
||||
<MAX_DATE>1389034690059</MAX_DATE>
|
||||
<MASTER_KEY_ID>2</MASTER_KEY_ID>
|
||||
</DELEGATION_TOKEN_IDENTIFIER>
|
||||
<EXPIRY_TIME>1388516290109</EXPIRY_TIME>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_CANCEL_DELEGATION_TOKEN</OPCODE>
|
||||
<DATA>
|
||||
<TXID>60</TXID>
|
||||
<DELEGATION_TOKEN_IDENTIFIER>
|
||||
<KIND>HDFS_DELEGATION_TOKEN</KIND>
|
||||
<SEQUENCE_NUMBER>1</SEQUENCE_NUMBER>
|
||||
<OWNER>andrew</OWNER>
|
||||
<RENEWER>JobTracker</RENEWER>
|
||||
<REALUSER></REALUSER>
|
||||
<ISSUE_DATE>1388429890059</ISSUE_DATE>
|
||||
<MAX_DATE>1389034690059</MAX_DATE>
|
||||
<MASTER_KEY_ID>2</MASTER_KEY_ID>
|
||||
</DELEGATION_TOKEN_IDENTIFIER>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_ADD_CACHE_POOL</OPCODE>
|
||||
<DATA>
|
||||
<TXID>61</TXID>
|
||||
<POOLNAME>poolparty</POOLNAME>
|
||||
<OWNERNAME>andrew</OWNERNAME>
|
||||
<GROUPNAME>andrew</GROUPNAME>
|
||||
<MODE>493</MODE>
|
||||
<LIMIT>9223372036854775807</LIMIT>
|
||||
<MAXRELATIVEEXPIRY>2305843009213693951</MAXRELATIVEEXPIRY>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<RPC_CALLID>68</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_MODIFY_CACHE_POOL</OPCODE>
|
||||
<DATA>
|
||||
<TXID>62</TXID>
|
||||
<POOLNAME>poolparty</POOLNAME>
|
||||
<OWNERNAME>carlton</OWNERNAME>
|
||||
<GROUPNAME>party</GROUPNAME>
|
||||
<MODE>448</MODE>
|
||||
<LIMIT>1989</LIMIT>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<RPC_CALLID>69</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_ADD_CACHE_DIRECTIVE</OPCODE>
|
||||
<DATA>
|
||||
<TXID>63</TXID>
|
||||
<ID>1</ID>
|
||||
<PATH>/bar</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<POOL>poolparty</POOL>
|
||||
<EXPIRATION>2305844397643584141</EXPIRATION>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<RPC_CALLID>70</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_MODIFY_CACHE_DIRECTIVE</OPCODE>
|
||||
<DATA>
|
||||
<TXID>64</TXID>
|
||||
<ID>1</ID>
|
||||
<PATH>/bar2</PATH>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<RPC_CALLID>71</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_REMOVE_CACHE_DIRECTIVE</OPCODE>
|
||||
<DATA>
|
||||
<TXID>65</TXID>
|
||||
<ID>1</ID>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<RPC_CALLID>72</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_REMOVE_CACHE_POOL</OPCODE>
|
||||
<DATA>
|
||||
<TXID>66</TXID>
|
||||
<POOLNAME>poolparty</POOLNAME>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<RPC_CALLID>73</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_ADD</OPCODE>
|
||||
<DATA>
|
||||
<TXID>67</TXID>
|
||||
<TXID>58</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>16393</INODEID>
|
||||
<PATH>/hard-lease-recovery-test</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1388429890261</MTIME>
|
||||
<ATIME>1388429890261</ATIME>
|
||||
<MTIME>1389045295543</MTIME>
|
||||
<ATIME>1389045295543</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-1396063717_1</CLIENT_NAME>
|
||||
<CLIENT_NAME>DFSClient_NONMAPREDUCE_-216163116_1</CLIENT_NAME>
|
||||
<CLIENT_MACHINE>127.0.0.1</CLIENT_MACHINE>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>andrew</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
<RPC_CLIENTID>bfe81b9e-5c10-4f90-a5e1-b707da7bb781</RPC_CLIENTID>
|
||||
<RPC_CALLID>74</RPC_CALLID>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>65</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_ALLOCATE_BLOCK_ID</OPCODE>
|
||||
<DATA>
|
||||
<TXID>68</TXID>
|
||||
<TXID>59</TXID>
|
||||
<BLOCK_ID>1073741834</BLOCK_ID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
|
||||
<DATA>
|
||||
<TXID>69</TXID>
|
||||
<TXID>60</TXID>
|
||||
<GENSTAMPV2>1010</GENSTAMPV2>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_UPDATE_BLOCKS</OPCODE>
|
||||
<OPCODE>OP_ADD_BLOCK</OPCODE>
|
||||
<DATA>
|
||||
<TXID>70</TXID>
|
||||
<TXID>61</TXID>
|
||||
<PATH>/hard-lease-recovery-test</PATH>
|
||||
<BLOCK>
|
||||
<BLOCK_ID>1073741834</BLOCK_ID>
|
||||
|
@ -932,7 +799,7 @@
|
|||
<RECORD>
|
||||
<OPCODE>OP_UPDATE_BLOCKS</OPCODE>
|
||||
<DATA>
|
||||
<TXID>71</TXID>
|
||||
<TXID>62</TXID>
|
||||
<PATH>/hard-lease-recovery-test</PATH>
|
||||
<BLOCK>
|
||||
<BLOCK_ID>1073741834</BLOCK_ID>
|
||||
|
@ -946,31 +813,15 @@
|
|||
<RECORD>
|
||||
<OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
|
||||
<DATA>
|
||||
<TXID>72</TXID>
|
||||
<TXID>63</TXID>
|
||||
<GENSTAMPV2>1011</GENSTAMPV2>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_REASSIGN_LEASE</OPCODE>
|
||||
<DATA>
|
||||
<TXID>73</TXID>
|
||||
<LEASEHOLDER>DFSClient_NONMAPREDUCE_-1396063717_1</LEASEHOLDER>
|
||||
<PATH>/hard-lease-recovery-test</PATH>
|
||||
<NEWHOLDER>HDFS_NameNode</NEWHOLDER>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_SET_GENSTAMP_V2</OPCODE>
|
||||
<DATA>
|
||||
<TXID>74</TXID>
|
||||
<GENSTAMPV2>1012</GENSTAMPV2>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_REASSIGN_LEASE</OPCODE>
|
||||
<DATA>
|
||||
<TXID>75</TXID>
|
||||
<LEASEHOLDER>HDFS_NameNode</LEASEHOLDER>
|
||||
<TXID>64</TXID>
|
||||
<LEASEHOLDER>DFSClient_NONMAPREDUCE_-216163116_1</LEASEHOLDER>
|
||||
<PATH>/hard-lease-recovery-test</PATH>
|
||||
<NEWHOLDER>HDFS_NameNode</NEWHOLDER>
|
||||
</DATA>
|
||||
|
@ -978,32 +829,97 @@
|
|||
<RECORD>
|
||||
<OPCODE>OP_CLOSE</OPCODE>
|
||||
<DATA>
|
||||
<TXID>76</TXID>
|
||||
<TXID>65</TXID>
|
||||
<LENGTH>0</LENGTH>
|
||||
<INODEID>0</INODEID>
|
||||
<PATH>/hard-lease-recovery-test</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<MTIME>1388429895216</MTIME>
|
||||
<ATIME>1388429890261</ATIME>
|
||||
<MTIME>1389045298180</MTIME>
|
||||
<ATIME>1389045295543</ATIME>
|
||||
<BLOCKSIZE>512</BLOCKSIZE>
|
||||
<CLIENT_NAME></CLIENT_NAME>
|
||||
<CLIENT_MACHINE></CLIENT_MACHINE>
|
||||
<BLOCK>
|
||||
<BLOCK_ID>1073741834</BLOCK_ID>
|
||||
<NUM_BYTES>11</NUM_BYTES>
|
||||
<GENSTAMP>1012</GENSTAMP>
|
||||
<GENSTAMP>1011</GENSTAMP>
|
||||
</BLOCK>
|
||||
<PERMISSION_STATUS>
|
||||
<USERNAME>andrew</USERNAME>
|
||||
<USERNAME>jing</USERNAME>
|
||||
<GROUPNAME>supergroup</GROUPNAME>
|
||||
<MODE>420</MODE>
|
||||
</PERMISSION_STATUS>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_ADD_CACHE_POOL</OPCODE>
|
||||
<DATA>
|
||||
<TXID>66</TXID>
|
||||
<POOLNAME>pool1</POOLNAME>
|
||||
<OWNERNAME>jing</OWNERNAME>
|
||||
<GROUPNAME>staff</GROUPNAME>
|
||||
<MODE>493</MODE>
|
||||
<LIMIT>9223372036854775807</LIMIT>
|
||||
<MAXRELATIVEEXPIRY>2305843009213693951</MAXRELATIVEEXPIRY>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>72</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_MODIFY_CACHE_POOL</OPCODE>
|
||||
<DATA>
|
||||
<TXID>67</TXID>
|
||||
<POOLNAME>pool1</POOLNAME>
|
||||
<LIMIT>99</LIMIT>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>73</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_ADD_CACHE_DIRECTIVE</OPCODE>
|
||||
<DATA>
|
||||
<TXID>68</TXID>
|
||||
<ID>1</ID>
|
||||
<PATH>/path</PATH>
|
||||
<REPLICATION>1</REPLICATION>
|
||||
<POOL>pool1</POOL>
|
||||
<EXPIRATION>2305844398258992525</EXPIRATION>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>74</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_MODIFY_CACHE_DIRECTIVE</OPCODE>
|
||||
<DATA>
|
||||
<TXID>69</TXID>
|
||||
<ID>1</ID>
|
||||
<REPLICATION>2</REPLICATION>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>75</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_REMOVE_CACHE_DIRECTIVE</OPCODE>
|
||||
<DATA>
|
||||
<TXID>70</TXID>
|
||||
<ID>1</ID>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>76</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_REMOVE_CACHE_POOL</OPCODE>
|
||||
<DATA>
|
||||
<TXID>71</TXID>
|
||||
<POOLNAME>pool1</POOLNAME>
|
||||
<RPC_CLIENTID>48c96601-9238-4d1f-b78b-ef0f1e922ba2</RPC_CLIENTID>
|
||||
<RPC_CALLID>77</RPC_CALLID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
<RECORD>
|
||||
<OPCODE>OP_END_LOG_SEGMENT</OPCODE>
|
||||
<DATA>
|
||||
<TXID>77</TXID>
|
||||
<TXID>72</TXID>
|
||||
</DATA>
|
||||
</RECORD>
|
||||
</EDITS>
|
||||
|
|
Loading…
Reference in New Issue