|
@@ -58,6 +58,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
|
|
|
import org.apache.hadoop.hdfs.util.Holder;
|
|
|
+import org.apache.hadoop.io.IOUtils;
|
|
|
|
|
|
import com.google.common.base.Joiner;
|
|
|
|
|
@@ -76,52 +77,41 @@ public class FSEditLogLoader {
|
|
|
* This is where we apply edits that we've been writing to disk all
|
|
|
* along.
|
|
|
*/
|
|
|
- int loadFSEdits(EditLogInputStream edits, long expectedStartingTxId)
|
|
|
- throws IOException {
|
|
|
- long startTime = now();
|
|
|
- fsNamesys.writeLock();
|
|
|
- try {
|
|
|
- int numEdits = loadFSEdits(edits, true, expectedStartingTxId);
|
|
|
- FSImage.LOG.info("Edits file " + edits.getName()
|
|
|
- + " of size " + edits.length() + " edits # " + numEdits
|
|
|
- + " loaded in " + (now()-startTime)/1000 + " seconds.");
|
|
|
- return numEdits;
|
|
|
- } finally {
|
|
|
- fsNamesys.writeUnlock();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private int loadFSEdits(EditLogInputStream edits, boolean closeOnExit,
|
|
|
- long expectedStartingTxId)
|
|
|
+ long loadFSEdits(EditLogInputStream edits, long expectedStartingTxId)
|
|
|
throws IOException {
|
|
|
- int numEdits = 0;
|
|
|
+ long numEdits = 0;
|
|
|
int logVersion = edits.getVersion();
|
|
|
|
|
|
+ fsNamesys.writeLock();
|
|
|
try {
|
|
|
+ long startTime = now();
|
|
|
numEdits = loadEditRecords(logVersion, edits, false,
|
|
|
expectedStartingTxId);
|
|
|
+ FSImage.LOG.info("Edits file " + edits.getName()
|
|
|
+ + " of size " + edits.length() + " edits # " + numEdits
|
|
|
+ + " loaded in " + (now()-startTime)/1000 + " seconds.");
|
|
|
} finally {
|
|
|
fsNamesys.setBlockTotal();
|
|
|
+
|
|
|
// Delay the notification of genstamp updates until after
|
|
|
// setBlockTotal() above. Otherwise, we will mark blocks
|
|
|
// as "safe" before they've been incorporated in the expected
|
|
|
// totalBlocks and threshold for SafeMode -- triggering an
|
|
|
// assertion failure and/or exiting safemode too early!
|
|
|
fsNamesys.notifyGenStampUpdate(maxGenStamp);
|
|
|
- if(closeOnExit) {
|
|
|
- edits.close();
|
|
|
- }
|
|
|
+
|
|
|
+ edits.close();
|
|
|
+ fsNamesys.writeUnlock();
|
|
|
}
|
|
|
|
|
|
return numEdits;
|
|
|
}
|
|
|
|
|
|
- @SuppressWarnings("deprecation")
|
|
|
- int loadEditRecords(int logVersion, EditLogInputStream in, boolean closeOnExit,
|
|
|
+ long loadEditRecords(int logVersion, EditLogInputStream in, boolean closeOnExit,
|
|
|
long expectedStartingTxId)
|
|
|
- throws IOException {
|
|
|
+ throws IOException, EditLogInputException {
|
|
|
FSDirectory fsDir = fsNamesys.dir;
|
|
|
- int numEdits = 0;
|
|
|
+ long numEdits = 0;
|
|
|
|
|
|
EnumMap<FSEditLogOpCodes, Holder<Integer>> opCounts =
|
|
|
new EnumMap<FSEditLogOpCodes, Holder<Integer>>(FSEditLogOpCodes.class);
|
|
@@ -136,9 +126,19 @@ public class FSEditLogLoader {
|
|
|
long txId = expectedStartingTxId - 1;
|
|
|
|
|
|
try {
|
|
|
- FSEditLogOp op;
|
|
|
- while ((op = in.readOp()) != null) {
|
|
|
- recentOpcodeOffsets[numEdits % recentOpcodeOffsets.length] =
|
|
|
+ while (true) {
|
|
|
+ FSEditLogOp op;
|
|
|
+ try {
|
|
|
+ if ((op = in.readOp()) == null) {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ String errorMessage = formatEditLogReplayError(in, recentOpcodeOffsets);
|
|
|
+ FSImage.LOG.error(errorMessage);
|
|
|
+ throw new EditLogInputException(errorMessage,
|
|
|
+ ioe, numEdits);
|
|
|
+ }
|
|
|
+ recentOpcodeOffsets[(int)(numEdits % recentOpcodeOffsets.length)] =
|
|
|
in.getPosition();
|
|
|
if (LayoutVersion.supports(Feature.STORED_TXIDS, logVersion)) {
|
|
|
long thisTxId = op.txid;
|
|
@@ -149,279 +149,291 @@ public class FSEditLogLoader {
|
|
|
txId = thisTxId;
|
|
|
}
|
|
|
|
|
|
- numEdits++;
|
|
|
incrOpCount(op.opCode, opCounts);
|
|
|
- switch (op.opCode) {
|
|
|
- case OP_ADD: {
|
|
|
- AddCloseOp addCloseOp = (AddCloseOp)op;
|
|
|
-
|
|
|
- // See if the file already exists (persistBlocks call)
|
|
|
- INodeFile oldFile = getINodeFile(fsDir, addCloseOp.path);
|
|
|
- if (oldFile == null) { // this is OP_ADD on a new file
|
|
|
- // versions > 0 support per file replication
|
|
|
- // get name and replication
|
|
|
- final short replication = fsNamesys.getBlockManager(
|
|
|
- ).adjustReplication(addCloseOp.replication);
|
|
|
- PermissionStatus permissions = fsNamesys.getUpgradePermission();
|
|
|
- if (addCloseOp.permissions != null) {
|
|
|
- permissions = addCloseOp.permissions;
|
|
|
- }
|
|
|
- long blockSize = addCloseOp.blockSize;
|
|
|
-
|
|
|
- if (FSNamesystem.LOG.isDebugEnabled()) {
|
|
|
- FSNamesystem.LOG.debug(op.opCode + ": " + addCloseOp.path +
|
|
|
- " numblocks : " + addCloseOp.blocks.length +
|
|
|
- " clientHolder " + addCloseOp.clientName +
|
|
|
- " clientMachine " + addCloseOp.clientMachine);
|
|
|
- }
|
|
|
-
|
|
|
- // Older versions of HDFS does not store the block size in inode.
|
|
|
- // If the file has more than one block, use the size of the
|
|
|
- // first block as the blocksize. Otherwise use the default
|
|
|
- // block size.
|
|
|
- if (-8 <= logVersion && blockSize == 0) {
|
|
|
- if (addCloseOp.blocks.length > 1) {
|
|
|
- blockSize = addCloseOp.blocks[0].getNumBytes();
|
|
|
- } else {
|
|
|
- long first = ((addCloseOp.blocks.length == 1)?
|
|
|
- addCloseOp.blocks[0].getNumBytes(): 0);
|
|
|
- blockSize = Math.max(fsNamesys.getDefaultBlockSize(), first);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // TODO: We should do away with this add-then-replace dance.
|
|
|
-
|
|
|
- // add to the file tree
|
|
|
- INodeFile node = (INodeFile)fsDir.unprotectedAddFile(
|
|
|
- addCloseOp.path, permissions,
|
|
|
- replication, addCloseOp.mtime,
|
|
|
- addCloseOp.atime, blockSize);
|
|
|
-
|
|
|
- fsNamesys.prepareFileForWrite(addCloseOp.path, node,
|
|
|
- addCloseOp.clientName, addCloseOp.clientMachine, null);
|
|
|
- } else { // This is OP_ADD on an existing file
|
|
|
- if (!oldFile.isUnderConstruction()) {
|
|
|
- // This is a call to append() on an already-closed file.
|
|
|
- fsNamesys.prepareFileForWrite(addCloseOp.path, oldFile,
|
|
|
- addCloseOp.clientName, addCloseOp.clientMachine, null);
|
|
|
- oldFile = getINodeFile(fsDir, addCloseOp.path);
|
|
|
- }
|
|
|
-
|
|
|
- updateBlocks(fsDir, addCloseOp, oldFile);
|
|
|
- }
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_CLOSE: {
|
|
|
- AddCloseOp addCloseOp = (AddCloseOp)op;
|
|
|
-
|
|
|
- INodeFile oldFile = getINodeFile(fsDir, addCloseOp.path);
|
|
|
- if (oldFile == null) {
|
|
|
- throw new IOException("Operation trying to close non-existent file " +
|
|
|
- addCloseOp.path);
|
|
|
- }
|
|
|
-
|
|
|
- // Update in-memory data structures
|
|
|
- updateBlocks(fsDir, addCloseOp, oldFile);
|
|
|
-
|
|
|
- // Now close the file
|
|
|
- INodeFileUnderConstruction ucFile = (INodeFileUnderConstruction) oldFile;
|
|
|
- // TODO: we could use removeLease(holder, path) here, but OP_CLOSE
|
|
|
- // doesn't seem to serialize the holder... unclear why!
|
|
|
- fsNamesys.leaseManager.removeLeaseWithPrefixPath(addCloseOp.path);
|
|
|
- INodeFile newFile = ucFile.convertToInodeFile();
|
|
|
- fsDir.replaceNode(addCloseOp.path, ucFile, newFile);
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_SET_REPLICATION: {
|
|
|
- SetReplicationOp setReplicationOp = (SetReplicationOp)op;
|
|
|
- short replication = fsNamesys.getBlockManager().adjustReplication(
|
|
|
- setReplicationOp.replication);
|
|
|
- fsDir.unprotectedSetReplication(setReplicationOp.path,
|
|
|
- replication, null);
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_CONCAT_DELETE: {
|
|
|
- ConcatDeleteOp concatDeleteOp = (ConcatDeleteOp)op;
|
|
|
- fsDir.unprotectedConcat(concatDeleteOp.trg, concatDeleteOp.srcs,
|
|
|
- concatDeleteOp.timestamp);
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_RENAME_OLD: {
|
|
|
- RenameOldOp renameOp = (RenameOldOp)op;
|
|
|
- HdfsFileStatus dinfo = fsDir.getFileInfo(renameOp.dst, false);
|
|
|
- fsDir.unprotectedRenameTo(renameOp.src, renameOp.dst,
|
|
|
- renameOp.timestamp);
|
|
|
- fsNamesys.unprotectedChangeLease(renameOp.src, renameOp.dst, dinfo);
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_DELETE: {
|
|
|
- DeleteOp deleteOp = (DeleteOp)op;
|
|
|
- fsDir.unprotectedDelete(deleteOp.path, deleteOp.timestamp);
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_MKDIR: {
|
|
|
- MkdirOp mkdirOp = (MkdirOp)op;
|
|
|
- PermissionStatus permissions = fsNamesys.getUpgradePermission();
|
|
|
- if (mkdirOp.permissions != null) {
|
|
|
- permissions = mkdirOp.permissions;
|
|
|
- }
|
|
|
-
|
|
|
- fsDir.unprotectedMkdir(mkdirOp.path, permissions,
|
|
|
- mkdirOp.timestamp);
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_SET_GENSTAMP: {
|
|
|
- SetGenstampOp setGenstampOp = (SetGenstampOp)op;
|
|
|
- fsNamesys.setGenerationStamp(setGenstampOp.genStamp);
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_SET_PERMISSIONS: {
|
|
|
- SetPermissionsOp setPermissionsOp = (SetPermissionsOp)op;
|
|
|
- fsDir.unprotectedSetPermission(setPermissionsOp.src,
|
|
|
- setPermissionsOp.permissions);
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_SET_OWNER: {
|
|
|
- SetOwnerOp setOwnerOp = (SetOwnerOp)op;
|
|
|
- fsDir.unprotectedSetOwner(setOwnerOp.src, setOwnerOp.username,
|
|
|
- setOwnerOp.groupname);
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_SET_NS_QUOTA: {
|
|
|
- SetNSQuotaOp setNSQuotaOp = (SetNSQuotaOp)op;
|
|
|
- fsDir.unprotectedSetQuota(setNSQuotaOp.src,
|
|
|
- setNSQuotaOp.nsQuota,
|
|
|
- HdfsConstants.QUOTA_DONT_SET);
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_CLEAR_NS_QUOTA: {
|
|
|
- ClearNSQuotaOp clearNSQuotaOp = (ClearNSQuotaOp)op;
|
|
|
- fsDir.unprotectedSetQuota(clearNSQuotaOp.src,
|
|
|
- HdfsConstants.QUOTA_RESET,
|
|
|
- HdfsConstants.QUOTA_DONT_SET);
|
|
|
- break;
|
|
|
- }
|
|
|
-
|
|
|
- case OP_SET_QUOTA:
|
|
|
- SetQuotaOp setQuotaOp = (SetQuotaOp)op;
|
|
|
- fsDir.unprotectedSetQuota(setQuotaOp.src,
|
|
|
- setQuotaOp.nsQuota,
|
|
|
- setQuotaOp.dsQuota);
|
|
|
- break;
|
|
|
-
|
|
|
- case OP_TIMES: {
|
|
|
- TimesOp timesOp = (TimesOp)op;
|
|
|
-
|
|
|
- fsDir.unprotectedSetTimes(timesOp.path,
|
|
|
- timesOp.mtime,
|
|
|
- timesOp.atime, true);
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_SYMLINK: {
|
|
|
- SymlinkOp symlinkOp = (SymlinkOp)op;
|
|
|
- fsDir.unprotectedSymlink(symlinkOp.path, symlinkOp.value,
|
|
|
- symlinkOp.mtime, symlinkOp.atime,
|
|
|
- symlinkOp.permissionStatus);
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_RENAME: {
|
|
|
- RenameOp renameOp = (RenameOp)op;
|
|
|
-
|
|
|
- HdfsFileStatus dinfo = fsDir.getFileInfo(renameOp.dst, false);
|
|
|
- fsDir.unprotectedRenameTo(renameOp.src, renameOp.dst,
|
|
|
- renameOp.timestamp, renameOp.options);
|
|
|
- fsNamesys.unprotectedChangeLease(renameOp.src, renameOp.dst, dinfo);
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_GET_DELEGATION_TOKEN: {
|
|
|
- GetDelegationTokenOp getDelegationTokenOp
|
|
|
- = (GetDelegationTokenOp)op;
|
|
|
-
|
|
|
- fsNamesys.getDelegationTokenSecretManager()
|
|
|
- .addPersistedDelegationToken(getDelegationTokenOp.token,
|
|
|
- getDelegationTokenOp.expiryTime);
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_RENEW_DELEGATION_TOKEN: {
|
|
|
- RenewDelegationTokenOp renewDelegationTokenOp
|
|
|
- = (RenewDelegationTokenOp)op;
|
|
|
- fsNamesys.getDelegationTokenSecretManager()
|
|
|
- .updatePersistedTokenRenewal(renewDelegationTokenOp.token,
|
|
|
- renewDelegationTokenOp.expiryTime);
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_CANCEL_DELEGATION_TOKEN: {
|
|
|
- CancelDelegationTokenOp cancelDelegationTokenOp
|
|
|
- = (CancelDelegationTokenOp)op;
|
|
|
- fsNamesys.getDelegationTokenSecretManager()
|
|
|
- .updatePersistedTokenCancellation(
|
|
|
- cancelDelegationTokenOp.token);
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_UPDATE_MASTER_KEY: {
|
|
|
- UpdateMasterKeyOp updateMasterKeyOp = (UpdateMasterKeyOp)op;
|
|
|
- fsNamesys.getDelegationTokenSecretManager()
|
|
|
- .updatePersistedMasterKey(updateMasterKeyOp.key);
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_REASSIGN_LEASE: {
|
|
|
- ReassignLeaseOp reassignLeaseOp = (ReassignLeaseOp)op;
|
|
|
-
|
|
|
- Lease lease = fsNamesys.leaseManager.getLease(
|
|
|
- reassignLeaseOp.leaseHolder);
|
|
|
- INodeFileUnderConstruction pendingFile =
|
|
|
- (INodeFileUnderConstruction) fsDir.getFileINode(
|
|
|
- reassignLeaseOp.path);
|
|
|
- fsNamesys.reassignLeaseInternal(lease,
|
|
|
- reassignLeaseOp.path, reassignLeaseOp.newHolder, pendingFile);
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_START_LOG_SEGMENT:
|
|
|
- case OP_END_LOG_SEGMENT: {
|
|
|
- // no data in here currently.
|
|
|
- break;
|
|
|
- }
|
|
|
- case OP_DATANODE_ADD:
|
|
|
- case OP_DATANODE_REMOVE:
|
|
|
- break;
|
|
|
- default:
|
|
|
- throw new IOException("Invalid operation read " + op.opCode);
|
|
|
+ try {
|
|
|
+ applyEditLogOp(op, fsDir, logVersion);
|
|
|
+ } catch (Throwable t) {
|
|
|
+ // Catch Throwable because in the case of a truly corrupt edits log, any
|
|
|
+ // sort of error might be thrown (NumberFormat, NullPointer, EOF, etc.)
|
|
|
+ String errorMessage = formatEditLogReplayError(in, recentOpcodeOffsets);
|
|
|
+ FSImage.LOG.error(errorMessage);
|
|
|
+ throw new IOException(errorMessage, t);
|
|
|
}
|
|
|
+ numEdits++;
|
|
|
}
|
|
|
-
|
|
|
} catch (IOException ex) {
|
|
|
check203UpgradeFailure(logVersion, ex);
|
|
|
} finally {
|
|
|
if(closeOnExit)
|
|
|
in.close();
|
|
|
}
|
|
|
- } catch (Throwable t) {
|
|
|
- // Catch Throwable because in the case of a truly corrupt edits log, any
|
|
|
- // sort of error might be thrown (NumberFormat, NullPointer, EOF, etc.)
|
|
|
- StringBuilder sb = new StringBuilder();
|
|
|
- sb.append("Error replaying edit log at offset " + in.getPosition());
|
|
|
- if (recentOpcodeOffsets[0] != -1) {
|
|
|
- Arrays.sort(recentOpcodeOffsets);
|
|
|
- sb.append("\nRecent opcode offsets:");
|
|
|
- for (long offset : recentOpcodeOffsets) {
|
|
|
- if (offset != -1) {
|
|
|
- sb.append(' ').append(offset);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- String errorMessage = sb.toString();
|
|
|
- FSImage.LOG.error(errorMessage);
|
|
|
- throw new IOException(errorMessage, t);
|
|
|
} finally {
|
|
|
fsDir.writeUnlock();
|
|
|
fsNamesys.writeUnlock();
|
|
|
- }
|
|
|
- if (FSImage.LOG.isDebugEnabled()) {
|
|
|
- dumpOpCounts(opCounts);
|
|
|
+ if (FSImage.LOG.isDebugEnabled()) {
|
|
|
+ dumpOpCounts(opCounts);
|
|
|
+ }
|
|
|
}
|
|
|
return numEdits;
|
|
|
}
|
|
|
|
|
|
+ @SuppressWarnings("deprecation")
|
|
|
+ private void applyEditLogOp(FSEditLogOp op, FSDirectory fsDir,
|
|
|
+ int logVersion) throws IOException {
|
|
|
+ switch (op.opCode) {
|
|
|
+ case OP_ADD: {
|
|
|
+ AddCloseOp addCloseOp = (AddCloseOp)op;
|
|
|
+
|
|
|
+ // See if the file already exists (persistBlocks call)
|
|
|
+ INodeFile oldFile = getINodeFile(fsDir, addCloseOp.path);
|
|
|
+ if (oldFile == null) { // this is OP_ADD on a new file
|
|
|
+ // versions > 0 support per file replication
|
|
|
+ // get name and replication
|
|
|
+ final short replication = fsNamesys.getBlockManager(
|
|
|
+ ).adjustReplication(addCloseOp.replication);
|
|
|
+ PermissionStatus permissions = fsNamesys.getUpgradePermission();
|
|
|
+ if (addCloseOp.permissions != null) {
|
|
|
+ permissions = addCloseOp.permissions;
|
|
|
+ }
|
|
|
+ long blockSize = addCloseOp.blockSize;
|
|
|
+
|
|
|
+ if (FSNamesystem.LOG.isDebugEnabled()) {
|
|
|
+ FSNamesystem.LOG.debug(op.opCode + ": " + addCloseOp.path +
|
|
|
+ " numblocks : " + addCloseOp.blocks.length +
|
|
|
+ " clientHolder " + addCloseOp.clientName +
|
|
|
+ " clientMachine " + addCloseOp.clientMachine);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Older versions of HDFS does not store the block size in inode.
|
|
|
+ // If the file has more than one block, use the size of the
|
|
|
+ // first block as the blocksize. Otherwise use the default
|
|
|
+ // block size.
|
|
|
+ if (-8 <= logVersion && blockSize == 0) {
|
|
|
+ if (addCloseOp.blocks.length > 1) {
|
|
|
+ blockSize = addCloseOp.blocks[0].getNumBytes();
|
|
|
+ } else {
|
|
|
+ long first = ((addCloseOp.blocks.length == 1)?
|
|
|
+ addCloseOp.blocks[0].getNumBytes(): 0);
|
|
|
+ blockSize = Math.max(fsNamesys.getDefaultBlockSize(), first);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // TODO: We should do away with this add-then-replace dance.
|
|
|
+
|
|
|
+ // add to the file tree
|
|
|
+ INodeFile node = (INodeFile)fsDir.unprotectedAddFile(
|
|
|
+ addCloseOp.path, permissions,
|
|
|
+ replication, addCloseOp.mtime,
|
|
|
+ addCloseOp.atime, blockSize);
|
|
|
+
|
|
|
+ fsNamesys.prepareFileForWrite(addCloseOp.path, node,
|
|
|
+ addCloseOp.clientName, addCloseOp.clientMachine, null);
|
|
|
+ } else { // This is OP_ADD on an existing file
|
|
|
+ if (!oldFile.isUnderConstruction()) {
|
|
|
+ // This is a call to append() on an already-closed file.
|
|
|
+ fsNamesys.prepareFileForWrite(addCloseOp.path, oldFile,
|
|
|
+ addCloseOp.clientName, addCloseOp.clientMachine, null);
|
|
|
+ oldFile = getINodeFile(fsDir, addCloseOp.path);
|
|
|
+ }
|
|
|
+
|
|
|
+ updateBlocks(fsDir, addCloseOp, oldFile);
|
|
|
+ }
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_CLOSE: {
|
|
|
+ AddCloseOp addCloseOp = (AddCloseOp)op;
|
|
|
+
|
|
|
+ INodeFile oldFile = getINodeFile(fsDir, addCloseOp.path);
|
|
|
+ if (oldFile == null) {
|
|
|
+ throw new IOException("Operation trying to close non-existent file " +
|
|
|
+ addCloseOp.path);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Update in-memory data structures
|
|
|
+ updateBlocks(fsDir, addCloseOp, oldFile);
|
|
|
+
|
|
|
+ // Now close the file
|
|
|
+ INodeFileUnderConstruction ucFile = (INodeFileUnderConstruction) oldFile;
|
|
|
+ // TODO: we could use removeLease(holder, path) here, but OP_CLOSE
|
|
|
+ // doesn't seem to serialize the holder... unclear why!
|
|
|
+ fsNamesys.leaseManager.removeLeaseWithPrefixPath(addCloseOp.path);
|
|
|
+ INodeFile newFile = ucFile.convertToInodeFile();
|
|
|
+ fsDir.replaceNode(addCloseOp.path, ucFile, newFile);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_SET_REPLICATION: {
|
|
|
+ SetReplicationOp setReplicationOp = (SetReplicationOp)op;
|
|
|
+ short replication = fsNamesys.getBlockManager().adjustReplication(
|
|
|
+ setReplicationOp.replication);
|
|
|
+ fsDir.unprotectedSetReplication(setReplicationOp.path,
|
|
|
+ replication, null);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_CONCAT_DELETE: {
|
|
|
+ ConcatDeleteOp concatDeleteOp = (ConcatDeleteOp)op;
|
|
|
+ fsDir.unprotectedConcat(concatDeleteOp.trg, concatDeleteOp.srcs,
|
|
|
+ concatDeleteOp.timestamp);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_RENAME_OLD: {
|
|
|
+ RenameOldOp renameOp = (RenameOldOp)op;
|
|
|
+ HdfsFileStatus dinfo = fsDir.getFileInfo(renameOp.dst, false);
|
|
|
+ fsDir.unprotectedRenameTo(renameOp.src, renameOp.dst,
|
|
|
+ renameOp.timestamp);
|
|
|
+ fsNamesys.unprotectedChangeLease(renameOp.src, renameOp.dst, dinfo);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_DELETE: {
|
|
|
+ DeleteOp deleteOp = (DeleteOp)op;
|
|
|
+ fsDir.unprotectedDelete(deleteOp.path, deleteOp.timestamp);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_MKDIR: {
|
|
|
+ MkdirOp mkdirOp = (MkdirOp)op;
|
|
|
+ PermissionStatus permissions = fsNamesys.getUpgradePermission();
|
|
|
+ if (mkdirOp.permissions != null) {
|
|
|
+ permissions = mkdirOp.permissions;
|
|
|
+ }
|
|
|
+
|
|
|
+ fsDir.unprotectedMkdir(mkdirOp.path, permissions,
|
|
|
+ mkdirOp.timestamp);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_SET_GENSTAMP: {
|
|
|
+ SetGenstampOp setGenstampOp = (SetGenstampOp)op;
|
|
|
+ fsNamesys.setGenerationStamp(setGenstampOp.genStamp);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_SET_PERMISSIONS: {
|
|
|
+ SetPermissionsOp setPermissionsOp = (SetPermissionsOp)op;
|
|
|
+ fsDir.unprotectedSetPermission(setPermissionsOp.src,
|
|
|
+ setPermissionsOp.permissions);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_SET_OWNER: {
|
|
|
+ SetOwnerOp setOwnerOp = (SetOwnerOp)op;
|
|
|
+ fsDir.unprotectedSetOwner(setOwnerOp.src, setOwnerOp.username,
|
|
|
+ setOwnerOp.groupname);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_SET_NS_QUOTA: {
|
|
|
+ SetNSQuotaOp setNSQuotaOp = (SetNSQuotaOp)op;
|
|
|
+ fsDir.unprotectedSetQuota(setNSQuotaOp.src,
|
|
|
+ setNSQuotaOp.nsQuota,
|
|
|
+ HdfsConstants.QUOTA_DONT_SET);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_CLEAR_NS_QUOTA: {
|
|
|
+ ClearNSQuotaOp clearNSQuotaOp = (ClearNSQuotaOp)op;
|
|
|
+ fsDir.unprotectedSetQuota(clearNSQuotaOp.src,
|
|
|
+ HdfsConstants.QUOTA_RESET,
|
|
|
+ HdfsConstants.QUOTA_DONT_SET);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+
|
|
|
+ case OP_SET_QUOTA:
|
|
|
+ SetQuotaOp setQuotaOp = (SetQuotaOp)op;
|
|
|
+ fsDir.unprotectedSetQuota(setQuotaOp.src,
|
|
|
+ setQuotaOp.nsQuota,
|
|
|
+ setQuotaOp.dsQuota);
|
|
|
+ break;
|
|
|
+
|
|
|
+ case OP_TIMES: {
|
|
|
+ TimesOp timesOp = (TimesOp)op;
|
|
|
+
|
|
|
+ fsDir.unprotectedSetTimes(timesOp.path,
|
|
|
+ timesOp.mtime,
|
|
|
+ timesOp.atime, true);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_SYMLINK: {
|
|
|
+ SymlinkOp symlinkOp = (SymlinkOp)op;
|
|
|
+ fsDir.unprotectedSymlink(symlinkOp.path, symlinkOp.value,
|
|
|
+ symlinkOp.mtime, symlinkOp.atime,
|
|
|
+ symlinkOp.permissionStatus);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_RENAME: {
|
|
|
+ RenameOp renameOp = (RenameOp)op;
|
|
|
+
|
|
|
+ HdfsFileStatus dinfo = fsDir.getFileInfo(renameOp.dst, false);
|
|
|
+ fsDir.unprotectedRenameTo(renameOp.src, renameOp.dst,
|
|
|
+ renameOp.timestamp, renameOp.options);
|
|
|
+ fsNamesys.unprotectedChangeLease(renameOp.src, renameOp.dst, dinfo);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_GET_DELEGATION_TOKEN: {
|
|
|
+ GetDelegationTokenOp getDelegationTokenOp
|
|
|
+ = (GetDelegationTokenOp)op;
|
|
|
+
|
|
|
+ fsNamesys.getDelegationTokenSecretManager()
|
|
|
+ .addPersistedDelegationToken(getDelegationTokenOp.token,
|
|
|
+ getDelegationTokenOp.expiryTime);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_RENEW_DELEGATION_TOKEN: {
|
|
|
+ RenewDelegationTokenOp renewDelegationTokenOp
|
|
|
+ = (RenewDelegationTokenOp)op;
|
|
|
+ fsNamesys.getDelegationTokenSecretManager()
|
|
|
+ .updatePersistedTokenRenewal(renewDelegationTokenOp.token,
|
|
|
+ renewDelegationTokenOp.expiryTime);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_CANCEL_DELEGATION_TOKEN: {
|
|
|
+ CancelDelegationTokenOp cancelDelegationTokenOp
|
|
|
+ = (CancelDelegationTokenOp)op;
|
|
|
+ fsNamesys.getDelegationTokenSecretManager()
|
|
|
+ .updatePersistedTokenCancellation(
|
|
|
+ cancelDelegationTokenOp.token);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_UPDATE_MASTER_KEY: {
|
|
|
+ UpdateMasterKeyOp updateMasterKeyOp = (UpdateMasterKeyOp)op;
|
|
|
+ fsNamesys.getDelegationTokenSecretManager()
|
|
|
+ .updatePersistedMasterKey(updateMasterKeyOp.key);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_REASSIGN_LEASE: {
|
|
|
+ ReassignLeaseOp reassignLeaseOp = (ReassignLeaseOp)op;
|
|
|
+
|
|
|
+ Lease lease = fsNamesys.leaseManager.getLease(
|
|
|
+ reassignLeaseOp.leaseHolder);
|
|
|
+ INodeFileUnderConstruction pendingFile =
|
|
|
+ (INodeFileUnderConstruction) fsDir.getFileINode(
|
|
|
+ reassignLeaseOp.path);
|
|
|
+ fsNamesys.reassignLeaseInternal(lease,
|
|
|
+ reassignLeaseOp.path, reassignLeaseOp.newHolder, pendingFile);
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_START_LOG_SEGMENT:
|
|
|
+ case OP_END_LOG_SEGMENT: {
|
|
|
+ // no data in here currently.
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ case OP_DATANODE_ADD:
|
|
|
+ case OP_DATANODE_REMOVE:
|
|
|
+ break;
|
|
|
+ default:
|
|
|
+ throw new IOException("Invalid operation read " + op.opCode);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private static String formatEditLogReplayError(EditLogInputStream in,
|
|
|
+ long recentOpcodeOffsets[]) {
|
|
|
+ StringBuilder sb = new StringBuilder();
|
|
|
+ sb.append("Error replaying edit log at offset " + in.getPosition());
|
|
|
+ if (recentOpcodeOffsets[0] != -1) {
|
|
|
+ Arrays.sort(recentOpcodeOffsets);
|
|
|
+ sb.append("\nRecent opcode offsets:");
|
|
|
+ for (long offset : recentOpcodeOffsets) {
|
|
|
+ if (offset != -1) {
|
|
|
+ sb.append(' ').append(offset);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return sb.toString();
|
|
|
+ }
|
|
|
+
|
|
|
private static INodeFile getINodeFile(FSDirectory fsDir, String path)
|
|
|
throws IOException {
|
|
|
INode inode = fsDir.getINode(path);
|