|
@@ -19,10 +19,12 @@ package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
+import java.io.DataOutputStream;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
import java.util.zip.Checksum;
|
|
|
+import java.util.zip.CheckedOutputStream;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -30,7 +32,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
|
import org.apache.hadoop.fs.Options;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
-import org.apache.hadoop.hdfs.DeprecatedUTF8;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
|
|
import org.apache.hadoop.hdfs.server.common.Storage;
|
|
@@ -43,14 +44,13 @@ import org.apache.hadoop.hdfs.server.namenode.NNStorage.NNStorageListener;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.JournalStream.JournalType;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
|
|
|
-import org.apache.hadoop.io.ArrayWritable;
|
|
|
-import org.apache.hadoop.io.BytesWritable;
|
|
|
import org.apache.hadoop.io.LongWritable;
|
|
|
import org.apache.hadoop.io.Writable;
|
|
|
import org.apache.hadoop.security.token.delegation.DelegationKey;
|
|
|
import org.apache.hadoop.util.PureJavaCrc32;
|
|
|
|
|
|
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.*;
|
|
|
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.*;
|
|
|
|
|
|
/**
|
|
|
* FSEditLog maintains a log of the namespace modifications.
|
|
@@ -319,7 +319,7 @@ public class FSEditLog implements NNStorageListener {
|
|
|
* Write an operation to the edit log. Do not sync to persistent
|
|
|
* store yet.
|
|
|
*/
|
|
|
- void logEdit(FSEditLogOpCodes opCode, Writable ... writables) {
|
|
|
+ void logEdit(FSEditLogOp op) {
|
|
|
synchronized (this) {
|
|
|
// wait if an automatic sync is scheduled
|
|
|
waitIfAutoSyncScheduled();
|
|
@@ -329,10 +329,10 @@ public class FSEditLog implements NNStorageListener {
|
|
|
ArrayList<EditLogOutputStream> errorStreams = null;
|
|
|
long start = now();
|
|
|
for(EditLogOutputStream eStream : editStreams) {
|
|
|
- if(!eStream.isOperationSupported(opCode.getOpCode()))
|
|
|
+ if(!eStream.isOperationSupported(op.opCode.getOpCode()))
|
|
|
continue;
|
|
|
try {
|
|
|
- eStream.write(opCode.getOpCode(), writables);
|
|
|
+ eStream.write(op);
|
|
|
} catch (IOException ie) {
|
|
|
LOG.error("logEdit: removing "+ eStream.getName(), ie);
|
|
|
if(errorStreams == null)
|
|
@@ -585,49 +585,45 @@ public class FSEditLog implements NNStorageListener {
|
|
|
* Records the block locations of the last block.
|
|
|
*/
|
|
|
public void logOpenFile(String path, INodeFileUnderConstruction newNode) {
|
|
|
-
|
|
|
- DeprecatedUTF8 nameReplicationPair[] = new DeprecatedUTF8[] {
|
|
|
- new DeprecatedUTF8(path),
|
|
|
- FSEditLog.toLogReplication(newNode.getReplication()),
|
|
|
- FSEditLog.toLogLong(newNode.getModificationTime()),
|
|
|
- FSEditLog.toLogLong(newNode.getAccessTime()),
|
|
|
- FSEditLog.toLogLong(newNode.getPreferredBlockSize())};
|
|
|
- logEdit(OP_ADD,
|
|
|
- new ArrayWritable(DeprecatedUTF8.class, nameReplicationPair),
|
|
|
- new ArrayWritable(Block.class, newNode.getBlocks()),
|
|
|
- newNode.getPermissionStatus(),
|
|
|
- new DeprecatedUTF8(newNode.getClientName()),
|
|
|
- new DeprecatedUTF8(newNode.getClientMachine()));
|
|
|
+ AddOp op = AddOp.getInstance()
|
|
|
+ .setPath(path)
|
|
|
+ .setReplication(newNode.getReplication())
|
|
|
+ .setModificationTime(newNode.getModificationTime())
|
|
|
+ .setAccessTime(newNode.getAccessTime())
|
|
|
+ .setBlockSize(newNode.getPreferredBlockSize())
|
|
|
+ .setBlocks(newNode.getBlocks())
|
|
|
+ .setPermissionStatus(newNode.getPermissionStatus())
|
|
|
+ .setClientName(newNode.getClientName())
|
|
|
+ .setClientMachine(newNode.getClientMachine());
|
|
|
+
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Add close lease record to edit log.
|
|
|
*/
|
|
|
public void logCloseFile(String path, INodeFile newNode) {
|
|
|
- DeprecatedUTF8 nameReplicationPair[] = new DeprecatedUTF8[] {
|
|
|
- new DeprecatedUTF8(path),
|
|
|
- FSEditLog.toLogReplication(newNode.getReplication()),
|
|
|
- FSEditLog.toLogLong(newNode.getModificationTime()),
|
|
|
- FSEditLog.toLogLong(newNode.getAccessTime()),
|
|
|
- FSEditLog.toLogLong(newNode.getPreferredBlockSize())};
|
|
|
- logEdit(OP_CLOSE,
|
|
|
- new ArrayWritable(DeprecatedUTF8.class, nameReplicationPair),
|
|
|
- new ArrayWritable(Block.class, newNode.getBlocks()),
|
|
|
- newNode.getPermissionStatus());
|
|
|
+ CloseOp op = CloseOp.getInstance()
|
|
|
+ .setPath(path)
|
|
|
+ .setReplication(newNode.getReplication())
|
|
|
+ .setModificationTime(newNode.getModificationTime())
|
|
|
+ .setAccessTime(newNode.getAccessTime())
|
|
|
+ .setBlockSize(newNode.getPreferredBlockSize())
|
|
|
+ .setBlocks(newNode.getBlocks())
|
|
|
+ .setPermissionStatus(newNode.getPermissionStatus());
|
|
|
+
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Add create directory record to edit log
|
|
|
*/
|
|
|
public void logMkDir(String path, INode newNode) {
|
|
|
- DeprecatedUTF8 info[] = new DeprecatedUTF8[] {
|
|
|
- new DeprecatedUTF8(path),
|
|
|
- FSEditLog.toLogLong(newNode.getModificationTime()),
|
|
|
- FSEditLog.toLogLong(newNode.getAccessTime())
|
|
|
- };
|
|
|
- logEdit(OP_MKDIR,
|
|
|
- new ArrayWritable(DeprecatedUTF8.class, info),
|
|
|
- newNode.getPermissionStatus());
|
|
|
+ MkdirOp op = MkdirOp.getInstance()
|
|
|
+ .setPath(path)
|
|
|
+ .setTimestamp(newNode.getModificationTime())
|
|
|
+ .setPermissionStatus(newNode.getPermissionStatus());
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -635,33 +631,33 @@ public class FSEditLog implements NNStorageListener {
|
|
|
* TODO: use String parameters until just before writing to disk
|
|
|
*/
|
|
|
void logRename(String src, String dst, long timestamp) {
|
|
|
- DeprecatedUTF8 info[] = new DeprecatedUTF8[] {
|
|
|
- new DeprecatedUTF8(src),
|
|
|
- new DeprecatedUTF8(dst),
|
|
|
- FSEditLog.toLogLong(timestamp)};
|
|
|
- logEdit(OP_RENAME_OLD, new ArrayWritable(DeprecatedUTF8.class, info));
|
|
|
+ RenameOldOp op = RenameOldOp.getInstance()
|
|
|
+ .setSource(src)
|
|
|
+ .setDestination(dst)
|
|
|
+ .setTimestamp(timestamp);
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Add rename record to edit log
|
|
|
*/
|
|
|
void logRename(String src, String dst, long timestamp, Options.Rename... options) {
|
|
|
- DeprecatedUTF8 info[] = new DeprecatedUTF8[] {
|
|
|
- new DeprecatedUTF8(src),
|
|
|
- new DeprecatedUTF8(dst),
|
|
|
- FSEditLog.toLogLong(timestamp)};
|
|
|
- logEdit(OP_RENAME,
|
|
|
- new ArrayWritable(DeprecatedUTF8.class, info),
|
|
|
- toBytesWritable(options));
|
|
|
+ RenameOp op = RenameOp.getInstance()
|
|
|
+ .setSource(src)
|
|
|
+ .setDestination(dst)
|
|
|
+ .setTimestamp(timestamp)
|
|
|
+ .setOptions(options);
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Add set replication record to edit log
|
|
|
*/
|
|
|
void logSetReplication(String src, short replication) {
|
|
|
- logEdit(OP_SET_REPLICATION,
|
|
|
- new DeprecatedUTF8(src),
|
|
|
- FSEditLog.toLogReplication(replication));
|
|
|
+ SetReplicationOp op = SetReplicationOp.getInstance()
|
|
|
+ .setPath(src)
|
|
|
+ .setReplication(replication);
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
/** Add set namespace quota record to edit log
|
|
@@ -670,64 +666,69 @@ public class FSEditLog implements NNStorageListener {
|
|
|
* @param quota the directory size limit
|
|
|
*/
|
|
|
void logSetQuota(String src, long nsQuota, long dsQuota) {
|
|
|
- logEdit(OP_SET_QUOTA,
|
|
|
- new DeprecatedUTF8(src),
|
|
|
- new LongWritable(nsQuota), new LongWritable(dsQuota));
|
|
|
+ SetQuotaOp op = SetQuotaOp.getInstance()
|
|
|
+ .setSource(src)
|
|
|
+ .setNSQuota(nsQuota)
|
|
|
+ .setDSQuota(dsQuota);
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
/** Add set permissions record to edit log */
|
|
|
void logSetPermissions(String src, FsPermission permissions) {
|
|
|
- logEdit(OP_SET_PERMISSIONS, new DeprecatedUTF8(src), permissions);
|
|
|
+ SetPermissionsOp op = SetPermissionsOp.getInstance()
|
|
|
+ .setSource(src)
|
|
|
+ .setPermissions(permissions);
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
/** Add set owner record to edit log */
|
|
|
void logSetOwner(String src, String username, String groupname) {
|
|
|
- DeprecatedUTF8 u = new DeprecatedUTF8(username == null? "": username);
|
|
|
- DeprecatedUTF8 g = new DeprecatedUTF8(groupname == null? "": groupname);
|
|
|
- logEdit(OP_SET_OWNER, new DeprecatedUTF8(src), u, g);
|
|
|
+ SetOwnerOp op = SetOwnerOp.getInstance()
|
|
|
+ .setSource(src)
|
|
|
+ .setUser(username)
|
|
|
+ .setGroup(groupname);
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* concat(trg,src..) log
|
|
|
*/
|
|
|
void logConcat(String trg, String [] srcs, long timestamp) {
|
|
|
- int size = 1 + srcs.length + 1; // trg, srcs, timestamp
|
|
|
- DeprecatedUTF8 info[] = new DeprecatedUTF8[size];
|
|
|
- int idx = 0;
|
|
|
- info[idx++] = new DeprecatedUTF8(trg);
|
|
|
- for(int i=0; i<srcs.length; i++) {
|
|
|
- info[idx++] = new DeprecatedUTF8(srcs[i]);
|
|
|
- }
|
|
|
- info[idx] = FSEditLog.toLogLong(timestamp);
|
|
|
- logEdit(OP_CONCAT_DELETE, new ArrayWritable(DeprecatedUTF8.class, info));
|
|
|
+ ConcatDeleteOp op = ConcatDeleteOp.getInstance()
|
|
|
+ .setTarget(trg)
|
|
|
+ .setSources(srcs)
|
|
|
+ .setTimestamp(timestamp);
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Add delete file record to edit log
|
|
|
*/
|
|
|
void logDelete(String src, long timestamp) {
|
|
|
- DeprecatedUTF8 info[] = new DeprecatedUTF8[] {
|
|
|
- new DeprecatedUTF8(src),
|
|
|
- FSEditLog.toLogLong(timestamp)};
|
|
|
- logEdit(OP_DELETE, new ArrayWritable(DeprecatedUTF8.class, info));
|
|
|
+ DeleteOp op = DeleteOp.getInstance()
|
|
|
+ .setPath(src)
|
|
|
+ .setTimestamp(timestamp);
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Add generation stamp record to edit log
|
|
|
*/
|
|
|
void logGenerationStamp(long genstamp) {
|
|
|
- logEdit(OP_SET_GENSTAMP, new LongWritable(genstamp));
|
|
|
+ SetGenstampOp op = SetGenstampOp.getInstance()
|
|
|
+ .setGenerationStamp(genstamp);
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
* Add access time record to edit log
|
|
|
*/
|
|
|
void logTimes(String src, long mtime, long atime) {
|
|
|
- DeprecatedUTF8 info[] = new DeprecatedUTF8[] {
|
|
|
- new DeprecatedUTF8(src),
|
|
|
- FSEditLog.toLogLong(mtime),
|
|
|
- FSEditLog.toLogLong(atime)};
|
|
|
- logEdit(OP_TIMES, new ArrayWritable(DeprecatedUTF8.class, info));
|
|
|
+ TimesOp op = TimesOp.getInstance()
|
|
|
+ .setPath(src)
|
|
|
+ .setModificationTime(mtime)
|
|
|
+ .setAccessTime(atime);
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -735,14 +736,13 @@ public class FSEditLog implements NNStorageListener {
|
|
|
*/
|
|
|
void logSymlink(String path, String value, long mtime,
|
|
|
long atime, INodeSymlink node) {
|
|
|
- DeprecatedUTF8 info[] = new DeprecatedUTF8[] {
|
|
|
- new DeprecatedUTF8(path),
|
|
|
- new DeprecatedUTF8(value),
|
|
|
- FSEditLog.toLogLong(mtime),
|
|
|
- FSEditLog.toLogLong(atime)};
|
|
|
- logEdit(OP_SYMLINK,
|
|
|
- new ArrayWritable(DeprecatedUTF8.class, info),
|
|
|
- node.getPermissionStatus());
|
|
|
+ SymlinkOp op = SymlinkOp.getInstance()
|
|
|
+ .setPath(path)
|
|
|
+ .setValue(value)
|
|
|
+ .setModificationTime(mtime)
|
|
|
+ .setAccessTime(atime)
|
|
|
+ .setPermissionStatus(node.getPermissionStatus());
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -753,36 +753,40 @@ public class FSEditLog implements NNStorageListener {
|
|
|
*/
|
|
|
void logGetDelegationToken(DelegationTokenIdentifier id,
|
|
|
long expiryTime) {
|
|
|
- logEdit(OP_GET_DELEGATION_TOKEN, id, FSEditLog.toLogLong(expiryTime));
|
|
|
+ GetDelegationTokenOp op = GetDelegationTokenOp.getInstance()
|
|
|
+ .setDelegationTokenIdentifier(id)
|
|
|
+ .setExpiryTime(expiryTime);
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
void logRenewDelegationToken(DelegationTokenIdentifier id,
|
|
|
long expiryTime) {
|
|
|
- logEdit(OP_RENEW_DELEGATION_TOKEN, id, FSEditLog.toLogLong(expiryTime));
|
|
|
+ RenewDelegationTokenOp op = RenewDelegationTokenOp.getInstance()
|
|
|
+ .setDelegationTokenIdentifier(id)
|
|
|
+ .setExpiryTime(expiryTime);
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
void logCancelDelegationToken(DelegationTokenIdentifier id) {
|
|
|
- logEdit(OP_CANCEL_DELEGATION_TOKEN, id);
|
|
|
+ CancelDelegationTokenOp op = CancelDelegationTokenOp.getInstance()
|
|
|
+ .setDelegationTokenIdentifier(id);
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
void logUpdateMasterKey(DelegationKey key) {
|
|
|
- logEdit(OP_UPDATE_MASTER_KEY, key);
|
|
|
+ UpdateMasterKeyOp op = UpdateMasterKeyOp.getInstance()
|
|
|
+ .setDelegationKey(key);
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
void logReassignLease(String leaseHolder, String src, String newHolder) {
|
|
|
- logEdit(OP_REASSIGN_LEASE, new DeprecatedUTF8(leaseHolder),
|
|
|
- new DeprecatedUTF8(src),
|
|
|
- new DeprecatedUTF8(newHolder));
|
|
|
- }
|
|
|
-
|
|
|
- static private DeprecatedUTF8 toLogReplication(short replication) {
|
|
|
- return new DeprecatedUTF8(Short.toString(replication));
|
|
|
+ ReassignLeaseOp op = ReassignLeaseOp.getInstance()
|
|
|
+ .setLeaseHolder(leaseHolder)
|
|
|
+ .setPath(src)
|
|
|
+ .setNewHolder(newHolder);
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
- static private DeprecatedUTF8 toLogLong(long timestamp) {
|
|
|
- return new DeprecatedUTF8(Long.toString(timestamp));
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Return the size of the current EditLog
|
|
|
*/
|
|
@@ -1030,7 +1034,7 @@ public class FSEditLog implements NNStorageListener {
|
|
|
boStream = new EditLogBackupOutputStream(bnReg, nnReg);
|
|
|
editStreams.add(boStream);
|
|
|
}
|
|
|
- logEdit(OP_JSPOOL_START, (Writable[])null);
|
|
|
+ logEdit(JSpoolStartOp.getInstance());
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1044,7 +1048,7 @@ public class FSEditLog implements NNStorageListener {
|
|
|
long start = now();
|
|
|
for(EditLogOutputStream eStream : editStreams) {
|
|
|
try {
|
|
|
- eStream.write(data, 0, length);
|
|
|
+ eStream.writeRaw(data, 0, length);
|
|
|
} catch (IOException ie) {
|
|
|
LOG.warn("Error in editStream " + eStream.getName(), ie);
|
|
|
if(errorStreams == null)
|
|
@@ -1127,8 +1131,9 @@ public class FSEditLog implements NNStorageListener {
|
|
|
|
|
|
void incrementCheckpointTime() {
|
|
|
storage.incrementCheckpointTime();
|
|
|
- Writable[] args = {new LongWritable(storage.getCheckpointTime())};
|
|
|
- logEdit(OP_CHECKPOINT_TIME, args);
|
|
|
+ CheckpointTimeOp op = CheckpointTimeOp.getInstance()
|
|
|
+ .setCheckpointTime(storage.getCheckpointTime());
|
|
|
+ logEdit(op);
|
|
|
}
|
|
|
|
|
|
synchronized void releaseBackupStream(NamenodeRegistration registration) {
|
|
@@ -1179,13 +1184,6 @@ public class FSEditLog implements NNStorageListener {
|
|
|
return regAllowed;
|
|
|
}
|
|
|
|
|
|
- static BytesWritable toBytesWritable(Options.Rename... options) {
|
|
|
- byte[] bytes = new byte[options.length];
|
|
|
- for (int i = 0; i < options.length; i++) {
|
|
|
- bytes[i] = options[i].value();
|
|
|
- }
|
|
|
- return new BytesWritable(bytes);
|
|
|
- }
|
|
|
|
|
|
/**
|
|
|
* Get the StorageDirectory for a stream
|