001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hdfs.server.namenode;
019
020import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD;
021import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_APPEND;
022import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_BLOCK;
023import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_CACHE_DIRECTIVE;
024import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ADD_CACHE_POOL;
025import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ALLOCATE_BLOCK_ID;
026import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ALLOW_SNAPSHOT;
027import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_CANCEL_DELEGATION_TOKEN;
028import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_CLEAR_NS_QUOTA;
029import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_CLOSE;
030import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_CONCAT_DELETE;
031import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_CREATE_SNAPSHOT;
032import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_DELETE;
033import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_DELETE_SNAPSHOT;
034import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_DISALLOW_SNAPSHOT;
035import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_END_LOG_SEGMENT;
036import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_GET_DELEGATION_TOKEN;
037import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_INVALID;
038import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_MKDIR;
039import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_MODIFY_CACHE_DIRECTIVE;
040import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_MODIFY_CACHE_POOL;
041import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_REASSIGN_LEASE;
042import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_REMOVE_CACHE_DIRECTIVE;
043import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_REMOVE_CACHE_POOL;
044import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_REMOVE_XATTR;
045import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_RENAME;
046import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_RENAME_OLD;
047import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_RENAME_SNAPSHOT;
048import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_RENEW_DELEGATION_TOKEN;
049import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_ACL;
050import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ROLLING_UPGRADE_FINALIZE;
051import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_ROLLING_UPGRADE_START;
052import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_GENSTAMP_V1;
053import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_GENSTAMP_V2;
054import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_NS_QUOTA;
055import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_OWNER;
056import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_PERMISSIONS;
057import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_QUOTA;
058import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_REPLICATION;
059import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_XATTR;
060import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_START_LOG_SEGMENT;
061import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SYMLINK;
062import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_TIMES;
063import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_TRUNCATE;
064import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_UPDATE_BLOCKS;
065import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_UPDATE_MASTER_KEY;
066import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_STORAGE_POLICY;
067import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.OP_SET_QUOTA_BY_STORAGETYPE;
068
069import java.io.DataInput;
070import java.io.DataInputStream;
071import java.io.DataOutput;
072import java.io.DataOutputStream;
073import java.io.EOFException;
074import java.io.IOException;
075import java.util.ArrayList;
076import java.util.Arrays;
077import java.util.EnumMap;
078import java.util.List;
079import java.util.zip.CheckedInputStream;
080import java.util.zip.Checksum;
081
082import org.apache.commons.codec.DecoderException;
083import org.apache.commons.codec.binary.Hex;
084import org.apache.hadoop.classification.InterfaceAudience;
085import org.apache.hadoop.classification.InterfaceStability;
086import org.apache.hadoop.fs.ChecksumException;
087import org.apache.hadoop.fs.Options.Rename;
088import org.apache.hadoop.fs.XAttr;
089import org.apache.hadoop.fs.XAttrCodec;
090import org.apache.hadoop.fs.permission.AclEntry;
091import org.apache.hadoop.fs.permission.AclEntryScope;
092import org.apache.hadoop.fs.permission.AclEntryType;
093import org.apache.hadoop.fs.permission.FsAction;
094import org.apache.hadoop.fs.permission.FsPermission;
095import org.apache.hadoop.fs.permission.PermissionStatus;
096import org.apache.hadoop.fs.StorageType;
097import org.apache.hadoop.hdfs.DFSConfigKeys;
098import org.apache.hadoop.hdfs.DeprecatedUTF8;
099import org.apache.hadoop.hdfs.protocol.Block;
100import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
101import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
102import org.apache.hadoop.hdfs.protocol.ClientProtocol;
103import org.apache.hadoop.hdfs.protocol.HdfsConstants;
104import org.apache.hadoop.hdfs.protocol.LayoutVersion;
105import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
106import org.apache.hadoop.hdfs.protocol.proto.AclProtos.AclEditLogProto;
107import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.XAttrEditLogProto;
108import org.apache.hadoop.hdfs.protocolPB.PBHelper;
109import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
110import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
111import org.apache.hadoop.hdfs.util.XMLUtils;
112import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
113import org.apache.hadoop.hdfs.util.XMLUtils.Stanza;
114import org.apache.hadoop.io.ArrayWritable;
115import org.apache.hadoop.io.BytesWritable;
116import org.apache.hadoop.io.DataOutputBuffer;
117import org.apache.hadoop.io.IOUtils;
118import org.apache.hadoop.io.Text;
119import org.apache.hadoop.io.Writable;
120import org.apache.hadoop.io.WritableFactories;
121import org.apache.hadoop.io.WritableFactory;
122import org.apache.hadoop.ipc.ClientId;
123import org.apache.hadoop.ipc.RpcConstants;
124import org.apache.hadoop.security.token.delegation.DelegationKey;
125import org.apache.hadoop.util.DataChecksum;
126import org.apache.hadoop.util.StringUtils;
127import org.xml.sax.ContentHandler;
128import org.xml.sax.SAXException;
129import org.xml.sax.helpers.AttributesImpl;
130
131import com.google.common.annotations.VisibleForTesting;
132import com.google.common.base.Joiner;
133import com.google.common.base.Preconditions;
134import com.google.common.collect.ImmutableMap;
135import com.google.common.collect.Lists;
136
137/**
138 * Helper classes for reading the ops from an InputStream.
139 * All ops derive from FSEditLogOp and are only
140 * instantiated from Reader#readOp()
141 */
142@InterfaceAudience.Private
143@InterfaceStability.Unstable
144public abstract class FSEditLogOp {
145  public final FSEditLogOpCodes opCode;
146  long txid;
147  byte[] rpcClientId;
148  int rpcCallId;
149
150  final void reset() {
151    txid = HdfsConstants.INVALID_TXID;
152    rpcClientId = RpcConstants.DUMMY_CLIENT_ID;
153    rpcCallId = RpcConstants.INVALID_CALL_ID;
154    resetSubFields();
155  }
156
157  abstract void resetSubFields();
158
159  final public static class OpInstanceCache {
160    private final EnumMap<FSEditLogOpCodes, FSEditLogOp> inst =
161        new EnumMap<FSEditLogOpCodes, FSEditLogOp>(FSEditLogOpCodes.class);
162    
163    public OpInstanceCache() {
164      inst.put(OP_ADD, new AddOp());
165      inst.put(OP_CLOSE, new CloseOp());
166      inst.put(OP_SET_REPLICATION, new SetReplicationOp());
167      inst.put(OP_CONCAT_DELETE, new ConcatDeleteOp());
168      inst.put(OP_RENAME_OLD, new RenameOldOp());
169      inst.put(OP_DELETE, new DeleteOp());
170      inst.put(OP_MKDIR, new MkdirOp());
171      inst.put(OP_SET_GENSTAMP_V1, new SetGenstampV1Op());
172      inst.put(OP_SET_PERMISSIONS, new SetPermissionsOp());
173      inst.put(OP_SET_OWNER, new SetOwnerOp());
174      inst.put(OP_SET_NS_QUOTA, new SetNSQuotaOp());
175      inst.put(OP_CLEAR_NS_QUOTA, new ClearNSQuotaOp());
176      inst.put(OP_SET_QUOTA, new SetQuotaOp());
177      inst.put(OP_TIMES, new TimesOp());
178      inst.put(OP_SYMLINK, new SymlinkOp());
179      inst.put(OP_RENAME, new RenameOp());
180      inst.put(OP_REASSIGN_LEASE, new ReassignLeaseOp());
181      inst.put(OP_GET_DELEGATION_TOKEN, new GetDelegationTokenOp());
182      inst.put(OP_RENEW_DELEGATION_TOKEN, new RenewDelegationTokenOp());
183      inst.put(OP_CANCEL_DELEGATION_TOKEN, new CancelDelegationTokenOp());
184      inst.put(OP_UPDATE_MASTER_KEY, new UpdateMasterKeyOp());
185      inst.put(OP_START_LOG_SEGMENT, new LogSegmentOp(OP_START_LOG_SEGMENT));
186      inst.put(OP_END_LOG_SEGMENT, new LogSegmentOp(OP_END_LOG_SEGMENT));
187      inst.put(OP_UPDATE_BLOCKS, new UpdateBlocksOp());
188      inst.put(OP_TRUNCATE, new TruncateOp());
189
190      inst.put(OP_ALLOW_SNAPSHOT, new AllowSnapshotOp());
191      inst.put(OP_DISALLOW_SNAPSHOT, new DisallowSnapshotOp());
192      inst.put(OP_CREATE_SNAPSHOT, new CreateSnapshotOp());
193      inst.put(OP_DELETE_SNAPSHOT, new DeleteSnapshotOp());
194      inst.put(OP_RENAME_SNAPSHOT, new RenameSnapshotOp());
195      inst.put(OP_SET_GENSTAMP_V2, new SetGenstampV2Op());
196      inst.put(OP_ALLOCATE_BLOCK_ID, new AllocateBlockIdOp());
197      inst.put(OP_ADD_BLOCK, new AddBlockOp());
198      inst.put(OP_ADD_CACHE_DIRECTIVE,
199          new AddCacheDirectiveInfoOp());
200      inst.put(OP_MODIFY_CACHE_DIRECTIVE,
201          new ModifyCacheDirectiveInfoOp());
202      inst.put(OP_REMOVE_CACHE_DIRECTIVE,
203          new RemoveCacheDirectiveInfoOp());
204      inst.put(OP_ADD_CACHE_POOL, new AddCachePoolOp());
205      inst.put(OP_MODIFY_CACHE_POOL, new ModifyCachePoolOp());
206      inst.put(OP_REMOVE_CACHE_POOL, new RemoveCachePoolOp());
207
208      inst.put(OP_SET_ACL, new SetAclOp());
209      inst.put(OP_ROLLING_UPGRADE_START, new RollingUpgradeOp(
210          OP_ROLLING_UPGRADE_START, "start"));
211      inst.put(OP_ROLLING_UPGRADE_FINALIZE, new RollingUpgradeOp(
212          OP_ROLLING_UPGRADE_FINALIZE, "finalize"));
213      inst.put(OP_SET_XATTR, new SetXAttrOp());
214      inst.put(OP_REMOVE_XATTR, new RemoveXAttrOp());
215      inst.put(OP_SET_STORAGE_POLICY, new SetStoragePolicyOp());
216      inst.put(OP_APPEND, new AppendOp());
217      inst.put(OP_SET_QUOTA_BY_STORAGETYPE, new SetQuotaByStorageTypeOp());
218    }
219    
220    public FSEditLogOp get(FSEditLogOpCodes opcode) {
221      return inst.get(opcode);
222    }
223  }
224
225  private static ImmutableMap<String, FsAction> fsActionMap() {
226    ImmutableMap.Builder<String, FsAction> b = ImmutableMap.builder();
227    for (FsAction v : FsAction.values())
228      b.put(v.SYMBOL, v);
229    return b.build();
230  }
231
232  private static final ImmutableMap<String, FsAction> FSACTION_SYMBOL_MAP
233    = fsActionMap();
234
235  /**
236   * Constructor for an EditLog Op. EditLog ops cannot be constructed
237   * directly, but only through Reader#readOp.
238   */
239  @VisibleForTesting
240  protected FSEditLogOp(FSEditLogOpCodes opCode) {
241    this.opCode = opCode;
242    reset();
243  }
244
245  public long getTransactionId() {
246    Preconditions.checkState(txid != HdfsConstants.INVALID_TXID);
247    return txid;
248  }
249
250  public String getTransactionIdStr() {
251    return (txid == HdfsConstants.INVALID_TXID) ? "(none)" : "" + txid;
252  }
253  
254  public boolean hasTransactionId() {
255    return (txid != HdfsConstants.INVALID_TXID);
256  }
257
258  public void setTransactionId(long txid) {
259    this.txid = txid;
260  }
261  
262  public boolean hasRpcIds() {
263    return rpcClientId != RpcConstants.DUMMY_CLIENT_ID
264        && rpcCallId != RpcConstants.INVALID_CALL_ID;
265  }
266  
267  /** this has to be called after calling {@link #hasRpcIds()} */
268  public byte[] getClientId() {
269    Preconditions.checkState(rpcClientId != RpcConstants.DUMMY_CLIENT_ID);
270    return rpcClientId;
271  }
272  
273  public void setRpcClientId(byte[] clientId) {
274    this.rpcClientId = clientId;
275  }
276  
277  /** this has to be called after calling {@link #hasRpcIds()} */
278  public int getCallId() {
279    Preconditions.checkState(rpcCallId != RpcConstants.INVALID_CALL_ID);
280    return rpcCallId;
281  }
282  
283  public void setRpcCallId(int callId) {
284    this.rpcCallId = callId;
285  }
286
287  abstract void readFields(DataInputStream in, int logVersion)
288      throws IOException;
289
290  public abstract void writeFields(DataOutputStream out)
291      throws IOException;
292
293  static interface BlockListUpdatingOp {
294    Block[] getBlocks();
295    String getPath();
296    boolean shouldCompleteLastBlock();
297  }
298  
299  private static void writeRpcIds(final byte[] clientId, final int callId,
300      DataOutputStream out) throws IOException {
301    FSImageSerialization.writeBytes(clientId, out);
302    FSImageSerialization.writeInt(callId, out);
303  }
304  
305  void readRpcIds(DataInputStream in, int logVersion)
306      throws IOException {
307    if (NameNodeLayoutVersion.supports(
308        LayoutVersion.Feature.EDITLOG_SUPPORT_RETRYCACHE, logVersion)) {
309      this.rpcClientId = FSImageSerialization.readBytes(in);
310      this.rpcCallId = FSImageSerialization.readInt(in);
311    }
312  }
313  
314  void readRpcIdsFromXml(Stanza st) {
315    this.rpcClientId = st.hasChildren("RPC_CLIENTID") ? 
316        ClientId.toBytes(st.getValue("RPC_CLIENTID"))
317        : RpcConstants.DUMMY_CLIENT_ID;
318    this.rpcCallId = st.hasChildren("RPC_CALLID") ? 
319        Integer.parseInt(st.getValue("RPC_CALLID"))
320        : RpcConstants.INVALID_CALL_ID;
321  }
322  
323  private static void appendRpcIdsToString(final StringBuilder builder,
324      final byte[] clientId, final int callId) {
325    builder.append(", RpcClientId=");
326    builder.append(ClientId.toString(clientId));
327    builder.append(", RpcCallId=");
328    builder.append(callId);
329  }
330  
331  private static void appendRpcIdsToXml(ContentHandler contentHandler,
332      final byte[] clientId, final int callId) throws SAXException {
333    XMLUtils.addSaxString(contentHandler, "RPC_CLIENTID",
334        ClientId.toString(clientId));
335    XMLUtils.addSaxString(contentHandler, "RPC_CALLID", 
336        Integer.toString(callId));
337  }
338
339  private static final class AclEditLogUtil {
340    private static final int ACL_EDITLOG_ENTRY_HAS_NAME_OFFSET = 6;
341    private static final int ACL_EDITLOG_ENTRY_TYPE_OFFSET = 3;
342    private static final int ACL_EDITLOG_ENTRY_SCOPE_OFFSET = 5;
343    private static final int ACL_EDITLOG_PERM_MASK = 7;
344    private static final int ACL_EDITLOG_ENTRY_TYPE_MASK = 3;
345    private static final int ACL_EDITLOG_ENTRY_SCOPE_MASK = 1;
346
347    private static final FsAction[] FSACTION_VALUES = FsAction.values();
348    private static final AclEntryScope[] ACL_ENTRY_SCOPE_VALUES = AclEntryScope
349        .values();
350    private static final AclEntryType[] ACL_ENTRY_TYPE_VALUES = AclEntryType
351        .values();
352
353    private static List<AclEntry> read(DataInputStream in, int logVersion)
354        throws IOException {
355      if (!NameNodeLayoutVersion.supports(Feature.EXTENDED_ACL, logVersion)) {
356        return null;
357      }
358
359      int size = in.readInt();
360      if (size == 0) {
361        return null;
362      }
363
364      List<AclEntry> aclEntries = Lists.newArrayListWithCapacity(size);
365      for (int i = 0; i < size; ++i) {
366        int v = in.read();
367        int p = v & ACL_EDITLOG_PERM_MASK;
368        int t = (v >> ACL_EDITLOG_ENTRY_TYPE_OFFSET)
369            & ACL_EDITLOG_ENTRY_TYPE_MASK;
370        int s = (v >> ACL_EDITLOG_ENTRY_SCOPE_OFFSET)
371            & ACL_EDITLOG_ENTRY_SCOPE_MASK;
372        boolean hasName = ((v >> ACL_EDITLOG_ENTRY_HAS_NAME_OFFSET) & 1) == 1;
373        String name = hasName ? FSImageSerialization.readString(in) : null;
374        aclEntries.add(new AclEntry.Builder().setName(name)
375            .setPermission(FSACTION_VALUES[p])
376            .setScope(ACL_ENTRY_SCOPE_VALUES[s])
377            .setType(ACL_ENTRY_TYPE_VALUES[t]).build());
378      }
379
380      return aclEntries;
381    }
382
383    private static void write(List<AclEntry> aclEntries, DataOutputStream out)
384        throws IOException {
385      if (aclEntries == null) {
386        out.writeInt(0);
387        return;
388      }
389
390      out.writeInt(aclEntries.size());
391      for (AclEntry e : aclEntries) {
392        boolean hasName = e.getName() != null;
393        int v = (e.getScope().ordinal() << ACL_EDITLOG_ENTRY_SCOPE_OFFSET)
394            | (e.getType().ordinal() << ACL_EDITLOG_ENTRY_TYPE_OFFSET)
395            | e.getPermission().ordinal();
396
397        if (hasName) {
398          v |= 1 << ACL_EDITLOG_ENTRY_HAS_NAME_OFFSET;
399        }
400        out.write(v);
401        if (hasName) {
402          FSImageSerialization.writeString(e.getName(), out);
403        }
404      }
405    }
406  }
407
408  private static List<XAttr> readXAttrsFromEditLog(DataInputStream in,
409      int logVersion) throws IOException {
410    if (!NameNodeLayoutVersion.supports(NameNodeLayoutVersion.Feature.XATTRS,
411        logVersion)) {
412      return null;
413    }
414    XAttrEditLogProto proto = XAttrEditLogProto.parseDelimitedFrom(in);
415    return PBHelper.convertXAttrs(proto.getXAttrsList());
416  }
417
418  @SuppressWarnings("unchecked")
419  static abstract class AddCloseOp extends FSEditLogOp implements BlockListUpdatingOp {
420    int length;
421    long inodeId;
422    String path;
423    short replication;
424    long mtime;
425    long atime;
426    long blockSize;
427    Block[] blocks;
428    PermissionStatus permissions;
429    List<AclEntry> aclEntries;
430    List<XAttr> xAttrs;
431    String clientName;
432    String clientMachine;
433    boolean overwrite;
434    byte storagePolicyId;
435    
436    private AddCloseOp(FSEditLogOpCodes opCode) {
437      super(opCode);
438      storagePolicyId = BlockStoragePolicySuite.ID_UNSPECIFIED;
439      assert(opCode == OP_ADD || opCode == OP_CLOSE || opCode == OP_APPEND);
440    }
441
442    @Override
443    void resetSubFields() {
444      length = 0;
445      inodeId = 0L;
446      path = null;
447      replication = 0;
448      mtime = 0L;
449      atime = 0L;
450      blockSize = 0L;
451      blocks = null;
452      permissions = null;
453      aclEntries = null;
454      xAttrs = null;
455      clientName = null;
456      clientMachine = null;
457      overwrite = false;
458      storagePolicyId = 0;
459    }
460
461    <T extends AddCloseOp> T setInodeId(long inodeId) {
462      this.inodeId = inodeId;
463      return (T)this;
464    }
465
466    <T extends AddCloseOp> T setPath(String path) {
467      this.path = path;
468      return (T)this;
469    }
470    
471    @Override
472    public String getPath() {
473      return path;
474    }
475
476    <T extends AddCloseOp> T setReplication(short replication) {
477      this.replication = replication;
478      return (T)this;
479    }
480
481    <T extends AddCloseOp> T setModificationTime(long mtime) {
482      this.mtime = mtime;
483      return (T)this;
484    }
485
486    <T extends AddCloseOp> T setAccessTime(long atime) {
487      this.atime = atime;
488      return (T)this;
489    }
490
491    <T extends AddCloseOp> T setBlockSize(long blockSize) {
492      this.blockSize = blockSize;
493      return (T)this;
494    }
495
496    <T extends AddCloseOp> T setBlocks(Block[] blocks) {
497      if (blocks.length > MAX_BLOCKS) {
498        throw new RuntimeException("Can't have more than " + MAX_BLOCKS +
499            " in an AddCloseOp.");
500      }
501      this.blocks = blocks;
502      return (T)this;
503    }
504    
505    @Override
506    public Block[] getBlocks() {
507      return blocks;
508    }
509
510    <T extends AddCloseOp> T setPermissionStatus(PermissionStatus permissions) {
511      this.permissions = permissions;
512      return (T)this;
513    }
514
515    <T extends AddCloseOp> T setAclEntries(List<AclEntry> aclEntries) {
516      this.aclEntries = aclEntries;
517      return (T)this;
518    }
519
520    <T extends AddCloseOp> T setXAttrs(List<XAttr> xAttrs) {
521      this.xAttrs = xAttrs;
522      return (T)this;
523    }
524
525    <T extends AddCloseOp> T setClientName(String clientName) {
526      this.clientName = clientName;
527      return (T)this;
528    }
529
530    <T extends AddCloseOp> T setClientMachine(String clientMachine) {
531      this.clientMachine = clientMachine;
532      return (T)this;
533    }
534    
535    <T extends AddCloseOp> T setOverwrite(boolean overwrite) {
536      this.overwrite = overwrite;
537      return (T)this;
538    }
539
540    <T extends AddCloseOp> T setStoragePolicyId(byte storagePolicyId) {
541      this.storagePolicyId = storagePolicyId;
542      return (T)this;
543    }
544
545    @Override
546    public void writeFields(DataOutputStream out) throws IOException {
547      FSImageSerialization.writeLong(inodeId, out);
548      FSImageSerialization.writeString(path, out);
549      FSImageSerialization.writeShort(replication, out);
550      FSImageSerialization.writeLong(mtime, out);
551      FSImageSerialization.writeLong(atime, out);
552      FSImageSerialization.writeLong(blockSize, out);
553      new ArrayWritable(Block.class, blocks).write(out);
554      permissions.write(out);
555
556      if (this.opCode == OP_ADD) {
557        AclEditLogUtil.write(aclEntries, out);
558        XAttrEditLogProto.Builder b = XAttrEditLogProto.newBuilder();
559        b.addAllXAttrs(PBHelper.convertXAttrProto(xAttrs));
560        b.build().writeDelimitedTo(out);
561        FSImageSerialization.writeString(clientName,out);
562        FSImageSerialization.writeString(clientMachine,out);
563        FSImageSerialization.writeBoolean(overwrite, out);
564        FSImageSerialization.writeByte(storagePolicyId, out);
565        // write clientId and callId
566        writeRpcIds(rpcClientId, rpcCallId, out);
567      }
568    }
569
570    @Override
571    void readFields(DataInputStream in, int logVersion)
572        throws IOException {
573      if (!NameNodeLayoutVersion.supports(
574          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
575        this.length = in.readInt();
576      }
577      if (NameNodeLayoutVersion.supports(
578          LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
579        this.inodeId = in.readLong();
580      } else {
581        // The inodeId should be updated when this editLogOp is applied
582        this.inodeId = INodeId.GRANDFATHER_INODE_ID;
583      }
584      if ((-17 < logVersion && length != 4) ||
585          (logVersion <= -17 && length != 5 && !NameNodeLayoutVersion.supports(
586              LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion))) {
587        throw new IOException("Incorrect data format."  +
588                              " logVersion is " + logVersion +
589                              " but writables.length is " +
590                              length + ". ");
591      }
592      this.path = FSImageSerialization.readString(in);
593
594      if (NameNodeLayoutVersion.supports(
595          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
596        this.replication = FSImageSerialization.readShort(in);
597        this.mtime = FSImageSerialization.readLong(in);
598      } else {
599        this.replication = readShort(in);
600        this.mtime = readLong(in);
601      }
602
603      if (NameNodeLayoutVersion.supports(
604          LayoutVersion.Feature.FILE_ACCESS_TIME, logVersion)) {
605        if (NameNodeLayoutVersion.supports(
606            LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
607          this.atime = FSImageSerialization.readLong(in);
608        } else {
609          this.atime = readLong(in);
610        }
611      } else {
612        this.atime = 0;
613      }
614
615      if (NameNodeLayoutVersion.supports(
616          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
617        this.blockSize = FSImageSerialization.readLong(in);
618      } else {
619        this.blockSize = readLong(in);
620      }
621
622      this.blocks = readBlocks(in, logVersion);
623      this.permissions = PermissionStatus.read(in);
624
625      if (this.opCode == OP_ADD) {
626        aclEntries = AclEditLogUtil.read(in, logVersion);
627        this.xAttrs = readXAttrsFromEditLog(in, logVersion);
628        this.clientName = FSImageSerialization.readString(in);
629        this.clientMachine = FSImageSerialization.readString(in);
630        if (NameNodeLayoutVersion.supports(
631            NameNodeLayoutVersion.Feature.CREATE_OVERWRITE, logVersion)) {
632          this.overwrite = FSImageSerialization.readBoolean(in);
633        } else {
634          this.overwrite = false;
635        }
636        if (NameNodeLayoutVersion.supports(
637            NameNodeLayoutVersion.Feature.BLOCK_STORAGE_POLICY, logVersion)) {
638          this.storagePolicyId = FSImageSerialization.readByte(in);
639        } else {
640          this.storagePolicyId = BlockStoragePolicySuite.ID_UNSPECIFIED;
641        }
642        // read clientId and callId
643        readRpcIds(in, logVersion);
644      } else {
645        this.clientName = "";
646        this.clientMachine = "";
647      }
648    }
649
650    static final public int MAX_BLOCKS = 1024 * 1024 * 64;
651    
652    private static Block[] readBlocks(
653        DataInputStream in,
654        int logVersion) throws IOException {
655      int numBlocks = in.readInt();
656      if (numBlocks < 0) {
657        throw new IOException("invalid negative number of blocks");
658      } else if (numBlocks > MAX_BLOCKS) {
659        throw new IOException("invalid number of blocks: " + numBlocks +
660            ".  The maximum number of blocks per file is " + MAX_BLOCKS);
661      }
662      Block[] blocks = new Block[numBlocks];
663      for (int i = 0; i < numBlocks; i++) {
664        Block blk = new Block();
665        blk.readFields(in);
666        blocks[i] = blk;
667      }
668      return blocks;
669    }
670
671    public String stringifyMembers() {
672      StringBuilder builder = new StringBuilder();
673      builder.append("[length=");
674      builder.append(length);
675      builder.append(", inodeId=");
676      builder.append(inodeId);
677      builder.append(", path=");
678      builder.append(path);
679      builder.append(", replication=");
680      builder.append(replication);
681      builder.append(", mtime=");
682      builder.append(mtime);
683      builder.append(", atime=");
684      builder.append(atime);
685      builder.append(", blockSize=");
686      builder.append(blockSize);
687      builder.append(", blocks=");
688      builder.append(Arrays.toString(blocks));
689      builder.append(", permissions=");
690      builder.append(permissions);
691      builder.append(", aclEntries=");
692      builder.append(aclEntries);
693      builder.append(", clientName=");
694      builder.append(clientName);
695      builder.append(", clientMachine=");
696      builder.append(clientMachine);
697      builder.append(", overwrite=");
698      builder.append(overwrite);
699      if (this.opCode == OP_ADD) {
700        appendRpcIdsToString(builder, rpcClientId, rpcCallId);
701      }
702      builder.append(", storagePolicyId=");
703      builder.append(storagePolicyId);
704      builder.append(", opCode=");
705      builder.append(opCode);
706      builder.append(", txid=");
707      builder.append(txid);
708      builder.append("]");
709      return builder.toString();
710    }
711    
712    @Override
713    protected void toXml(ContentHandler contentHandler) throws SAXException {
714      XMLUtils.addSaxString(contentHandler, "LENGTH",
715          Integer.toString(length));
716      XMLUtils.addSaxString(contentHandler, "INODEID",
717          Long.toString(inodeId));
718      XMLUtils.addSaxString(contentHandler, "PATH", path);
719      XMLUtils.addSaxString(contentHandler, "REPLICATION",
720          Short.valueOf(replication).toString());
721      XMLUtils.addSaxString(contentHandler, "MTIME",
722          Long.toString(mtime));
723      XMLUtils.addSaxString(contentHandler, "ATIME",
724          Long.toString(atime));
725      XMLUtils.addSaxString(contentHandler, "BLOCKSIZE",
726          Long.toString(blockSize));
727      XMLUtils.addSaxString(contentHandler, "CLIENT_NAME", clientName);
728      XMLUtils.addSaxString(contentHandler, "CLIENT_MACHINE", clientMachine);
729      XMLUtils.addSaxString(contentHandler, "OVERWRITE", 
730          Boolean.toString(overwrite));
731      for (Block b : blocks) {
732        FSEditLogOp.blockToXml(contentHandler, b);
733      }
734      FSEditLogOp.permissionStatusToXml(contentHandler, permissions);
735      if (this.opCode == OP_ADD) {
736        if (aclEntries != null) {
737          appendAclEntriesToXml(contentHandler, aclEntries);
738        }
739        appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
740      }
741    }
742
743    @Override 
744    void fromXml(Stanza st) throws InvalidXmlException {
745      this.length = Integer.parseInt(st.getValue("LENGTH"));
746      this.inodeId = Long.parseLong(st.getValue("INODEID"));
747      this.path = st.getValue("PATH");
748      this.replication = Short.valueOf(st.getValue("REPLICATION"));
749      this.mtime = Long.parseLong(st.getValue("MTIME"));
750      this.atime = Long.parseLong(st.getValue("ATIME"));
751      this.blockSize = Long.parseLong(st.getValue("BLOCKSIZE"));
752
753      this.clientName = st.getValue("CLIENT_NAME");
754      this.clientMachine = st.getValue("CLIENT_MACHINE");
755      this.overwrite = Boolean.parseBoolean(st.getValueOrNull("OVERWRITE"));
756      if (st.hasChildren("BLOCK")) {
757        List<Stanza> blocks = st.getChildren("BLOCK");
758        this.blocks = new Block[blocks.size()];
759        for (int i = 0; i < blocks.size(); i++) {
760          this.blocks[i] = FSEditLogOp.blockFromXml(blocks.get(i));
761        }
762      } else {
763        this.blocks = new Block[0];
764      }
765      this.permissions = permissionStatusFromXml(st);
766      aclEntries = readAclEntriesFromXml(st);
767      readRpcIdsFromXml(st);
768    }
769  }
770
771  /**
772   * {@literal @AtMostOnce} for {@link ClientProtocol#create} and
773   * {@link ClientProtocol#append}
774   */
775  static class AddOp extends AddCloseOp {
776    private AddOp() {
777      super(OP_ADD);
778    }
779
780    static AddOp getInstance(OpInstanceCache cache) {
781      return (AddOp) cache.get(OP_ADD);
782    }
783
784    @Override
785    public boolean shouldCompleteLastBlock() {
786      return false;
787    }
788
789    @Override
790    public String toString() {
791      StringBuilder builder = new StringBuilder();
792      builder.append("AddOp ");
793      builder.append(stringifyMembers());
794      return builder.toString();
795    }
796  }
797
798  /**
799   * Although {@link ClientProtocol#append} may also log a close op, we do
800   * not need to record the rpc ids here since a successful appendFile op will
801   * finally log an AddOp.
802   */
803  static class CloseOp extends AddCloseOp {
804    private CloseOp() {
805      super(OP_CLOSE);
806    }
807
808    static CloseOp getInstance(OpInstanceCache cache) {
809      return (CloseOp)cache.get(OP_CLOSE);
810    }
811
812    @Override
813    public boolean shouldCompleteLastBlock() {
814      return true;
815    }
816
817    @Override
818    public String toString() {
819      StringBuilder builder = new StringBuilder();
820      builder.append("CloseOp ");
821      builder.append(stringifyMembers());
822      return builder.toString();
823    }
824  }
825
826  static class AppendOp extends FSEditLogOp {
827    String path;
828    String clientName;
829    String clientMachine;
830    boolean newBlock;
831
832    private AppendOp() {
833      super(OP_APPEND);
834    }
835
836    static AppendOp getInstance(OpInstanceCache cache) {
837      return (AppendOp) cache.get(OP_APPEND);
838    }
839
840    AppendOp setPath(String path) {
841      this.path = path;
842      return this;
843    }
844
845    AppendOp setClientName(String clientName) {
846      this.clientName = clientName;
847      return this;
848    }
849
850    AppendOp setClientMachine(String clientMachine) {
851      this.clientMachine = clientMachine;
852      return this;
853    }
854
855    AppendOp setNewBlock(boolean newBlock) {
856      this.newBlock = newBlock;
857      return this;
858    }
859
860    @Override
861    public String toString() {
862      StringBuilder builder = new StringBuilder();
863      builder.append("AppendOp ");
864      builder.append("[path=").append(path);
865      builder.append(", clientName=").append(clientName);
866      builder.append(", clientMachine=").append(clientMachine);
867      builder.append(", newBlock=").append(newBlock).append("]");
868      return builder.toString();
869    }
870
871    @Override
872    void resetSubFields() {
873      this.path = null;
874      this.clientName = null;
875      this.clientMachine = null;
876      this.newBlock = false;
877    }
878
879    @Override
880    void readFields(DataInputStream in, int logVersion) throws IOException {
881      this.path = FSImageSerialization.readString(in);
882      this.clientName = FSImageSerialization.readString(in);
883      this.clientMachine = FSImageSerialization.readString(in);
884      this.newBlock = FSImageSerialization.readBoolean(in);
885      readRpcIds(in, logVersion);
886    }
887
888    @Override
889    public void writeFields(DataOutputStream out) throws IOException {
890      FSImageSerialization.writeString(path, out);
891      FSImageSerialization.writeString(clientName, out);
892      FSImageSerialization.writeString(clientMachine, out);
893      FSImageSerialization.writeBoolean(newBlock, out);
894      writeRpcIds(rpcClientId, rpcCallId, out);
895    }
896
897    @Override
898    protected void toXml(ContentHandler contentHandler) throws SAXException {
899      XMLUtils.addSaxString(contentHandler, "PATH", path);
900      XMLUtils.addSaxString(contentHandler, "CLIENT_NAME", clientName);
901      XMLUtils.addSaxString(contentHandler, "CLIENT_MACHINE", clientMachine);
902      XMLUtils.addSaxString(contentHandler, "NEWBLOCK",
903          Boolean.toString(newBlock));
904      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
905    }
906
907    @Override
908    void fromXml(Stanza st) throws InvalidXmlException {
909      this.path = st.getValue("PATH");
910      this.clientName = st.getValue("CLIENT_NAME");
911      this.clientMachine = st.getValue("CLIENT_MACHINE");
912      this.newBlock = Boolean.parseBoolean(st.getValue("NEWBLOCK"));
913      readRpcIdsFromXml(st);
914    }
915  }
916  
917  static class AddBlockOp extends FSEditLogOp {
918    private String path;
919    private Block penultimateBlock;
920    private Block lastBlock;
921    
922    private AddBlockOp() {
923      super(OP_ADD_BLOCK);
924    }
925    
926    static AddBlockOp getInstance(OpInstanceCache cache) {
927      return (AddBlockOp) cache.get(OP_ADD_BLOCK);
928    }
929
930    @Override
931    void resetSubFields() {
932      path = null;
933      penultimateBlock = null;
934      lastBlock = null;
935    }
936    
937    AddBlockOp setPath(String path) {
938      this.path = path;
939      return this;
940    }
941    
942    public String getPath() {
943      return path;
944    }
945
946    AddBlockOp setPenultimateBlock(Block pBlock) {
947      this.penultimateBlock = pBlock;
948      return this;
949    }
950    
951    Block getPenultimateBlock() {
952      return penultimateBlock;
953    }
954    
955    AddBlockOp setLastBlock(Block lastBlock) {
956      this.lastBlock = lastBlock;
957      return this;
958    }
959    
960    Block getLastBlock() {
961      return lastBlock;
962    }
963
964    @Override
965    public void writeFields(DataOutputStream out) throws IOException {
966      FSImageSerialization.writeString(path, out);
967      int size = penultimateBlock != null ? 2 : 1;
968      Block[] blocks = new Block[size];
969      if (penultimateBlock != null) {
970        blocks[0] = penultimateBlock;
971      }
972      blocks[size - 1] = lastBlock;
973      FSImageSerialization.writeCompactBlockArray(blocks, out);
974      // clientId and callId
975      writeRpcIds(rpcClientId, rpcCallId, out);
976    }
977
978    @Override
979    void readFields(DataInputStream in, int logVersion) throws IOException {
980      path = FSImageSerialization.readString(in);
981      Block[] blocks = FSImageSerialization.readCompactBlockArray(in,
982          logVersion);
983      Preconditions.checkState(blocks.length == 2 || blocks.length == 1);
984      penultimateBlock = blocks.length == 1 ? null : blocks[0];
985      lastBlock = blocks[blocks.length - 1];
986      readRpcIds(in, logVersion);
987    }
988
989    @Override
990    public String toString() {
991      StringBuilder sb = new StringBuilder();
992      sb.append("AddBlockOp [path=")
993        .append(path)
994        .append(", penultimateBlock=")
995        .append(penultimateBlock == null ? "NULL" : penultimateBlock)
996        .append(", lastBlock=")
997        .append(lastBlock);
998      appendRpcIdsToString(sb, rpcClientId, rpcCallId);
999      sb.append("]");
1000      return sb.toString();
1001    }
1002    
1003    @Override
1004    protected void toXml(ContentHandler contentHandler) throws SAXException {
1005      XMLUtils.addSaxString(contentHandler, "PATH", path);
1006      if (penultimateBlock != null) {
1007        FSEditLogOp.blockToXml(contentHandler, penultimateBlock);
1008      }
1009      FSEditLogOp.blockToXml(contentHandler, lastBlock);
1010      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
1011    }
1012    
1013    @Override 
1014    void fromXml(Stanza st) throws InvalidXmlException {
1015      this.path = st.getValue("PATH");
1016      List<Stanza> blocks = st.getChildren("BLOCK");
1017      int size = blocks.size();
1018      Preconditions.checkState(size == 1 || size == 2);
1019      this.penultimateBlock = size == 2 ? 
1020          FSEditLogOp.blockFromXml(blocks.get(0)) : null;
1021      this.lastBlock = FSEditLogOp.blockFromXml(blocks.get(size - 1));
1022      readRpcIdsFromXml(st);
1023    }
1024  }
1025  
1026  /**
1027   * {@literal @AtMostOnce} for {@link ClientProtocol#updatePipeline}, but 
1028   * {@literal @Idempotent} for some other ops.
1029   */
1030  static class UpdateBlocksOp extends FSEditLogOp implements BlockListUpdatingOp {
1031    String path;
1032    Block[] blocks;
1033    
1034    private UpdateBlocksOp() {
1035      super(OP_UPDATE_BLOCKS);
1036    }
1037    
1038    static UpdateBlocksOp getInstance(OpInstanceCache cache) {
1039      return (UpdateBlocksOp)cache.get(OP_UPDATE_BLOCKS);
1040    }
1041
1042    @Override
1043    void resetSubFields() {
1044      path = null;
1045      blocks = null;
1046    }
1047    
1048    UpdateBlocksOp setPath(String path) {
1049      this.path = path;
1050      return this;
1051    }
1052    
1053    @Override
1054    public String getPath() {
1055      return path;
1056    }
1057
1058    UpdateBlocksOp setBlocks(Block[] blocks) {
1059      this.blocks = blocks;
1060      return this;
1061    }
1062    
1063    @Override
1064    public Block[] getBlocks() {
1065      return blocks;
1066    }
1067
1068    @Override
1069    public
1070    void writeFields(DataOutputStream out) throws IOException {
1071      FSImageSerialization.writeString(path, out);
1072      FSImageSerialization.writeCompactBlockArray(blocks, out);
1073      // clientId and callId
1074      writeRpcIds(rpcClientId, rpcCallId, out);
1075    }
1076    
1077    @Override
1078    void readFields(DataInputStream in, int logVersion) throws IOException {
1079      path = FSImageSerialization.readString(in);
1080      this.blocks = FSImageSerialization.readCompactBlockArray(
1081          in, logVersion);
1082      readRpcIds(in, logVersion);
1083    }
1084
1085    @Override
1086    public boolean shouldCompleteLastBlock() {
1087      return false;
1088    }
1089
1090    @Override
1091    public String toString() {
1092      StringBuilder sb = new StringBuilder();
1093      sb.append("UpdateBlocksOp [path=")
1094        .append(path)
1095        .append(", blocks=")
1096        .append(Arrays.toString(blocks));
1097      appendRpcIdsToString(sb, rpcClientId, rpcCallId);
1098      sb.append("]");
1099      return sb.toString();
1100    }
1101    
1102    @Override
1103    protected void toXml(ContentHandler contentHandler) throws SAXException {
1104      XMLUtils.addSaxString(contentHandler, "PATH", path);
1105      for (Block b : blocks) {
1106        FSEditLogOp.blockToXml(contentHandler, b);
1107      }
1108      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
1109    }
1110    
1111    @Override void fromXml(Stanza st) throws InvalidXmlException {
1112      this.path = st.getValue("PATH");
1113      List<Stanza> blocks = st.getChildren("BLOCK");
1114      this.blocks = new Block[blocks.size()];
1115      for (int i = 0; i < blocks.size(); i++) {
1116        this.blocks[i] = FSEditLogOp.blockFromXml(blocks.get(i));
1117      }
1118      readRpcIdsFromXml(st);
1119    }
1120  }
1121
1122  /** {@literal @Idempotent} for {@link ClientProtocol#setReplication} */
1123  static class SetReplicationOp extends FSEditLogOp {
1124    String path;
1125    short replication;
1126
1127    private SetReplicationOp() {
1128      super(OP_SET_REPLICATION);
1129    }
1130
1131    static SetReplicationOp getInstance(OpInstanceCache cache) {
1132      return (SetReplicationOp)cache.get(OP_SET_REPLICATION);
1133    }
1134
1135    @Override
1136    void resetSubFields() {
1137      path = null;
1138      replication = 0;
1139    }
1140
1141    SetReplicationOp setPath(String path) {
1142      this.path = path;
1143      return this;
1144    }
1145
1146    SetReplicationOp setReplication(short replication) {
1147      this.replication = replication;
1148      return this;
1149    }
1150
1151    @Override
1152    public 
1153    void writeFields(DataOutputStream out) throws IOException {
1154      FSImageSerialization.writeString(path, out);
1155      FSImageSerialization.writeShort(replication, out);
1156    }
1157    
1158    @Override
1159    void readFields(DataInputStream in, int logVersion)
1160        throws IOException {
1161      this.path = FSImageSerialization.readString(in);
1162      if (NameNodeLayoutVersion.supports(
1163          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
1164        this.replication = FSImageSerialization.readShort(in);
1165      } else {
1166        this.replication = readShort(in);
1167      }
1168    }
1169
1170    @Override
1171    public String toString() {
1172      StringBuilder builder = new StringBuilder();
1173      builder.append("SetReplicationOp [path=");
1174      builder.append(path);
1175      builder.append(", replication=");
1176      builder.append(replication);
1177      builder.append(", opCode=");
1178      builder.append(opCode);
1179      builder.append(", txid=");
1180      builder.append(txid);
1181      builder.append("]");
1182      return builder.toString();
1183    }
1184    
1185    @Override
1186    protected void toXml(ContentHandler contentHandler) throws SAXException {
1187      XMLUtils.addSaxString(contentHandler, "PATH", path);
1188      XMLUtils.addSaxString(contentHandler, "REPLICATION",
1189          Short.valueOf(replication).toString());
1190    }
1191    
1192    @Override void fromXml(Stanza st) throws InvalidXmlException {
1193      this.path = st.getValue("PATH");
1194      this.replication = Short.valueOf(st.getValue("REPLICATION"));
1195    }
1196  }
1197
1198  /** {@literal @AtMostOnce} for {@link ClientProtocol#concat} */
1199  static class ConcatDeleteOp extends FSEditLogOp {
1200    int length;
1201    String trg;
1202    String[] srcs;
1203    long timestamp;
1204    final static public int MAX_CONCAT_SRC = 1024 * 1024;
1205
1206    private ConcatDeleteOp() {
1207      super(OP_CONCAT_DELETE);
1208    }
1209
1210    static ConcatDeleteOp getInstance(OpInstanceCache cache) {
1211      return (ConcatDeleteOp)cache.get(OP_CONCAT_DELETE);
1212    }
1213
1214    @Override
1215    void resetSubFields() {
1216      length = 0;
1217      trg = null;
1218      srcs = null;
1219      timestamp = 0L;
1220    }
1221
1222    ConcatDeleteOp setTarget(String trg) {
1223      this.trg = trg;
1224      return this;
1225    }
1226
1227    ConcatDeleteOp setSources(String[] srcs) {
1228      if (srcs.length > MAX_CONCAT_SRC) {
1229        throw new RuntimeException("ConcatDeleteOp can only have " +
1230            MAX_CONCAT_SRC + " sources at most.");
1231      }
1232      this.srcs = srcs;
1233
1234      return this;
1235    }
1236
1237    ConcatDeleteOp setTimestamp(long timestamp) {
1238      this.timestamp = timestamp;
1239      return this;
1240    }
1241
1242    @Override
1243    public void writeFields(DataOutputStream out) throws IOException {
1244      FSImageSerialization.writeString(trg, out);
1245            
1246      DeprecatedUTF8 info[] = new DeprecatedUTF8[srcs.length];
1247      int idx = 0;
1248      for(int i=0; i<srcs.length; i++) {
1249        info[idx++] = new DeprecatedUTF8(srcs[i]);
1250      }
1251      new ArrayWritable(DeprecatedUTF8.class, info).write(out);
1252
1253      FSImageSerialization.writeLong(timestamp, out);
1254      
1255      // rpc ids
1256      writeRpcIds(rpcClientId, rpcCallId, out);
1257    }
1258
1259    @Override
1260    void readFields(DataInputStream in, int logVersion)
1261        throws IOException {
1262      if (!NameNodeLayoutVersion.supports(
1263          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
1264        this.length = in.readInt();
1265        if (length < 3) { // trg, srcs.., timestamp
1266          throw new IOException("Incorrect data format " +
1267              "for ConcatDeleteOp.");
1268        }
1269      }
1270      this.trg = FSImageSerialization.readString(in);
1271      int srcSize = 0;
1272      if (NameNodeLayoutVersion.supports(
1273          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
1274        srcSize = in.readInt();
1275      } else {
1276        srcSize = this.length - 1 - 1; // trg and timestamp
1277      }
1278      if (srcSize < 0) {
1279          throw new IOException("Incorrect data format. "
1280              + "ConcatDeleteOp cannot have a negative number of data " +
1281              " sources.");
1282      } else if (srcSize > MAX_CONCAT_SRC) {
1283          throw new IOException("Incorrect data format. "
1284              + "ConcatDeleteOp can have at most " + MAX_CONCAT_SRC +
1285              " sources, but we tried to have " + (length - 3) + " sources.");
1286      }
1287      this.srcs = new String [srcSize];
1288      for(int i=0; i<srcSize;i++) {
1289        srcs[i]= FSImageSerialization.readString(in);
1290      }
1291      
1292      if (NameNodeLayoutVersion.supports(
1293          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
1294        this.timestamp = FSImageSerialization.readLong(in);
1295      } else {
1296        this.timestamp = readLong(in);
1297      }
1298      // read RPC ids if necessary
1299      readRpcIds(in, logVersion);
1300    }
1301
1302    @Override
1303    public String toString() {
1304      StringBuilder builder = new StringBuilder();
1305      builder.append("ConcatDeleteOp [length=");
1306      builder.append(length);
1307      builder.append(", trg=");
1308      builder.append(trg);
1309      builder.append(", srcs=");
1310      builder.append(Arrays.toString(srcs));
1311      builder.append(", timestamp=");
1312      builder.append(timestamp);
1313      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
1314      builder.append(", opCode=");
1315      builder.append(opCode);
1316      builder.append(", txid=");
1317      builder.append(txid);
1318      builder.append("]");
1319      return builder.toString();
1320    }
1321    
1322    @Override
1323    protected void toXml(ContentHandler contentHandler) throws SAXException {
1324      XMLUtils.addSaxString(contentHandler, "LENGTH",
1325          Integer.toString(length));
1326      XMLUtils.addSaxString(contentHandler, "TRG", trg);
1327      XMLUtils.addSaxString(contentHandler, "TIMESTAMP",
1328          Long.toString(timestamp));
1329      contentHandler.startElement("", "", "SOURCES", new AttributesImpl());
1330      for (int i = 0; i < srcs.length; ++i) {
1331        XMLUtils.addSaxString(contentHandler,
1332            "SOURCE" + (i + 1), srcs[i]);
1333      }
1334      contentHandler.endElement("", "", "SOURCES");
1335      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
1336    }
1337    
1338    @Override void fromXml(Stanza st) throws InvalidXmlException {
1339      this.length = Integer.parseInt(st.getValue("LENGTH"));
1340      this.trg = st.getValue("TRG");
1341      this.timestamp = Long.parseLong(st.getValue("TIMESTAMP"));
1342      List<Stanza> sources = st.getChildren("SOURCES");
1343      int i = 0;
1344      while (true) {
1345        if (!sources.get(0).hasChildren("SOURCE" + (i + 1)))
1346          break;
1347        i++;
1348      }
1349      srcs = new String[i];
1350      for (i = 0; i < srcs.length; i++) {
1351        srcs[i] = sources.get(0).getValue("SOURCE" + (i + 1));
1352      }
1353      readRpcIdsFromXml(st);
1354    }
1355  }
1356
1357  /** {@literal @AtMostOnce} for {@link ClientProtocol#rename} */
1358  static class RenameOldOp extends FSEditLogOp {
1359    int length;
1360    String src;
1361    String dst;
1362    long timestamp;
1363
1364    private RenameOldOp() {
1365      super(OP_RENAME_OLD);
1366    }
1367
1368    static RenameOldOp getInstance(OpInstanceCache cache) {
1369      return (RenameOldOp)cache.get(OP_RENAME_OLD);
1370    }
1371
1372    @Override
1373    void resetSubFields() {
1374      length = 0;
1375      src = null;
1376      dst = null;
1377      timestamp = 0L;
1378    }
1379
1380    RenameOldOp setSource(String src) {
1381      this.src = src;
1382      return this;
1383    }
1384
1385    RenameOldOp setDestination(String dst) {
1386      this.dst = dst;
1387      return this;
1388    }
1389
1390    RenameOldOp setTimestamp(long timestamp) {
1391      this.timestamp = timestamp;
1392      return this;
1393    }
1394
1395    @Override
1396    public 
1397    void writeFields(DataOutputStream out) throws IOException {
1398      FSImageSerialization.writeString(src, out);
1399      FSImageSerialization.writeString(dst, out);
1400      FSImageSerialization.writeLong(timestamp, out);
1401      writeRpcIds(rpcClientId, rpcCallId, out);
1402    }
1403
1404    @Override
1405    void readFields(DataInputStream in, int logVersion)
1406        throws IOException {
1407      if (!NameNodeLayoutVersion.supports(
1408          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
1409        this.length = in.readInt();
1410        if (this.length != 3) {
1411          throw new IOException("Incorrect data format. "
1412              + "Old rename operation.");
1413        }
1414      }
1415      this.src = FSImageSerialization.readString(in);
1416      this.dst = FSImageSerialization.readString(in);
1417      if (NameNodeLayoutVersion.supports(
1418          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
1419        this.timestamp = FSImageSerialization.readLong(in);
1420      } else {
1421        this.timestamp = readLong(in);
1422      }
1423      
1424      // read RPC ids if necessary
1425      readRpcIds(in, logVersion);
1426    }
1427
1428    @Override
1429    public String toString() {
1430      StringBuilder builder = new StringBuilder();
1431      builder.append("RenameOldOp [length=");
1432      builder.append(length);
1433      builder.append(", src=");
1434      builder.append(src);
1435      builder.append(", dst=");
1436      builder.append(dst);
1437      builder.append(", timestamp=");
1438      builder.append(timestamp);
1439      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
1440      builder.append(", opCode=");
1441      builder.append(opCode);
1442      builder.append(", txid=");
1443      builder.append(txid);
1444      builder.append("]");
1445      return builder.toString();
1446    }
1447    
1448    @Override
1449    protected void toXml(ContentHandler contentHandler) throws SAXException {
1450      XMLUtils.addSaxString(contentHandler, "LENGTH",
1451          Integer.toString(length));
1452      XMLUtils.addSaxString(contentHandler, "SRC", src);
1453      XMLUtils.addSaxString(contentHandler, "DST", dst);
1454      XMLUtils.addSaxString(contentHandler, "TIMESTAMP",
1455          Long.toString(timestamp));
1456      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
1457    }
1458    
1459    @Override 
1460    void fromXml(Stanza st) throws InvalidXmlException {
1461      this.length = Integer.parseInt(st.getValue("LENGTH"));
1462      this.src = st.getValue("SRC");
1463      this.dst = st.getValue("DST");
1464      this.timestamp = Long.parseLong(st.getValue("TIMESTAMP"));
1465      
1466      readRpcIdsFromXml(st);
1467    }
1468  }
1469
1470  /** {@literal @AtMostOnce} for {@link ClientProtocol#delete} */
1471  static class DeleteOp extends FSEditLogOp {
1472    int length;
1473    String path;
1474    long timestamp;
1475
1476    private DeleteOp() {
1477      super(OP_DELETE);
1478    }
1479
1480    static DeleteOp getInstance(OpInstanceCache cache) {
1481      return (DeleteOp)cache.get(OP_DELETE);
1482    }
1483
1484    @Override
1485    void resetSubFields() {
1486      length = 0;
1487      path = null;
1488      timestamp = 0L;
1489    }
1490
1491    DeleteOp setPath(String path) {
1492      this.path = path;
1493      return this;
1494    }
1495
1496    DeleteOp setTimestamp(long timestamp) {
1497      this.timestamp = timestamp;
1498      return this;
1499    }
1500
1501    @Override
1502    public 
1503    void writeFields(DataOutputStream out) throws IOException {
1504      FSImageSerialization.writeString(path, out);
1505      FSImageSerialization.writeLong(timestamp, out);
1506      writeRpcIds(rpcClientId, rpcCallId, out);
1507    }
1508
1509    @Override
1510    void readFields(DataInputStream in, int logVersion)
1511        throws IOException {
1512      if (!NameNodeLayoutVersion.supports(
1513          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
1514        this.length = in.readInt();
1515        if (this.length != 2) {
1516          throw new IOException("Incorrect data format. " + "delete operation.");
1517        }
1518      }
1519      this.path = FSImageSerialization.readString(in);
1520      if (NameNodeLayoutVersion.supports(
1521          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
1522        this.timestamp = FSImageSerialization.readLong(in);
1523      } else {
1524        this.timestamp = readLong(in);
1525      }
1526      // read RPC ids if necessary
1527      readRpcIds(in, logVersion);
1528    }
1529
1530    @Override
1531    public String toString() {
1532      StringBuilder builder = new StringBuilder();
1533      builder.append("DeleteOp [length=");
1534      builder.append(length);
1535      builder.append(", path=");
1536      builder.append(path);
1537      builder.append(", timestamp=");
1538      builder.append(timestamp);
1539      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
1540      builder.append(", opCode=");
1541      builder.append(opCode);
1542      builder.append(", txid=");
1543      builder.append(txid);
1544      builder.append("]");
1545      return builder.toString();
1546    }
1547    
1548    @Override
1549    protected void toXml(ContentHandler contentHandler) throws SAXException {
1550      XMLUtils.addSaxString(contentHandler, "LENGTH",
1551          Integer.toString(length));
1552      XMLUtils.addSaxString(contentHandler, "PATH", path);
1553      XMLUtils.addSaxString(contentHandler, "TIMESTAMP",
1554          Long.toString(timestamp));
1555      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
1556    }
1557    
1558    @Override void fromXml(Stanza st) throws InvalidXmlException {
1559      this.length = Integer.parseInt(st.getValue("LENGTH"));
1560      this.path = st.getValue("PATH");
1561      this.timestamp = Long.parseLong(st.getValue("TIMESTAMP"));
1562      
1563      readRpcIdsFromXml(st);
1564    }
1565  }
1566
1567  /** {@literal @Idempotent} for {@link ClientProtocol#mkdirs} */
1568  static class MkdirOp extends FSEditLogOp {
1569    int length;
1570    long inodeId;
1571    String path;
1572    long timestamp;
1573    PermissionStatus permissions;
1574    List<AclEntry> aclEntries;
1575    List<XAttr> xAttrs;
1576
1577    private MkdirOp() {
1578      super(OP_MKDIR);
1579    }
1580    
1581    static MkdirOp getInstance(OpInstanceCache cache) {
1582      return (MkdirOp)cache.get(OP_MKDIR);
1583    }
1584
1585    @Override
1586    void resetSubFields() {
1587      length = 0;
1588      inodeId = 0L;
1589      path = null;
1590      timestamp = 0L;
1591      permissions = null;
1592      aclEntries = null;
1593      xAttrs = null;
1594    }
1595
1596    MkdirOp setInodeId(long inodeId) {
1597      this.inodeId = inodeId;
1598      return this;
1599    }
1600    
1601    MkdirOp setPath(String path) {
1602      this.path = path;
1603      return this;
1604    }
1605
1606    MkdirOp setTimestamp(long timestamp) {
1607      this.timestamp = timestamp;
1608      return this;
1609    }
1610
1611    MkdirOp setPermissionStatus(PermissionStatus permissions) {
1612      this.permissions = permissions;
1613      return this;
1614    }
1615
1616    MkdirOp setAclEntries(List<AclEntry> aclEntries) {
1617      this.aclEntries = aclEntries;
1618      return this;
1619    }
1620
1621    MkdirOp setXAttrs(List<XAttr> xAttrs) {
1622      this.xAttrs = xAttrs;
1623      return this;
1624    }
1625
1626    @Override
1627    public 
1628    void writeFields(DataOutputStream out) throws IOException {
1629      FSImageSerialization.writeLong(inodeId, out);
1630      FSImageSerialization.writeString(path, out);
1631      FSImageSerialization.writeLong(timestamp, out); // mtime
1632      FSImageSerialization.writeLong(timestamp, out); // atime, unused at this
1633      permissions.write(out);
1634      AclEditLogUtil.write(aclEntries, out);
1635      XAttrEditLogProto.Builder b = XAttrEditLogProto.newBuilder();
1636      b.addAllXAttrs(PBHelper.convertXAttrProto(xAttrs));
1637      b.build().writeDelimitedTo(out);
1638    }
1639    
1640    @Override
1641    void readFields(DataInputStream in, int logVersion) throws IOException {
1642      if (!NameNodeLayoutVersion.supports(
1643          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
1644        this.length = in.readInt();
1645      }
1646      if (-17 < logVersion && length != 2 ||
1647          logVersion <= -17 && length != 3
1648          && !NameNodeLayoutVersion.supports(
1649              LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
1650        throw new IOException("Incorrect data format. Mkdir operation.");
1651      }
1652      if (NameNodeLayoutVersion.supports(
1653          LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
1654        this.inodeId = FSImageSerialization.readLong(in);
1655      } else {
1656        // This id should be updated when this editLogOp is applied
1657        this.inodeId = INodeId.GRANDFATHER_INODE_ID;
1658      }
1659      this.path = FSImageSerialization.readString(in);
1660      if (NameNodeLayoutVersion.supports(
1661          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
1662        this.timestamp = FSImageSerialization.readLong(in);
1663      } else {
1664        this.timestamp = readLong(in);
1665      }
1666
1667      // The disk format stores atimes for directories as well.
1668      // However, currently this is not being updated/used because of
1669      // performance reasons.
1670      if (NameNodeLayoutVersion.supports(
1671          LayoutVersion.Feature.FILE_ACCESS_TIME, logVersion)) {
1672        if (NameNodeLayoutVersion.supports(
1673            LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
1674          FSImageSerialization.readLong(in);
1675        } else {
1676          readLong(in);
1677        }
1678      }
1679
1680      this.permissions = PermissionStatus.read(in);
1681      aclEntries = AclEditLogUtil.read(in, logVersion);
1682
1683      xAttrs = readXAttrsFromEditLog(in, logVersion);
1684    }
1685
1686    @Override
1687    public String toString() {
1688      StringBuilder builder = new StringBuilder();
1689      builder.append("MkdirOp [length=");
1690      builder.append(length);
1691      builder.append(", inodeId=");
1692      builder.append(inodeId);
1693      builder.append(", path=");
1694      builder.append(path);
1695      builder.append(", timestamp=");
1696      builder.append(timestamp);
1697      builder.append(", permissions=");
1698      builder.append(permissions);
1699      builder.append(", aclEntries=");
1700      builder.append(aclEntries);
1701      builder.append(", opCode=");
1702      builder.append(opCode);
1703      builder.append(", txid=");
1704      builder.append(txid);
1705      builder.append(", xAttrs=");
1706      builder.append(xAttrs);
1707      builder.append("]");
1708      return builder.toString();
1709    }
1710
1711    @Override
1712    protected void toXml(ContentHandler contentHandler) throws SAXException {
1713      XMLUtils.addSaxString(contentHandler, "LENGTH",
1714          Integer.toString(length));
1715      XMLUtils.addSaxString(contentHandler, "INODEID",
1716          Long.toString(inodeId));
1717      XMLUtils.addSaxString(contentHandler, "PATH", path);
1718      XMLUtils.addSaxString(contentHandler, "TIMESTAMP",
1719          Long.toString(timestamp));
1720      FSEditLogOp.permissionStatusToXml(contentHandler, permissions);
1721      if (aclEntries != null) {
1722        appendAclEntriesToXml(contentHandler, aclEntries);
1723      }
1724      if (xAttrs != null) {
1725        appendXAttrsToXml(contentHandler, xAttrs);
1726      }
1727    }
1728    
1729    @Override void fromXml(Stanza st) throws InvalidXmlException {
1730      this.length = Integer.parseInt(st.getValue("LENGTH"));
1731      this.inodeId = Long.parseLong(st.getValue("INODEID"));
1732      this.path = st.getValue("PATH");
1733      this.timestamp = Long.parseLong(st.getValue("TIMESTAMP"));
1734      this.permissions = permissionStatusFromXml(st);
1735      aclEntries = readAclEntriesFromXml(st);
1736      xAttrs = readXAttrsFromXml(st);
1737    }
1738  }
1739
1740  /**
1741   * The corresponding operations are either {@literal @Idempotent} (
1742   * {@link ClientProtocol#updateBlockForPipeline},
1743   * {@link ClientProtocol#recoverLease}, {@link ClientProtocol#addBlock}) or
1744   * already bound with other editlog op which records rpc ids (
1745   * {@link ClientProtocol#create}). Thus no need to record rpc ids here.
1746   */
1747  static class SetGenstampV1Op extends FSEditLogOp {
1748    long genStampV1;
1749
1750    private SetGenstampV1Op() {
1751      super(OP_SET_GENSTAMP_V1);
1752    }
1753
1754    static SetGenstampV1Op getInstance(OpInstanceCache cache) {
1755      return (SetGenstampV1Op)cache.get(OP_SET_GENSTAMP_V1);
1756    }
1757
1758    @Override
1759    void resetSubFields() {
1760      genStampV1 = 0L;
1761    }
1762
1763    SetGenstampV1Op setGenerationStamp(long genStamp) {
1764      this.genStampV1 = genStamp;
1765      return this;
1766    }
1767
1768    @Override
1769    public
1770    void writeFields(DataOutputStream out) throws IOException {
1771      FSImageSerialization.writeLong(genStampV1, out);
1772    }
1773
1774    @Override
1775    void readFields(DataInputStream in, int logVersion)
1776        throws IOException {
1777      this.genStampV1 = FSImageSerialization.readLong(in);
1778    }
1779
1780    @Override
1781    public String toString() {
1782      StringBuilder builder = new StringBuilder();
1783      builder.append("SetGenstampOp [GenStamp=");
1784      builder.append(genStampV1);
1785      builder.append(", opCode=");
1786      builder.append(opCode);
1787      builder.append(", txid=");
1788      builder.append(txid);
1789      builder.append("]");
1790      return builder.toString();
1791    }
1792
1793    @Override
1794    protected void toXml(ContentHandler contentHandler) throws SAXException {
1795      XMLUtils.addSaxString(contentHandler, "GENSTAMP",
1796                            Long.toString(genStampV1));
1797    }
1798
1799    @Override void fromXml(Stanza st) throws InvalidXmlException {
1800      this.genStampV1 = Long.parseLong(st.getValue("GENSTAMP"));
1801    }
1802  }
1803
1804  /** Similar with {@link SetGenstampV1Op} */
1805  static class SetGenstampV2Op extends FSEditLogOp {
1806    long genStampV2;
1807
1808    private SetGenstampV2Op() {
1809      super(OP_SET_GENSTAMP_V2);
1810    }
1811
1812    static SetGenstampV2Op getInstance(OpInstanceCache cache) {
1813      return (SetGenstampV2Op)cache.get(OP_SET_GENSTAMP_V2);
1814    }
1815
1816    @Override
1817    void resetSubFields() {
1818      genStampV2 = 0L;
1819    }
1820
1821    SetGenstampV2Op setGenerationStamp(long genStamp) {
1822      this.genStampV2 = genStamp;
1823      return this;
1824    }
1825
1826    @Override
1827    public
1828    void writeFields(DataOutputStream out) throws IOException {
1829      FSImageSerialization.writeLong(genStampV2, out);
1830    }
1831
1832    @Override
1833    void readFields(DataInputStream in, int logVersion)
1834        throws IOException {
1835      this.genStampV2 = FSImageSerialization.readLong(in);
1836    }
1837
1838    @Override
1839    public String toString() {
1840      StringBuilder builder = new StringBuilder();
1841      builder.append("SetGenstampV2Op [GenStampV2=");
1842      builder.append(genStampV2);
1843      builder.append(", opCode=");
1844      builder.append(opCode);
1845      builder.append(", txid=");
1846      builder.append(txid);
1847      builder.append("]");
1848      return builder.toString();
1849    }
1850
1851    @Override
1852    protected void toXml(ContentHandler contentHandler) throws SAXException {
1853      XMLUtils.addSaxString(contentHandler, "GENSTAMPV2",
1854                            Long.toString(genStampV2));
1855    }
1856
1857    @Override void fromXml(Stanza st) throws InvalidXmlException {
1858      this.genStampV2 = Long.parseLong(st.getValue("GENSTAMPV2"));
1859    }
1860  }
1861
1862  /** {@literal @Idempotent} for {@link ClientProtocol#addBlock} */
1863  static class AllocateBlockIdOp extends FSEditLogOp {
1864    long blockId;
1865
1866    private AllocateBlockIdOp() {
1867      super(OP_ALLOCATE_BLOCK_ID);
1868    }
1869
1870    static AllocateBlockIdOp getInstance(OpInstanceCache cache) {
1871      return (AllocateBlockIdOp)cache.get(OP_ALLOCATE_BLOCK_ID);
1872    }
1873
1874    @Override
1875    void resetSubFields() {
1876      blockId = 0L;
1877    }
1878
1879    AllocateBlockIdOp setBlockId(long blockId) {
1880      this.blockId = blockId;
1881      return this;
1882    }
1883
1884    @Override
1885    public
1886    void writeFields(DataOutputStream out) throws IOException {
1887      FSImageSerialization.writeLong(blockId, out);
1888    }
1889
1890    @Override
1891    void readFields(DataInputStream in, int logVersion)
1892        throws IOException {
1893      this.blockId = FSImageSerialization.readLong(in);
1894    }
1895
1896    @Override
1897    public String toString() {
1898      StringBuilder builder = new StringBuilder();
1899      builder.append("AllocateBlockIdOp [blockId=");
1900      builder.append(blockId);
1901      builder.append(", opCode=");
1902      builder.append(opCode);
1903      builder.append(", txid=");
1904      builder.append(txid);
1905      builder.append("]");
1906      return builder.toString();
1907    }
1908
1909    @Override
1910    protected void toXml(ContentHandler contentHandler) throws SAXException {
1911      XMLUtils.addSaxString(contentHandler, "BLOCK_ID",
1912                            Long.toString(blockId));
1913    }
1914
1915    @Override void fromXml(Stanza st) throws InvalidXmlException {
1916      this.blockId = Long.parseLong(st.getValue("BLOCK_ID"));
1917    }
1918  }
1919
1920  /** {@literal @Idempotent} for {@link ClientProtocol#setPermission} */
1921  static class SetPermissionsOp extends FSEditLogOp {
1922    String src;
1923    FsPermission permissions;
1924
1925    private SetPermissionsOp() {
1926      super(OP_SET_PERMISSIONS);
1927    }
1928
1929    static SetPermissionsOp getInstance(OpInstanceCache cache) {
1930      return (SetPermissionsOp)cache.get(OP_SET_PERMISSIONS);
1931    }
1932
1933    @Override
1934    void resetSubFields() {
1935      src = null;
1936      permissions = null;
1937    }
1938
1939    SetPermissionsOp setSource(String src) {
1940      this.src = src;
1941      return this;
1942    }
1943
1944    SetPermissionsOp setPermissions(FsPermission permissions) {
1945      this.permissions = permissions;
1946      return this;
1947    }
1948
1949    @Override
1950    public 
1951    void writeFields(DataOutputStream out) throws IOException {
1952      FSImageSerialization.writeString(src, out);
1953      permissions.write(out);
1954     }
1955 
1956    @Override
1957    void readFields(DataInputStream in, int logVersion)
1958        throws IOException {
1959      this.src = FSImageSerialization.readString(in);
1960      this.permissions = FsPermission.read(in);
1961    }
1962
1963    @Override
1964    public String toString() {
1965      StringBuilder builder = new StringBuilder();
1966      builder.append("SetPermissionsOp [src=");
1967      builder.append(src);
1968      builder.append(", permissions=");
1969      builder.append(permissions);
1970      builder.append(", opCode=");
1971      builder.append(opCode);
1972      builder.append(", txid=");
1973      builder.append(txid);
1974      builder.append("]");
1975      return builder.toString();
1976    }
1977    
1978    @Override
1979    protected void toXml(ContentHandler contentHandler) throws SAXException {
1980      XMLUtils.addSaxString(contentHandler, "SRC", src);
1981      XMLUtils.addSaxString(contentHandler, "MODE",
1982          Short.valueOf(permissions.toShort()).toString());
1983    }
1984    
1985    @Override void fromXml(Stanza st) throws InvalidXmlException {
1986      this.src = st.getValue("SRC");
1987      this.permissions = new FsPermission(
1988          Short.valueOf(st.getValue("MODE")));
1989    }
1990  }
1991
1992  /** {@literal @Idempotent} for {@link ClientProtocol#setOwner} */
1993  static class SetOwnerOp extends FSEditLogOp {
1994    String src;
1995    String username;
1996    String groupname;
1997
1998    private SetOwnerOp() {
1999      super(OP_SET_OWNER);
2000    }
2001
2002    static SetOwnerOp getInstance(OpInstanceCache cache) {
2003      return (SetOwnerOp)cache.get(OP_SET_OWNER);
2004    }
2005
2006    @Override
2007    void resetSubFields() {
2008      src = null;
2009      username = null;
2010      groupname = null;
2011    }
2012
2013    SetOwnerOp setSource(String src) {
2014      this.src = src;
2015      return this;
2016    }
2017
2018    SetOwnerOp setUser(String username) {
2019      this.username = username;
2020      return this;
2021    }
2022
2023    SetOwnerOp setGroup(String groupname) {
2024      this.groupname = groupname;
2025      return this;
2026    }
2027
2028    @Override
2029    public 
2030    void writeFields(DataOutputStream out) throws IOException {
2031      FSImageSerialization.writeString(src, out);
2032      FSImageSerialization.writeString(username == null ? "" : username, out);
2033      FSImageSerialization.writeString(groupname == null ? "" : groupname, out);
2034    }
2035
2036    @Override
2037    void readFields(DataInputStream in, int logVersion)
2038        throws IOException {
2039      this.src = FSImageSerialization.readString(in);
2040      this.username = FSImageSerialization.readString_EmptyAsNull(in);
2041      this.groupname = FSImageSerialization.readString_EmptyAsNull(in);
2042    }
2043
2044    @Override
2045    public String toString() {
2046      StringBuilder builder = new StringBuilder();
2047      builder.append("SetOwnerOp [src=");
2048      builder.append(src);
2049      builder.append(", username=");
2050      builder.append(username);
2051      builder.append(", groupname=");
2052      builder.append(groupname);
2053      builder.append(", opCode=");
2054      builder.append(opCode);
2055      builder.append(", txid=");
2056      builder.append(txid);
2057      builder.append("]");
2058      return builder.toString();
2059    }
2060    
2061    @Override
2062    protected void toXml(ContentHandler contentHandler) throws SAXException {
2063      XMLUtils.addSaxString(contentHandler, "SRC", src);
2064      if (username != null) {
2065        XMLUtils.addSaxString(contentHandler, "USERNAME", username);
2066      }
2067      if (groupname != null) {
2068        XMLUtils.addSaxString(contentHandler, "GROUPNAME", groupname);
2069      }
2070    }
2071    
2072    @Override void fromXml(Stanza st) throws InvalidXmlException {
2073      this.src = st.getValue("SRC");
2074      this.username = (st.hasChildren("USERNAME")) ? 
2075          st.getValue("USERNAME") : null;
2076      this.groupname = (st.hasChildren("GROUPNAME")) ? 
2077          st.getValue("GROUPNAME") : null;
2078    }
2079  }
2080  
2081  static class SetNSQuotaOp extends FSEditLogOp {
2082    String src;
2083    long nsQuota;
2084
2085    private SetNSQuotaOp() {
2086      super(OP_SET_NS_QUOTA);
2087    }
2088
2089    static SetNSQuotaOp getInstance(OpInstanceCache cache) {
2090      return (SetNSQuotaOp)cache.get(OP_SET_NS_QUOTA);
2091    }
2092
2093    @Override
2094    void resetSubFields() {
2095      src = null;
2096      nsQuota = 0L;
2097    }
2098
2099    @Override
2100    public 
2101    void writeFields(DataOutputStream out) throws IOException {
2102      throw new IOException("Deprecated");      
2103    }
2104
2105    @Override
2106    void readFields(DataInputStream in, int logVersion)
2107        throws IOException {
2108      this.src = FSImageSerialization.readString(in);
2109      this.nsQuota = FSImageSerialization.readLong(in);
2110    }
2111
2112    @Override
2113    public String toString() {
2114      StringBuilder builder = new StringBuilder();
2115      builder.append("SetNSQuotaOp [src=");
2116      builder.append(src);
2117      builder.append(", nsQuota=");
2118      builder.append(nsQuota);
2119      builder.append(", opCode=");
2120      builder.append(opCode);
2121      builder.append(", txid=");
2122      builder.append(txid);
2123      builder.append("]");
2124      return builder.toString();
2125    }
2126    
2127    @Override
2128    protected void toXml(ContentHandler contentHandler) throws SAXException {
2129      XMLUtils.addSaxString(contentHandler, "SRC", src);
2130      XMLUtils.addSaxString(contentHandler, "NSQUOTA",
2131          Long.toString(nsQuota));
2132    }
2133    
2134    @Override void fromXml(Stanza st) throws InvalidXmlException {
2135      this.src = st.getValue("SRC");
2136      this.nsQuota = Long.parseLong(st.getValue("NSQUOTA"));
2137    }
2138  }
2139
2140  static class ClearNSQuotaOp extends FSEditLogOp {
2141    String src;
2142
2143    private ClearNSQuotaOp() {
2144      super(OP_CLEAR_NS_QUOTA);
2145    }
2146
2147    static ClearNSQuotaOp getInstance(OpInstanceCache cache) {
2148      return (ClearNSQuotaOp)cache.get(OP_CLEAR_NS_QUOTA);
2149    }
2150
2151    @Override
2152    void resetSubFields() {
2153      src = null;
2154    }
2155
2156    @Override
2157    public 
2158    void writeFields(DataOutputStream out) throws IOException {
2159      throw new IOException("Deprecated");      
2160    }
2161
2162    @Override
2163    void readFields(DataInputStream in, int logVersion)
2164        throws IOException {
2165      this.src = FSImageSerialization.readString(in);
2166    }
2167
2168    @Override
2169    public String toString() {
2170      StringBuilder builder = new StringBuilder();
2171      builder.append("ClearNSQuotaOp [src=");
2172      builder.append(src);
2173      builder.append(", opCode=");
2174      builder.append(opCode);
2175      builder.append(", txid=");
2176      builder.append(txid);
2177      builder.append("]");
2178      return builder.toString();
2179    }
2180    
2181    @Override
2182    protected void toXml(ContentHandler contentHandler) throws SAXException {
2183      XMLUtils.addSaxString(contentHandler, "SRC", src);
2184    }
2185    
2186    @Override void fromXml(Stanza st) throws InvalidXmlException {
2187      this.src = st.getValue("SRC");
2188    }
2189  }
2190
2191  /** {@literal @Idempotent} for {@link ClientProtocol#setQuota} */
2192  static class SetQuotaOp extends FSEditLogOp {
2193    String src;
2194    long nsQuota;
2195    long dsQuota;
2196
2197    private SetQuotaOp() {
2198      super(OP_SET_QUOTA);
2199    }
2200
2201    static SetQuotaOp getInstance(OpInstanceCache cache) {
2202      return (SetQuotaOp)cache.get(OP_SET_QUOTA);
2203    }
2204
2205    @Override
2206    void resetSubFields() {
2207      src = null;
2208      nsQuota = 0L;
2209      dsQuota = 0L;
2210    }
2211
2212    SetQuotaOp setSource(String src) {
2213      this.src = src;
2214      return this;
2215    }
2216
2217    SetQuotaOp setNSQuota(long nsQuota) {
2218      this.nsQuota = nsQuota;
2219      return this;
2220    }
2221
2222    SetQuotaOp setDSQuota(long dsQuota) {
2223      this.dsQuota = dsQuota;
2224      return this;
2225    }
2226
2227    @Override
2228    public 
2229    void writeFields(DataOutputStream out) throws IOException {
2230      FSImageSerialization.writeString(src, out);
2231      FSImageSerialization.writeLong(nsQuota, out);
2232      FSImageSerialization.writeLong(dsQuota, out);
2233    }
2234
2235    @Override
2236    void readFields(DataInputStream in, int logVersion)
2237        throws IOException {
2238      this.src = FSImageSerialization.readString(in);
2239      this.nsQuota = FSImageSerialization.readLong(in);
2240      this.dsQuota = FSImageSerialization.readLong(in);
2241    }
2242
2243    @Override
2244    public String toString() {
2245      StringBuilder builder = new StringBuilder();
2246      builder.append("SetQuotaOp [src=");
2247      builder.append(src);
2248      builder.append(", nsQuota=");
2249      builder.append(nsQuota);
2250      builder.append(", dsQuota=");
2251      builder.append(dsQuota);
2252      builder.append(", opCode=");
2253      builder.append(opCode);
2254      builder.append(", txid=");
2255      builder.append(txid);
2256      builder.append("]");
2257      return builder.toString();
2258    }
2259    
2260    @Override
2261    protected void toXml(ContentHandler contentHandler) throws SAXException {
2262      XMLUtils.addSaxString(contentHandler, "SRC", src);
2263      XMLUtils.addSaxString(contentHandler, "NSQUOTA",
2264          Long.toString(nsQuota));
2265      XMLUtils.addSaxString(contentHandler, "DSQUOTA",
2266          Long.toString(dsQuota));
2267    }
2268    
2269    @Override void fromXml(Stanza st) throws InvalidXmlException {
2270      this.src = st.getValue("SRC");
2271      this.nsQuota = Long.parseLong(st.getValue("NSQUOTA"));
2272      this.dsQuota = Long.parseLong(st.getValue("DSQUOTA"));
2273    }
2274  }
2275
2276  /** {@literal @Idempotent} for {@link ClientProtocol#setQuota} */
2277  static class SetQuotaByStorageTypeOp extends FSEditLogOp {
2278    String src;
2279    long dsQuota;
2280    StorageType type;
2281
2282    private SetQuotaByStorageTypeOp() {
2283      super(OP_SET_QUOTA_BY_STORAGETYPE);
2284    }
2285
2286    static SetQuotaByStorageTypeOp getInstance(OpInstanceCache cache) {
2287      return (SetQuotaByStorageTypeOp)cache.get(OP_SET_QUOTA_BY_STORAGETYPE);
2288    }
2289
2290    @Override
2291    void resetSubFields() {
2292      src = null;
2293      dsQuota = -1L;
2294      type = StorageType.DEFAULT;
2295    }
2296
2297    SetQuotaByStorageTypeOp setSource(String src) {
2298      this.src = src;
2299      return this;
2300    }
2301
2302    SetQuotaByStorageTypeOp setQuotaByStorageType(long dsQuota, StorageType type) {
2303      this.type = type;
2304      this.dsQuota = dsQuota;
2305      return this;
2306    }
2307
2308    @Override
2309    public
2310    void writeFields(DataOutputStream out) throws IOException {
2311      FSImageSerialization.writeString(src, out);
2312      FSImageSerialization.writeInt(type.ordinal(), out);
2313      FSImageSerialization.writeLong(dsQuota, out);
2314    }
2315
2316    @Override
2317    void readFields(DataInputStream in, int logVersion)
2318      throws IOException {
2319      this.src = FSImageSerialization.readString(in);
2320      this.type = StorageType.parseStorageType(FSImageSerialization.readInt(in));
2321      this.dsQuota = FSImageSerialization.readLong(in);
2322    }
2323
2324    @Override
2325    public String toString() {
2326      StringBuilder builder = new StringBuilder();
2327      builder.append("SetTypeQuotaOp [src=");
2328      builder.append(src);
2329      builder.append(", storageType=");
2330      builder.append(type);
2331      builder.append(", dsQuota=");
2332      builder.append(dsQuota);
2333      builder.append(", opCode=");
2334      builder.append(opCode);
2335      builder.append(", txid=");
2336      builder.append(txid);
2337      builder.append("]");
2338      return builder.toString();
2339    }
2340
2341    @Override
2342    protected void toXml(ContentHandler contentHandler) throws SAXException {
2343      XMLUtils.addSaxString(contentHandler, "SRC", src);
2344      XMLUtils.addSaxString(contentHandler, "STORAGETYPE",
2345        Integer.toString(type.ordinal()));
2346      XMLUtils.addSaxString(contentHandler, "DSQUOTA",
2347        Long.toString(dsQuota));
2348    }
2349
2350    @Override void fromXml(Stanza st) throws InvalidXmlException {
2351      this.src = st.getValue("SRC");
2352      this.type = StorageType.parseStorageType(
2353          Integer.parseInt(st.getValue("STORAGETYPE")));
2354      this.dsQuota = Long.parseLong(st.getValue("DSQUOTA"));
2355    }
2356  }
2357
2358  /** {@literal @Idempotent} for {@link ClientProtocol#setTimes} */
2359  static class TimesOp extends FSEditLogOp {
2360    int length;
2361    String path;
2362    long mtime;
2363    long atime;
2364
2365    private TimesOp() {
2366      super(OP_TIMES);
2367    }
2368
2369    static TimesOp getInstance(OpInstanceCache cache) {
2370      return (TimesOp)cache.get(OP_TIMES);
2371    }
2372
2373    @Override
2374    void resetSubFields() {
2375      length = 0;
2376      path = null;
2377      mtime = 0L;
2378      atime = 0L;
2379    }
2380
2381    TimesOp setPath(String path) {
2382      this.path = path;
2383      return this;
2384    }
2385
2386    TimesOp setModificationTime(long mtime) {
2387      this.mtime = mtime;
2388      return this;
2389    }
2390
2391    TimesOp setAccessTime(long atime) {
2392      this.atime = atime;
2393      return this;
2394    }
2395
2396    @Override
2397    public 
2398    void writeFields(DataOutputStream out) throws IOException {
2399      FSImageSerialization.writeString(path, out);
2400      FSImageSerialization.writeLong(mtime, out);
2401      FSImageSerialization.writeLong(atime, out);
2402    }
2403
2404    @Override
2405    void readFields(DataInputStream in, int logVersion)
2406        throws IOException {
2407      if (!NameNodeLayoutVersion.supports(
2408          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
2409        this.length = in.readInt();
2410        if (length != 3) {
2411          throw new IOException("Incorrect data format. " + "times operation.");
2412        }
2413      }
2414      this.path = FSImageSerialization.readString(in);
2415
2416      if (NameNodeLayoutVersion.supports(
2417          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
2418        this.mtime = FSImageSerialization.readLong(in);
2419        this.atime = FSImageSerialization.readLong(in);
2420      } else {
2421        this.mtime = readLong(in);
2422        this.atime = readLong(in);
2423      }
2424    }
2425
2426    @Override
2427    public String toString() {
2428      StringBuilder builder = new StringBuilder();
2429      builder.append("TimesOp [length=");
2430      builder.append(length);
2431      builder.append(", path=");
2432      builder.append(path);
2433      builder.append(", mtime=");
2434      builder.append(mtime);
2435      builder.append(", atime=");
2436      builder.append(atime);
2437      builder.append(", opCode=");
2438      builder.append(opCode);
2439      builder.append(", txid=");
2440      builder.append(txid);
2441      builder.append("]");
2442      return builder.toString();
2443    }
2444    
2445    @Override
2446    protected void toXml(ContentHandler contentHandler) throws SAXException {
2447      XMLUtils.addSaxString(contentHandler, "LENGTH",
2448          Integer.toString(length));
2449      XMLUtils.addSaxString(contentHandler, "PATH", path);
2450      XMLUtils.addSaxString(contentHandler, "MTIME",
2451          Long.toString(mtime));
2452      XMLUtils.addSaxString(contentHandler, "ATIME",
2453          Long.toString(atime));
2454    }
2455    
2456    @Override void fromXml(Stanza st) throws InvalidXmlException {
2457      this.length = Integer.parseInt(st.getValue("LENGTH"));
2458      this.path = st.getValue("PATH");
2459      this.mtime = Long.parseLong(st.getValue("MTIME"));
2460      this.atime = Long.parseLong(st.getValue("ATIME"));
2461    }
2462  }
2463
2464  /** {@literal @AtMostOnce} for {@link ClientProtocol#createSymlink} */
2465  static class SymlinkOp extends FSEditLogOp {
2466    int length;
2467    long inodeId;
2468    String path;
2469    String value;
2470    long mtime;
2471    long atime;
2472    PermissionStatus permissionStatus;
2473
2474    private SymlinkOp() {
2475      super(OP_SYMLINK);
2476    }
2477
2478    static SymlinkOp getInstance(OpInstanceCache cache) {
2479      return (SymlinkOp)cache.get(OP_SYMLINK);
2480    }
2481
2482    @Override
2483    void resetSubFields() {
2484      length = 0;
2485      inodeId = 0L;
2486      path = null;
2487      value = null;
2488      mtime = 0L;
2489      atime = 0L;
2490      permissionStatus = null;
2491    }
2492
2493    SymlinkOp setId(long inodeId) {
2494      this.inodeId = inodeId;
2495      return this;
2496    }
2497    
2498    SymlinkOp setPath(String path) {
2499      this.path = path;
2500      return this;
2501    }
2502
2503    SymlinkOp setValue(String value) {
2504      this.value = value;
2505      return this;
2506    }
2507
2508    SymlinkOp setModificationTime(long mtime) {
2509      this.mtime = mtime;
2510      return this;
2511    }
2512
2513    SymlinkOp setAccessTime(long atime) {
2514      this.atime = atime;
2515      return this;
2516    }
2517
2518    SymlinkOp setPermissionStatus(PermissionStatus permissionStatus) {
2519      this.permissionStatus = permissionStatus;
2520      return this;
2521    }
2522
2523    @Override
2524    public void writeFields(DataOutputStream out) throws IOException {
2525      FSImageSerialization.writeLong(inodeId, out);      
2526      FSImageSerialization.writeString(path, out);
2527      FSImageSerialization.writeString(value, out);
2528      FSImageSerialization.writeLong(mtime, out);
2529      FSImageSerialization.writeLong(atime, out);
2530      permissionStatus.write(out);
2531      writeRpcIds(rpcClientId, rpcCallId, out);
2532    }
2533
2534    @Override
2535    void readFields(DataInputStream in, int logVersion)
2536        throws IOException {
2537      if (!NameNodeLayoutVersion.supports(
2538          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
2539        this.length = in.readInt();
2540        if (this.length != 4) {
2541          throw new IOException("Incorrect data format. "
2542              + "symlink operation.");
2543        }
2544      }
2545      if (NameNodeLayoutVersion.supports(
2546          LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
2547        this.inodeId = FSImageSerialization.readLong(in);
2548      } else {
2549        // This id should be updated when the editLogOp is applied
2550        this.inodeId = INodeId.GRANDFATHER_INODE_ID;
2551      }
2552      this.path = FSImageSerialization.readString(in);
2553      this.value = FSImageSerialization.readString(in);
2554
2555      if (NameNodeLayoutVersion.supports(
2556          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
2557        this.mtime = FSImageSerialization.readLong(in);
2558        this.atime = FSImageSerialization.readLong(in);
2559      } else {
2560        this.mtime = readLong(in);
2561        this.atime = readLong(in);
2562      }
2563      this.permissionStatus = PermissionStatus.read(in);
2564      
2565      // read RPC ids if necessary
2566      readRpcIds(in, logVersion);
2567    }
2568
2569    @Override
2570    public String toString() {
2571      StringBuilder builder = new StringBuilder();
2572      builder.append("SymlinkOp [length=");
2573      builder.append(length);
2574      builder.append(", inodeId=");
2575      builder.append(inodeId);
2576      builder.append(", path=");
2577      builder.append(path);
2578      builder.append(", value=");
2579      builder.append(value);
2580      builder.append(", mtime=");
2581      builder.append(mtime);
2582      builder.append(", atime=");
2583      builder.append(atime);
2584      builder.append(", permissionStatus=");
2585      builder.append(permissionStatus);
2586      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
2587      builder.append(", opCode=");
2588      builder.append(opCode);
2589      builder.append(", txid=");
2590      builder.append(txid);
2591      builder.append("]");
2592      return builder.toString();
2593    }
2594    
2595    @Override
2596    protected void toXml(ContentHandler contentHandler) throws SAXException {
2597      XMLUtils.addSaxString(contentHandler, "LENGTH",
2598          Integer.toString(length));
2599      XMLUtils.addSaxString(contentHandler, "INODEID",
2600          Long.toString(inodeId));
2601      XMLUtils.addSaxString(contentHandler, "PATH", path);
2602      XMLUtils.addSaxString(contentHandler, "VALUE", value);
2603      XMLUtils.addSaxString(contentHandler, "MTIME",
2604          Long.toString(mtime));
2605      XMLUtils.addSaxString(contentHandler, "ATIME",
2606          Long.toString(atime));
2607      FSEditLogOp.permissionStatusToXml(contentHandler, permissionStatus);
2608      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
2609    }
2610
2611    @Override 
2612    void fromXml(Stanza st) throws InvalidXmlException {
2613      this.length = Integer.parseInt(st.getValue("LENGTH"));
2614      this.inodeId = Long.parseLong(st.getValue("INODEID"));
2615      this.path = st.getValue("PATH");
2616      this.value = st.getValue("VALUE");
2617      this.mtime = Long.parseLong(st.getValue("MTIME"));
2618      this.atime = Long.parseLong(st.getValue("ATIME"));
2619      this.permissionStatus = permissionStatusFromXml(st);
2620      
2621      readRpcIdsFromXml(st);
2622    }
2623  }
2624
2625  /** {@literal @AtMostOnce} for {@link ClientProtocol#rename2} */
2626  static class RenameOp extends FSEditLogOp {
2627    int length;
2628    String src;
2629    String dst;
2630    long timestamp;
2631    Rename[] options;
2632
2633    private RenameOp() {
2634      super(OP_RENAME);
2635    }
2636
2637    static RenameOp getInstance(OpInstanceCache cache) {
2638      return (RenameOp)cache.get(OP_RENAME);
2639    }
2640
2641    @Override
2642    void resetSubFields() {
2643      length = 0;
2644      src = null;
2645      dst = null;
2646      timestamp = 0L;
2647      options = null;
2648    }
2649
2650    RenameOp setSource(String src) {
2651      this.src = src;
2652      return this;
2653    }
2654
2655    RenameOp setDestination(String dst) {
2656      this.dst = dst;
2657      return this;
2658    }
2659    
2660    RenameOp setTimestamp(long timestamp) {
2661      this.timestamp = timestamp;
2662      return this;
2663    }
2664    
2665    RenameOp setOptions(Rename[] options) {
2666      this.options = options;
2667      return this;
2668    }
2669
2670    @Override
2671    public 
2672    void writeFields(DataOutputStream out) throws IOException {
2673      FSImageSerialization.writeString(src, out);
2674      FSImageSerialization.writeString(dst, out);
2675      FSImageSerialization.writeLong(timestamp, out);
2676      toBytesWritable(options).write(out);
2677      writeRpcIds(rpcClientId, rpcCallId, out);
2678    }
2679
2680    @Override
2681    void readFields(DataInputStream in, int logVersion)
2682        throws IOException {
2683      if (!NameNodeLayoutVersion.supports(
2684          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
2685        this.length = in.readInt();
2686        if (this.length != 3) {
2687          throw new IOException("Incorrect data format. " + "Rename operation.");
2688        }
2689      }
2690      this.src = FSImageSerialization.readString(in);
2691      this.dst = FSImageSerialization.readString(in);
2692
2693      if (NameNodeLayoutVersion.supports(
2694          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
2695        this.timestamp = FSImageSerialization.readLong(in);
2696      } else {
2697        this.timestamp = readLong(in);
2698      }
2699      this.options = readRenameOptions(in);
2700      
2701      // read RPC ids if necessary
2702      readRpcIds(in, logVersion);
2703    }
2704
2705    private static Rename[] readRenameOptions(DataInputStream in) throws IOException {
2706      BytesWritable writable = new BytesWritable();
2707      writable.readFields(in);
2708
2709      byte[] bytes = writable.getBytes();
2710      Rename[] options = new Rename[bytes.length];
2711
2712      for (int i = 0; i < bytes.length; i++) {
2713        options[i] = Rename.valueOf(bytes[i]);
2714      }
2715      return options;
2716    }
2717
2718    static BytesWritable toBytesWritable(Rename... options) {
2719      byte[] bytes = new byte[options.length];
2720      for (int i = 0; i < options.length; i++) {
2721        bytes[i] = options[i].value();
2722      }
2723      return new BytesWritable(bytes);
2724    }
2725
2726    @Override
2727    public String toString() {
2728      StringBuilder builder = new StringBuilder();
2729      builder.append("RenameOp [length=");
2730      builder.append(length);
2731      builder.append(", src=");
2732      builder.append(src);
2733      builder.append(", dst=");
2734      builder.append(dst);
2735      builder.append(", timestamp=");
2736      builder.append(timestamp);
2737      builder.append(", options=");
2738      builder.append(Arrays.toString(options));
2739      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
2740      builder.append(", opCode=");
2741      builder.append(opCode);
2742      builder.append(", txid=");
2743      builder.append(txid);
2744      builder.append("]");
2745      return builder.toString();
2746    }
2747    
2748    @Override
2749    protected void toXml(ContentHandler contentHandler) throws SAXException {
2750      XMLUtils.addSaxString(contentHandler, "LENGTH",
2751          Integer.toString(length));
2752      XMLUtils.addSaxString(contentHandler, "SRC", src);
2753      XMLUtils.addSaxString(contentHandler, "DST", dst);
2754      XMLUtils.addSaxString(contentHandler, "TIMESTAMP",
2755          Long.toString(timestamp));
2756      StringBuilder bld = new StringBuilder();
2757      String prefix = "";
2758      for (Rename r : options) {
2759        bld.append(prefix).append(r.toString());
2760        prefix = "|";
2761      }
2762      XMLUtils.addSaxString(contentHandler, "OPTIONS", bld.toString());
2763      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
2764    }
2765    
2766    @Override void fromXml(Stanza st) throws InvalidXmlException {
2767      this.length = Integer.parseInt(st.getValue("LENGTH"));
2768      this.src = st.getValue("SRC");
2769      this.dst = st.getValue("DST");
2770      this.timestamp = Long.parseLong(st.getValue("TIMESTAMP"));
2771      String opts = st.getValue("OPTIONS");
2772      String o[] = opts.split("\\|");
2773      this.options = new Rename[o.length];
2774      for (int i = 0; i < o.length; i++) {
2775        if (o[i].equals(""))
2776          continue;
2777        try {
2778          this.options[i] = Rename.valueOf(o[i]);
2779        } finally {
2780          if (this.options[i] == null) {
2781            System.err.println("error parsing Rename value: \"" + o[i] + "\"");
2782          }
2783        }
2784      }
2785      readRpcIdsFromXml(st);
2786    }
2787  }
2788
2789  static class TruncateOp extends FSEditLogOp {
2790    String src;
2791    String clientName;
2792    String clientMachine;
2793    long newLength;
2794    long timestamp;
2795    Block truncateBlock;
2796
2797    private TruncateOp() {
2798      super(OP_TRUNCATE);
2799    }
2800
2801    static TruncateOp getInstance(OpInstanceCache cache) {
2802      return (TruncateOp)cache.get(OP_TRUNCATE);
2803    }
2804
2805    @Override
2806    void resetSubFields() {
2807      src = null;
2808      clientName = null;
2809      clientMachine = null;
2810      newLength = 0L;
2811      timestamp = 0L;
2812    }
2813
2814    TruncateOp setPath(String src) {
2815      this.src = src;
2816      return this;
2817    }
2818
2819    TruncateOp setClientName(String clientName) {
2820      this.clientName = clientName;
2821      return this;
2822    }
2823
2824    TruncateOp setClientMachine(String clientMachine) {
2825      this.clientMachine = clientMachine;
2826      return this;
2827    }
2828
2829    TruncateOp setNewLength(long newLength) {
2830      this.newLength = newLength;
2831      return this;
2832    }
2833
2834    TruncateOp setTimestamp(long timestamp) {
2835      this.timestamp = timestamp;
2836      return this;
2837    }
2838
2839    TruncateOp setTruncateBlock(Block truncateBlock) {
2840      this.truncateBlock = truncateBlock;
2841      return this;
2842    }
2843
2844    @Override
2845    void readFields(DataInputStream in, int logVersion) throws IOException {
2846      src = FSImageSerialization.readString(in);
2847      clientName = FSImageSerialization.readString(in);
2848      clientMachine = FSImageSerialization.readString(in);
2849      newLength = FSImageSerialization.readLong(in);
2850      timestamp = FSImageSerialization.readLong(in);
2851      Block[] blocks =
2852          FSImageSerialization.readCompactBlockArray(in, logVersion);
2853      assert blocks.length <= 1 : "Truncate op should have 1 or 0 blocks";
2854      truncateBlock = (blocks.length == 0) ? null : blocks[0];
2855    }
2856
2857    @Override
2858    public void writeFields(DataOutputStream out) throws IOException {
2859      FSImageSerialization.writeString(src, out);
2860      FSImageSerialization.writeString(clientName, out);
2861      FSImageSerialization.writeString(clientMachine, out);
2862      FSImageSerialization.writeLong(newLength, out);
2863      FSImageSerialization.writeLong(timestamp, out);
2864      int size = truncateBlock != null ? 1 : 0;
2865      Block[] blocks = new Block[size];
2866      if (truncateBlock != null) {
2867        blocks[0] = truncateBlock;
2868      }
2869      FSImageSerialization.writeCompactBlockArray(blocks, out);
2870    }
2871
2872    @Override
2873    protected void toXml(ContentHandler contentHandler) throws SAXException {
2874      XMLUtils.addSaxString(contentHandler, "SRC", src);
2875      XMLUtils.addSaxString(contentHandler, "CLIENTNAME", clientName);
2876      XMLUtils.addSaxString(contentHandler, "CLIENTMACHINE", clientMachine);
2877      XMLUtils.addSaxString(contentHandler, "NEWLENGTH",
2878          Long.toString(newLength));
2879      XMLUtils.addSaxString(contentHandler, "TIMESTAMP",
2880          Long.toString(timestamp));
2881      if(truncateBlock != null)
2882        FSEditLogOp.blockToXml(contentHandler, truncateBlock);
2883    }
2884
2885    @Override
2886    void fromXml(Stanza st) throws InvalidXmlException {
2887      this.src = st.getValue("SRC");
2888      this.clientName = st.getValue("CLIENTNAME");
2889      this.clientMachine = st.getValue("CLIENTMACHINE");
2890      this.newLength = Long.parseLong(st.getValue("NEWLENGTH"));
2891      this.timestamp = Long.parseLong(st.getValue("TIMESTAMP"));
2892      if (st.hasChildren("BLOCK"))
2893        this.truncateBlock = FSEditLogOp.blockFromXml(st);
2894    }
2895
2896    @Override
2897    public String toString() {
2898      StringBuilder builder = new StringBuilder();
2899      builder.append("TruncateOp [src=");
2900      builder.append(src);
2901      builder.append(", clientName=");
2902      builder.append(clientName);
2903      builder.append(", clientMachine=");
2904      builder.append(clientMachine);
2905      builder.append(", newLength=");
2906      builder.append(newLength);
2907      builder.append(", timestamp=");
2908      builder.append(timestamp);
2909      builder.append(", truncateBlock=");
2910      builder.append(truncateBlock);
2911      builder.append(", opCode=");
2912      builder.append(opCode);
2913      builder.append(", txid=");
2914      builder.append(txid);
2915      builder.append("]");
2916      return builder.toString();
2917    }
2918  }
2919 
2920  /**
2921   * {@literal @Idempotent} for {@link ClientProtocol#recoverLease}. In the
2922   * meanwhile, startFile and appendFile both have their own corresponding
2923   * editlog op.
2924   */
2925  static class ReassignLeaseOp extends FSEditLogOp {
2926    String leaseHolder;
2927    String path;
2928    String newHolder;
2929
2930    private ReassignLeaseOp() {
2931      super(OP_REASSIGN_LEASE);
2932    }
2933
2934    static ReassignLeaseOp getInstance(OpInstanceCache cache) {
2935      return (ReassignLeaseOp)cache.get(OP_REASSIGN_LEASE);
2936    }
2937
2938    @Override
2939    void resetSubFields() {
2940      leaseHolder = null;
2941      path = null;
2942      newHolder = null;
2943    }
2944
2945    ReassignLeaseOp setLeaseHolder(String leaseHolder) {
2946      this.leaseHolder = leaseHolder;
2947      return this;
2948    }
2949
2950    ReassignLeaseOp setPath(String path) {
2951      this.path = path;
2952      return this;
2953    }
2954
2955    ReassignLeaseOp setNewHolder(String newHolder) {
2956      this.newHolder = newHolder;
2957      return this;
2958    }
2959
2960    @Override
2961    public 
2962    void writeFields(DataOutputStream out) throws IOException {
2963      FSImageSerialization.writeString(leaseHolder, out);
2964      FSImageSerialization.writeString(path, out);
2965      FSImageSerialization.writeString(newHolder, out);
2966    }
2967
2968    @Override
2969    void readFields(DataInputStream in, int logVersion)
2970        throws IOException {
2971      this.leaseHolder = FSImageSerialization.readString(in);
2972      this.path = FSImageSerialization.readString(in);
2973      this.newHolder = FSImageSerialization.readString(in);
2974    }
2975
2976    @Override
2977    public String toString() {
2978      StringBuilder builder = new StringBuilder();
2979      builder.append("ReassignLeaseOp [leaseHolder=");
2980      builder.append(leaseHolder);
2981      builder.append(", path=");
2982      builder.append(path);
2983      builder.append(", newHolder=");
2984      builder.append(newHolder);
2985      builder.append(", opCode=");
2986      builder.append(opCode);
2987      builder.append(", txid=");
2988      builder.append(txid);
2989      builder.append("]");
2990      return builder.toString();
2991    }
2992    
2993    @Override
2994    protected void toXml(ContentHandler contentHandler) throws SAXException {
2995      XMLUtils.addSaxString(contentHandler, "LEASEHOLDER", leaseHolder);
2996      XMLUtils.addSaxString(contentHandler, "PATH", path);
2997      XMLUtils.addSaxString(contentHandler, "NEWHOLDER", newHolder);
2998    }
2999    
3000    @Override void fromXml(Stanza st) throws InvalidXmlException {
3001      this.leaseHolder = st.getValue("LEASEHOLDER");
3002      this.path = st.getValue("PATH");
3003      this.newHolder = st.getValue("NEWHOLDER");
3004    }
3005  }
3006
3007  /** {@literal @Idempotent} for {@link ClientProtocol#getDelegationToken} */
3008  static class GetDelegationTokenOp extends FSEditLogOp {
3009    DelegationTokenIdentifier token;
3010    long expiryTime;
3011
3012    private GetDelegationTokenOp() {
3013      super(OP_GET_DELEGATION_TOKEN);
3014    }
3015
3016    static GetDelegationTokenOp getInstance(OpInstanceCache cache) {
3017      return (GetDelegationTokenOp)cache.get(OP_GET_DELEGATION_TOKEN);
3018    }
3019
3020    @Override
3021    void resetSubFields() {
3022      token = null;
3023      expiryTime = 0L;
3024    }
3025
3026    GetDelegationTokenOp setDelegationTokenIdentifier(
3027        DelegationTokenIdentifier token) {
3028      this.token = token;
3029      return this;
3030    }
3031
3032    GetDelegationTokenOp setExpiryTime(long expiryTime) {
3033      this.expiryTime = expiryTime;
3034      return this;
3035    }
3036
3037    @Override
3038    public 
3039    void writeFields(DataOutputStream out) throws IOException {
3040      token.write(out);
3041      FSImageSerialization.writeLong(expiryTime, out);
3042    }
3043
3044    @Override
3045    void readFields(DataInputStream in, int logVersion)
3046        throws IOException {
3047      this.token = new DelegationTokenIdentifier();
3048      this.token.readFields(in);
3049      if (NameNodeLayoutVersion.supports(
3050          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
3051        this.expiryTime = FSImageSerialization.readLong(in);
3052      } else {
3053        this.expiryTime = readLong(in);
3054      }
3055    }
3056
3057    @Override
3058    public String toString() {
3059      StringBuilder builder = new StringBuilder();
3060      builder.append("GetDelegationTokenOp [token=");
3061      builder.append(token);
3062      builder.append(", expiryTime=");
3063      builder.append(expiryTime);
3064      builder.append(", opCode=");
3065      builder.append(opCode);
3066      builder.append(", txid=");
3067      builder.append(txid);
3068      builder.append("]");
3069      return builder.toString();
3070    }
3071    
3072    @Override
3073    protected void toXml(ContentHandler contentHandler) throws SAXException {
3074      FSEditLogOp.delegationTokenToXml(contentHandler, token);
3075      XMLUtils.addSaxString(contentHandler, "EXPIRY_TIME",
3076          Long.toString(expiryTime));
3077    }
3078    
3079    @Override void fromXml(Stanza st) throws InvalidXmlException {
3080      this.token = delegationTokenFromXml(st.getChildren(
3081          "DELEGATION_TOKEN_IDENTIFIER").get(0));
3082      this.expiryTime = Long.parseLong(st.getValue("EXPIRY_TIME"));
3083    }
3084  }
3085
3086  /** {@literal @Idempotent} for {@link ClientProtocol#renewDelegationToken} */
3087  static class RenewDelegationTokenOp extends FSEditLogOp {
3088    DelegationTokenIdentifier token;
3089    long expiryTime;
3090
3091    private RenewDelegationTokenOp() {
3092      super(OP_RENEW_DELEGATION_TOKEN);
3093    }
3094
3095    static RenewDelegationTokenOp getInstance(OpInstanceCache cache) {
3096      return (RenewDelegationTokenOp)cache.get(OP_RENEW_DELEGATION_TOKEN);
3097    }
3098
3099    @Override
3100    void resetSubFields() {
3101      token = null;
3102      expiryTime = 0L;
3103    }
3104
3105    RenewDelegationTokenOp setDelegationTokenIdentifier(
3106        DelegationTokenIdentifier token) {
3107      this.token = token;
3108      return this;
3109    }
3110
3111    RenewDelegationTokenOp setExpiryTime(long expiryTime) {
3112      this.expiryTime = expiryTime;
3113      return this;
3114    }
3115
3116    @Override
3117    public 
3118    void writeFields(DataOutputStream out) throws IOException {
3119      token.write(out);
3120      FSImageSerialization.writeLong(expiryTime, out);
3121    }
3122
3123    @Override
3124    void readFields(DataInputStream in, int logVersion)
3125        throws IOException {
3126      this.token = new DelegationTokenIdentifier();
3127      this.token.readFields(in);
3128      if (NameNodeLayoutVersion.supports(
3129          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
3130        this.expiryTime = FSImageSerialization.readLong(in);
3131      } else {
3132        this.expiryTime = readLong(in);
3133      }
3134    }
3135
3136    @Override
3137    public String toString() {
3138      StringBuilder builder = new StringBuilder();
3139      builder.append("RenewDelegationTokenOp [token=");
3140      builder.append(token);
3141      builder.append(", expiryTime=");
3142      builder.append(expiryTime);
3143      builder.append(", opCode=");
3144      builder.append(opCode);
3145      builder.append(", txid=");
3146      builder.append(txid);
3147      builder.append("]");
3148      return builder.toString();
3149    }
3150    
3151    @Override
3152    protected void toXml(ContentHandler contentHandler) throws SAXException {
3153      FSEditLogOp.delegationTokenToXml(contentHandler, token);
3154      XMLUtils.addSaxString(contentHandler, "EXPIRY_TIME",
3155          Long.toString(expiryTime));
3156    }
3157    
3158    @Override void fromXml(Stanza st) throws InvalidXmlException {
3159      this.token = delegationTokenFromXml(st.getChildren(
3160          "DELEGATION_TOKEN_IDENTIFIER").get(0));
3161      this.expiryTime = Long.parseLong(st.getValue("EXPIRY_TIME"));
3162    }
3163  }
3164
3165  /** {@literal @Idempotent} for {@link ClientProtocol#cancelDelegationToken} */
3166  static class CancelDelegationTokenOp extends FSEditLogOp {
3167    DelegationTokenIdentifier token;
3168
3169    private CancelDelegationTokenOp() {
3170      super(OP_CANCEL_DELEGATION_TOKEN);
3171    }
3172
3173    static CancelDelegationTokenOp getInstance(OpInstanceCache cache) {
3174      return (CancelDelegationTokenOp)cache.get(OP_CANCEL_DELEGATION_TOKEN);
3175    }
3176
3177    @Override
3178    void resetSubFields() {
3179      token = null;
3180    }
3181
3182    CancelDelegationTokenOp setDelegationTokenIdentifier(
3183        DelegationTokenIdentifier token) {
3184      this.token = token;
3185      return this;
3186    }
3187
3188    @Override
3189    public 
3190    void writeFields(DataOutputStream out) throws IOException {
3191      token.write(out);
3192    }
3193
3194    @Override
3195    void readFields(DataInputStream in, int logVersion)
3196        throws IOException {
3197      this.token = new DelegationTokenIdentifier();
3198      this.token.readFields(in);
3199    }
3200
3201    @Override
3202    public String toString() {
3203      StringBuilder builder = new StringBuilder();
3204      builder.append("CancelDelegationTokenOp [token=");
3205      builder.append(token);
3206      builder.append(", opCode=");
3207      builder.append(opCode);
3208      builder.append(", txid=");
3209      builder.append(txid);
3210      builder.append("]");
3211      return builder.toString();
3212    }
3213    
3214    @Override
3215    protected void toXml(ContentHandler contentHandler) throws SAXException {
3216      FSEditLogOp.delegationTokenToXml(contentHandler, token);
3217    }
3218    
3219    @Override void fromXml(Stanza st) throws InvalidXmlException {
3220      this.token = delegationTokenFromXml(st.getChildren(
3221          "DELEGATION_TOKEN_IDENTIFIER").get(0));
3222    }
3223  }
3224
3225  static class UpdateMasterKeyOp extends FSEditLogOp {
3226    DelegationKey key;
3227
3228    private UpdateMasterKeyOp() {
3229      super(OP_UPDATE_MASTER_KEY);
3230    }
3231
3232    static UpdateMasterKeyOp getInstance(OpInstanceCache cache) {
3233      return (UpdateMasterKeyOp)cache.get(OP_UPDATE_MASTER_KEY);
3234    }
3235
3236    @Override
3237    void resetSubFields() {
3238      key = null;
3239    }
3240
3241    UpdateMasterKeyOp setDelegationKey(DelegationKey key) {
3242      this.key = key;
3243      return this;
3244    }
3245    
3246    @Override
3247    public 
3248    void writeFields(DataOutputStream out) throws IOException {
3249      key.write(out);
3250    }
3251
3252    @Override
3253    void readFields(DataInputStream in, int logVersion)
3254        throws IOException {
3255      this.key = new DelegationKey();
3256      this.key.readFields(in);
3257    }
3258
3259    @Override
3260    public String toString() {
3261      StringBuilder builder = new StringBuilder();
3262      builder.append("UpdateMasterKeyOp [key=");
3263      builder.append(key);
3264      builder.append(", opCode=");
3265      builder.append(opCode);
3266      builder.append(", txid=");
3267      builder.append(txid);
3268      builder.append("]");
3269      return builder.toString();
3270    }
3271    
3272    @Override
3273    protected void toXml(ContentHandler contentHandler) throws SAXException {
3274      FSEditLogOp.delegationKeyToXml(contentHandler, key);
3275    }
3276    
3277    @Override void fromXml(Stanza st) throws InvalidXmlException {
3278      this.key = delegationKeyFromXml(st.getChildren(
3279          "DELEGATION_KEY").get(0));
3280    }
3281  }
3282  
3283  static class LogSegmentOp extends FSEditLogOp {
3284    private LogSegmentOp(FSEditLogOpCodes code) {
3285      super(code);
3286      assert code == OP_START_LOG_SEGMENT ||
3287             code == OP_END_LOG_SEGMENT : "Bad op: " + code;
3288    }
3289
3290    static LogSegmentOp getInstance(OpInstanceCache cache,
3291        FSEditLogOpCodes code) {
3292      return (LogSegmentOp)cache.get(code);
3293    }
3294
3295    @Override
3296    void resetSubFields() {
3297      // no data stored in these ops yet
3298    }
3299
3300    @Override
3301    public void readFields(DataInputStream in, int logVersion)
3302        throws IOException {
3303      // no data stored in these ops yet
3304    }
3305
3306    @Override
3307    public
3308    void writeFields(DataOutputStream out) throws IOException {
3309      // no data stored
3310    }
3311
3312    @Override
3313    public String toString() {
3314      StringBuilder builder = new StringBuilder();
3315      builder.append("LogSegmentOp [opCode=");
3316      builder.append(opCode);
3317      builder.append(", txid=");
3318      builder.append(txid);
3319      builder.append("]");
3320      return builder.toString();
3321    }
3322
3323    @Override
3324    protected void toXml(ContentHandler contentHandler) throws SAXException {
3325      // no data stored
3326    }
3327    
3328    @Override void fromXml(Stanza st) throws InvalidXmlException {
3329      // do nothing
3330    }
3331  }
3332
3333  static class InvalidOp extends FSEditLogOp {
3334    private InvalidOp() {
3335      super(OP_INVALID);
3336    }
3337
3338    static InvalidOp getInstance(OpInstanceCache cache) {
3339      return (InvalidOp)cache.get(OP_INVALID);
3340    }
3341
3342    @Override
3343    void resetSubFields() {
3344    }
3345
3346    @Override
3347    public 
3348    void writeFields(DataOutputStream out) throws IOException {
3349    }
3350    
3351    @Override
3352    void readFields(DataInputStream in, int logVersion)
3353        throws IOException {
3354      // nothing to read
3355    }
3356
3357    @Override
3358    public String toString() {
3359      StringBuilder builder = new StringBuilder();
3360      builder.append("InvalidOp [opCode=");
3361      builder.append(opCode);
3362      builder.append(", txid=");
3363      builder.append(txid);
3364      builder.append("]");
3365      return builder.toString();
3366    }
3367    @Override
3368    protected void toXml(ContentHandler contentHandler) throws SAXException {
3369      // no data stored
3370    }
3371    
3372    @Override void fromXml(Stanza st) throws InvalidXmlException {
3373      // do nothing
3374    }
3375  }
3376
3377  /**
3378   * Operation corresponding to creating a snapshot.
3379   * {@literal @AtMostOnce} for {@link ClientProtocol#createSnapshot}.
3380   */
3381  static class CreateSnapshotOp extends FSEditLogOp {
3382    String snapshotRoot;
3383    String snapshotName;
3384    
3385    public CreateSnapshotOp() {
3386      super(OP_CREATE_SNAPSHOT);
3387    }
3388    
3389    static CreateSnapshotOp getInstance(OpInstanceCache cache) {
3390      return (CreateSnapshotOp)cache.get(OP_CREATE_SNAPSHOT);
3391    }
3392
3393    @Override
3394    void resetSubFields() {
3395      snapshotRoot = null;
3396      snapshotName = null;
3397    }
3398
3399    CreateSnapshotOp setSnapshotName(String snapName) {
3400      this.snapshotName = snapName;
3401      return this;
3402    }
3403
3404    public CreateSnapshotOp setSnapshotRoot(String snapRoot) {
3405      snapshotRoot = snapRoot;
3406      return this;
3407    }
3408    
3409    @Override
3410    void readFields(DataInputStream in, int logVersion) throws IOException {
3411      snapshotRoot = FSImageSerialization.readString(in);
3412      snapshotName = FSImageSerialization.readString(in);
3413      
3414      // read RPC ids if necessary
3415      readRpcIds(in, logVersion);
3416    }
3417
3418    @Override
3419    public void writeFields(DataOutputStream out) throws IOException {
3420      FSImageSerialization.writeString(snapshotRoot, out);
3421      FSImageSerialization.writeString(snapshotName, out);
3422      writeRpcIds(rpcClientId, rpcCallId, out);
3423    }
3424
3425    @Override
3426    protected void toXml(ContentHandler contentHandler) throws SAXException {
3427      XMLUtils.addSaxString(contentHandler, "SNAPSHOTROOT", snapshotRoot);
3428      XMLUtils.addSaxString(contentHandler, "SNAPSHOTNAME", snapshotName);
3429      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
3430    }
3431
3432    @Override
3433    void fromXml(Stanza st) throws InvalidXmlException {
3434      snapshotRoot = st.getValue("SNAPSHOTROOT");
3435      snapshotName = st.getValue("SNAPSHOTNAME");
3436      
3437      readRpcIdsFromXml(st);
3438    }
3439    
3440    @Override
3441    public String toString() {
3442      StringBuilder builder = new StringBuilder();
3443      builder.append("CreateSnapshotOp [snapshotRoot=");
3444      builder.append(snapshotRoot);
3445      builder.append(", snapshotName=");
3446      builder.append(snapshotName);
3447      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
3448      builder.append("]");
3449      return builder.toString();
3450    }
3451  }
3452  
3453  /**
3454   * Operation corresponding to delete a snapshot.
3455   * {@literal @AtMostOnce} for {@link ClientProtocol#deleteSnapshot}.
3456   */
3457  static class DeleteSnapshotOp extends FSEditLogOp {
3458    String snapshotRoot;
3459    String snapshotName;
3460    
3461    DeleteSnapshotOp() {
3462      super(OP_DELETE_SNAPSHOT);
3463    }
3464    
3465    static DeleteSnapshotOp getInstance(OpInstanceCache cache) {
3466      return (DeleteSnapshotOp)cache.get(OP_DELETE_SNAPSHOT);
3467    }
3468
3469    @Override
3470    void resetSubFields() {
3471      snapshotRoot = null;
3472      snapshotName = null;
3473    }
3474    
3475    DeleteSnapshotOp setSnapshotName(String snapName) {
3476      this.snapshotName = snapName;
3477      return this;
3478    }
3479
3480    DeleteSnapshotOp setSnapshotRoot(String snapRoot) {
3481      snapshotRoot = snapRoot;
3482      return this;
3483    }
3484    
3485    @Override
3486    void readFields(DataInputStream in, int logVersion) throws IOException {
3487      snapshotRoot = FSImageSerialization.readString(in);
3488      snapshotName = FSImageSerialization.readString(in);
3489      
3490      // read RPC ids if necessary
3491      readRpcIds(in, logVersion);
3492    }
3493
3494    @Override
3495    public void writeFields(DataOutputStream out) throws IOException {
3496      FSImageSerialization.writeString(snapshotRoot, out);
3497      FSImageSerialization.writeString(snapshotName, out);
3498      writeRpcIds(rpcClientId, rpcCallId, out);
3499    }
3500
3501    @Override
3502    protected void toXml(ContentHandler contentHandler) throws SAXException {
3503      XMLUtils.addSaxString(contentHandler, "SNAPSHOTROOT", snapshotRoot);
3504      XMLUtils.addSaxString(contentHandler, "SNAPSHOTNAME", snapshotName);
3505      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
3506    }
3507
3508    @Override
3509    void fromXml(Stanza st) throws InvalidXmlException {
3510      snapshotRoot = st.getValue("SNAPSHOTROOT");
3511      snapshotName = st.getValue("SNAPSHOTNAME");
3512      
3513      readRpcIdsFromXml(st);
3514    }
3515    
3516    @Override
3517    public String toString() {
3518      StringBuilder builder = new StringBuilder();
3519      builder.append("DeleteSnapshotOp [snapshotRoot=");
3520      builder.append(snapshotRoot);
3521      builder.append(", snapshotName=");
3522      builder.append(snapshotName);
3523      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
3524      builder.append("]");
3525      return builder.toString();
3526    }
3527  }
3528  
3529  /**
3530   * Operation corresponding to rename a snapshot.
3531   * {@literal @AtMostOnce} for {@link ClientProtocol#renameSnapshot}.
3532   */
3533  static class RenameSnapshotOp extends FSEditLogOp {
3534    String snapshotRoot;
3535    String snapshotOldName;
3536    String snapshotNewName;
3537    
3538    RenameSnapshotOp() {
3539      super(OP_RENAME_SNAPSHOT);
3540    }
3541    
3542    static RenameSnapshotOp getInstance(OpInstanceCache cache) {
3543      return (RenameSnapshotOp) cache.get(OP_RENAME_SNAPSHOT);
3544    }
3545
3546    @Override
3547    void resetSubFields() {
3548      snapshotRoot = null;
3549      snapshotOldName = null;
3550      snapshotNewName = null;
3551    }
3552    
3553    RenameSnapshotOp setSnapshotOldName(String snapshotOldName) {
3554      this.snapshotOldName = snapshotOldName;
3555      return this;
3556    }
3557
3558    RenameSnapshotOp setSnapshotNewName(String snapshotNewName) {
3559      this.snapshotNewName = snapshotNewName;
3560      return this;
3561    }
3562    
3563    RenameSnapshotOp setSnapshotRoot(String snapshotRoot) {
3564      this.snapshotRoot = snapshotRoot;
3565      return this;
3566    }
3567    
3568    @Override
3569    void readFields(DataInputStream in, int logVersion) throws IOException {
3570      snapshotRoot = FSImageSerialization.readString(in);
3571      snapshotOldName = FSImageSerialization.readString(in);
3572      snapshotNewName = FSImageSerialization.readString(in);
3573      
3574      // read RPC ids if necessary
3575      readRpcIds(in, logVersion);
3576    }
3577
3578    @Override
3579    public void writeFields(DataOutputStream out) throws IOException {
3580      FSImageSerialization.writeString(snapshotRoot, out);
3581      FSImageSerialization.writeString(snapshotOldName, out);
3582      FSImageSerialization.writeString(snapshotNewName, out);
3583      
3584      writeRpcIds(rpcClientId, rpcCallId, out);
3585    }
3586
3587    @Override
3588    protected void toXml(ContentHandler contentHandler) throws SAXException {
3589      XMLUtils.addSaxString(contentHandler, "SNAPSHOTROOT", snapshotRoot);
3590      XMLUtils.addSaxString(contentHandler, "SNAPSHOTOLDNAME", snapshotOldName);
3591      XMLUtils.addSaxString(contentHandler, "SNAPSHOTNEWNAME", snapshotNewName);
3592      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
3593    }
3594
3595    @Override
3596    void fromXml(Stanza st) throws InvalidXmlException {
3597      snapshotRoot = st.getValue("SNAPSHOTROOT");
3598      snapshotOldName = st.getValue("SNAPSHOTOLDNAME");
3599      snapshotNewName = st.getValue("SNAPSHOTNEWNAME");
3600      
3601      readRpcIdsFromXml(st);
3602    }
3603    
3604    @Override
3605    public String toString() {
3606      StringBuilder builder = new StringBuilder();
3607      builder.append("RenameSnapshotOp [snapshotRoot=");
3608      builder.append(snapshotRoot);
3609      builder.append(", snapshotOldName=");
3610      builder.append(snapshotOldName);
3611      builder.append(", snapshotNewName=");
3612      builder.append(snapshotNewName);
3613      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
3614      builder.append("]");
3615      return builder.toString();
3616    }
3617  }
3618
3619  /**
3620   * Operation corresponding to allow creating snapshot on a directory
3621   */
3622  static class AllowSnapshotOp extends FSEditLogOp { // @Idempotent
3623    String snapshotRoot;
3624
3625    public AllowSnapshotOp() {
3626      super(OP_ALLOW_SNAPSHOT);
3627    }
3628
3629    public AllowSnapshotOp(String snapRoot) {
3630      super(OP_ALLOW_SNAPSHOT);
3631      snapshotRoot = snapRoot;
3632    }
3633
3634    static AllowSnapshotOp getInstance(OpInstanceCache cache) {
3635      return (AllowSnapshotOp) cache.get(OP_ALLOW_SNAPSHOT);
3636    }
3637
3638    @Override
3639    void resetSubFields() {
3640      snapshotRoot = null;
3641    }
3642
3643    public AllowSnapshotOp setSnapshotRoot(String snapRoot) {
3644      snapshotRoot = snapRoot;
3645      return this;
3646    }
3647
3648    @Override
3649    void readFields(DataInputStream in, int logVersion) throws IOException {
3650      snapshotRoot = FSImageSerialization.readString(in);
3651    }
3652
3653    @Override
3654    public void writeFields(DataOutputStream out) throws IOException {
3655      FSImageSerialization.writeString(snapshotRoot, out);
3656    }
3657
3658    @Override
3659    protected void toXml(ContentHandler contentHandler) throws SAXException {
3660      XMLUtils.addSaxString(contentHandler, "SNAPSHOTROOT", snapshotRoot);
3661    }
3662
3663    @Override
3664    void fromXml(Stanza st) throws InvalidXmlException {
3665      snapshotRoot = st.getValue("SNAPSHOTROOT");
3666    }
3667
3668    @Override
3669    public String toString() {
3670      StringBuilder builder = new StringBuilder();
3671      builder.append("AllowSnapshotOp [snapshotRoot=");
3672      builder.append(snapshotRoot);
3673      builder.append("]");
3674      return builder.toString();
3675    }
3676  }
3677
3678  /**
3679   * Operation corresponding to disallow creating snapshot on a directory
3680   */
3681  static class DisallowSnapshotOp extends FSEditLogOp { // @Idempotent
3682    String snapshotRoot;
3683
3684    public DisallowSnapshotOp() {
3685      super(OP_DISALLOW_SNAPSHOT);
3686    }
3687
3688    public DisallowSnapshotOp(String snapRoot) {
3689      super(OP_DISALLOW_SNAPSHOT);
3690      snapshotRoot = snapRoot;
3691    }
3692
3693    static DisallowSnapshotOp getInstance(OpInstanceCache cache) {
3694      return (DisallowSnapshotOp) cache.get(OP_DISALLOW_SNAPSHOT);
3695    }
3696
3697    void resetSubFields() {
3698      snapshotRoot = null;
3699    }
3700
3701    public DisallowSnapshotOp setSnapshotRoot(String snapRoot) {
3702      snapshotRoot = snapRoot;
3703      return this;
3704    }
3705
3706    @Override
3707    void readFields(DataInputStream in, int logVersion) throws IOException {
3708      snapshotRoot = FSImageSerialization.readString(in);
3709    }
3710
3711    @Override
3712    public void writeFields(DataOutputStream out) throws IOException {
3713      FSImageSerialization.writeString(snapshotRoot, out);
3714    }
3715
3716    @Override
3717    protected void toXml(ContentHandler contentHandler) throws SAXException {
3718      XMLUtils.addSaxString(contentHandler, "SNAPSHOTROOT", snapshotRoot);
3719    }
3720
3721    @Override
3722    void fromXml(Stanza st) throws InvalidXmlException {
3723      snapshotRoot = st.getValue("SNAPSHOTROOT");
3724    }
3725
3726    @Override
3727    public String toString() {
3728      StringBuilder builder = new StringBuilder();
3729      builder.append("DisallowSnapshotOp [snapshotRoot=");
3730      builder.append(snapshotRoot);
3731      builder.append("]");
3732      return builder.toString();
3733    }
3734  }
3735
3736  /**
3737   * {@literal @AtMostOnce} for
3738   * {@link ClientProtocol#addCacheDirective}
3739   */
3740  static class AddCacheDirectiveInfoOp extends FSEditLogOp {
3741    CacheDirectiveInfo directive;
3742
3743    public AddCacheDirectiveInfoOp() {
3744      super(OP_ADD_CACHE_DIRECTIVE);
3745    }
3746
3747    static AddCacheDirectiveInfoOp getInstance(OpInstanceCache cache) {
3748      return (AddCacheDirectiveInfoOp) cache
3749          .get(OP_ADD_CACHE_DIRECTIVE);
3750    }
3751
3752    @Override
3753    void resetSubFields() {
3754      directive = null;
3755    }
3756
3757    public AddCacheDirectiveInfoOp setDirective(
3758        CacheDirectiveInfo directive) {
3759      this.directive = directive;
3760      assert(directive.getId() != null);
3761      assert(directive.getPath() != null);
3762      assert(directive.getReplication() != null);
3763      assert(directive.getPool() != null);
3764      assert(directive.getExpiration() != null);
3765      return this;
3766    }
3767
3768    @Override
3769    void readFields(DataInputStream in, int logVersion) throws IOException {
3770      directive = FSImageSerialization.readCacheDirectiveInfo(in);
3771      readRpcIds(in, logVersion);
3772    }
3773
3774    @Override
3775    public void writeFields(DataOutputStream out) throws IOException {
3776      FSImageSerialization.writeCacheDirectiveInfo(out, directive);
3777      writeRpcIds(rpcClientId, rpcCallId, out);
3778    }
3779
3780    @Override
3781    protected void toXml(ContentHandler contentHandler) throws SAXException {
3782      FSImageSerialization.writeCacheDirectiveInfo(contentHandler, directive);
3783      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
3784    }
3785
3786    @Override
3787    void fromXml(Stanza st) throws InvalidXmlException {
3788      directive = FSImageSerialization.readCacheDirectiveInfo(st);
3789      readRpcIdsFromXml(st);
3790    }
3791
3792    @Override
3793    public String toString() {
3794      StringBuilder builder = new StringBuilder();
3795      builder.append("AddCacheDirectiveInfo [");
3796      builder.append("id=" + directive.getId() + ",");
3797      builder.append("path=" + directive.getPath().toUri().getPath() + ",");
3798      builder.append("replication=" + directive.getReplication() + ",");
3799      builder.append("pool=" + directive.getPool() + ",");
3800      builder.append("expiration=" + directive.getExpiration().getMillis());
3801      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
3802      builder.append("]");
3803      return builder.toString();
3804    }
3805  }
3806
3807  /**
3808   * {@literal @AtMostOnce} for
3809   * {@link ClientProtocol#modifyCacheDirective}
3810   */
3811  static class ModifyCacheDirectiveInfoOp extends FSEditLogOp {
3812    CacheDirectiveInfo directive;
3813
3814    public ModifyCacheDirectiveInfoOp() {
3815      super(OP_MODIFY_CACHE_DIRECTIVE);
3816    }
3817
3818    static ModifyCacheDirectiveInfoOp getInstance(OpInstanceCache cache) {
3819      return (ModifyCacheDirectiveInfoOp) cache
3820          .get(OP_MODIFY_CACHE_DIRECTIVE);
3821    }
3822
3823    @Override
3824    void resetSubFields() {
3825      directive = null;
3826    }
3827
3828    public ModifyCacheDirectiveInfoOp setDirective(
3829        CacheDirectiveInfo directive) {
3830      this.directive = directive;
3831      assert(directive.getId() != null);
3832      return this;
3833    }
3834
3835    @Override
3836    void readFields(DataInputStream in, int logVersion) throws IOException {
3837      this.directive = FSImageSerialization.readCacheDirectiveInfo(in);
3838      readRpcIds(in, logVersion);
3839    }
3840
3841    @Override
3842    public void writeFields(DataOutputStream out) throws IOException {
3843      FSImageSerialization.writeCacheDirectiveInfo(out, directive);
3844      writeRpcIds(rpcClientId, rpcCallId, out);
3845    }
3846
3847    @Override
3848    protected void toXml(ContentHandler contentHandler) throws SAXException {
3849      FSImageSerialization.writeCacheDirectiveInfo(contentHandler, directive);
3850      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
3851    }
3852
3853    @Override
3854    void fromXml(Stanza st) throws InvalidXmlException {
3855      this.directive = FSImageSerialization.readCacheDirectiveInfo(st);
3856      readRpcIdsFromXml(st);
3857    }
3858
3859    @Override
3860    public String toString() {
3861      StringBuilder builder = new StringBuilder();
3862      builder.append("ModifyCacheDirectiveInfoOp[");
3863      builder.append("id=").append(directive.getId());
3864      if (directive.getPath() != null) {
3865        builder.append(",").append("path=").append(directive.getPath());
3866      }
3867      if (directive.getReplication() != null) {
3868        builder.append(",").append("replication=").
3869            append(directive.getReplication());
3870      }
3871      if (directive.getPool() != null) {
3872        builder.append(",").append("pool=").append(directive.getPool());
3873      }
3874      if (directive.getExpiration() != null) {
3875        builder.append(",").append("expiration=").
3876            append(directive.getExpiration().getMillis());
3877      }
3878      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
3879      builder.append("]");
3880      return builder.toString();
3881    }
3882  }
3883
3884  /**
3885   * {@literal @AtMostOnce} for
3886   * {@link ClientProtocol#removeCacheDirective}
3887   */
3888  static class RemoveCacheDirectiveInfoOp extends FSEditLogOp {
3889    long id;
3890
3891    public RemoveCacheDirectiveInfoOp() {
3892      super(OP_REMOVE_CACHE_DIRECTIVE);
3893    }
3894
3895    static RemoveCacheDirectiveInfoOp getInstance(OpInstanceCache cache) {
3896      return (RemoveCacheDirectiveInfoOp) cache
3897          .get(OP_REMOVE_CACHE_DIRECTIVE);
3898    }
3899
3900    @Override
3901    void resetSubFields() {
3902      id = 0L;
3903    }
3904
3905    public RemoveCacheDirectiveInfoOp setId(long id) {
3906      this.id = id;
3907      return this;
3908    }
3909
3910    @Override
3911    void readFields(DataInputStream in, int logVersion) throws IOException {
3912      this.id = FSImageSerialization.readLong(in);
3913      readRpcIds(in, logVersion);
3914    }
3915
3916    @Override
3917    public void writeFields(DataOutputStream out) throws IOException {
3918      FSImageSerialization.writeLong(id, out);
3919      writeRpcIds(rpcClientId, rpcCallId, out);
3920    }
3921
3922    @Override
3923    protected void toXml(ContentHandler contentHandler) throws SAXException {
3924      XMLUtils.addSaxString(contentHandler, "ID", Long.toString(id));
3925      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
3926    }
3927
3928    @Override
3929    void fromXml(Stanza st) throws InvalidXmlException {
3930      this.id = Long.parseLong(st.getValue("ID"));
3931      readRpcIdsFromXml(st);
3932    }
3933
3934    @Override
3935    public String toString() {
3936      StringBuilder builder = new StringBuilder();
3937      builder.append("RemoveCacheDirectiveInfo [");
3938      builder.append("id=" + Long.toString(id));
3939      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
3940      builder.append("]");
3941      return builder.toString();
3942    }
3943  }
3944
3945  /** {@literal @AtMostOnce} for {@link ClientProtocol#addCachePool} */
3946  static class AddCachePoolOp extends FSEditLogOp {
3947    CachePoolInfo info;
3948
3949    public AddCachePoolOp() {
3950      super(OP_ADD_CACHE_POOL);
3951    }
3952
3953    static AddCachePoolOp getInstance(OpInstanceCache cache) {
3954      return (AddCachePoolOp) cache.get(OP_ADD_CACHE_POOL);
3955    }
3956
3957    @Override
3958    void resetSubFields() {
3959      info = null;
3960    }
3961
3962    public AddCachePoolOp setPool(CachePoolInfo info) {
3963      this.info = info;
3964      assert(info.getPoolName() != null);
3965      assert(info.getOwnerName() != null);
3966      assert(info.getGroupName() != null);
3967      assert(info.getMode() != null);
3968      assert(info.getLimit() != null);
3969      return this;
3970    }
3971
3972    @Override
3973    void readFields(DataInputStream in, int logVersion) throws IOException {
3974      info = FSImageSerialization.readCachePoolInfo(in);
3975      readRpcIds(in, logVersion);
3976    }
3977
3978    @Override
3979    public void writeFields(DataOutputStream out) throws IOException {
3980      FSImageSerialization.writeCachePoolInfo(out, info);
3981      writeRpcIds(rpcClientId, rpcCallId, out);
3982    }
3983
3984    @Override
3985    protected void toXml(ContentHandler contentHandler) throws SAXException {
3986      FSImageSerialization.writeCachePoolInfo(contentHandler, info);
3987      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
3988    }
3989
3990    @Override
3991    void fromXml(Stanza st) throws InvalidXmlException {
3992      this.info = FSImageSerialization.readCachePoolInfo(st);
3993      readRpcIdsFromXml(st);
3994    }
3995
3996    @Override
3997    public String toString() {
3998      StringBuilder builder = new StringBuilder();
3999      builder.append("AddCachePoolOp [");
4000      builder.append("poolName=" + info.getPoolName() + ",");
4001      builder.append("ownerName=" + info.getOwnerName() + ",");
4002      builder.append("groupName=" + info.getGroupName() + ",");
4003      builder.append("mode=" + Short.toString(info.getMode().toShort()) + ",");
4004      builder.append("limit=" + Long.toString(info.getLimit()));
4005      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
4006      builder.append("]");
4007      return builder.toString();
4008    }
4009  }
4010
4011  /** {@literal @AtMostOnce} for {@link ClientProtocol#modifyCachePool} */
4012  static class ModifyCachePoolOp extends FSEditLogOp {
4013    CachePoolInfo info;
4014
4015    public ModifyCachePoolOp() {
4016      super(OP_MODIFY_CACHE_POOL);
4017    }
4018
4019    static ModifyCachePoolOp getInstance(OpInstanceCache cache) {
4020      return (ModifyCachePoolOp) cache.get(OP_MODIFY_CACHE_POOL);
4021    }
4022
4023    @Override
4024    void resetSubFields() {
4025      info = null;
4026    }
4027
4028    public ModifyCachePoolOp setInfo(CachePoolInfo info) {
4029      this.info = info;
4030      return this;
4031    }
4032
4033    @Override
4034    void readFields(DataInputStream in, int logVersion) throws IOException {
4035      info = FSImageSerialization.readCachePoolInfo(in);
4036      readRpcIds(in, logVersion);
4037    }
4038
4039    @Override
4040    public void writeFields(DataOutputStream out) throws IOException {
4041      FSImageSerialization.writeCachePoolInfo(out, info);
4042      writeRpcIds(rpcClientId, rpcCallId, out);
4043    }
4044
4045    @Override
4046    protected void toXml(ContentHandler contentHandler) throws SAXException {
4047      FSImageSerialization.writeCachePoolInfo(contentHandler, info);
4048      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
4049    }
4050
4051    @Override
4052    void fromXml(Stanza st) throws InvalidXmlException {
4053      this.info = FSImageSerialization.readCachePoolInfo(st);
4054      readRpcIdsFromXml(st);
4055    }
4056
4057    @Override
4058    public String toString() {
4059      StringBuilder builder = new StringBuilder();
4060      builder.append("ModifyCachePoolOp [");
4061      ArrayList<String> fields = new ArrayList<String>(5);
4062      if (info.getPoolName() != null) {
4063        fields.add("poolName=" + info.getPoolName());
4064      }
4065      if (info.getOwnerName() != null) {
4066        fields.add("ownerName=" + info.getOwnerName());
4067      }
4068      if (info.getGroupName() != null) {
4069        fields.add("groupName=" + info.getGroupName());
4070      }
4071      if (info.getMode() != null) {
4072        fields.add("mode=" + info.getMode().toString());
4073      }
4074      if (info.getLimit() != null) {
4075        fields.add("limit=" + info.getLimit());
4076      }
4077      builder.append(Joiner.on(",").join(fields));
4078      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
4079      builder.append("]");
4080      return builder.toString();
4081    }
4082  }
4083
4084  /** {@literal @AtMostOnce} for {@link ClientProtocol#removeCachePool} */
4085  static class RemoveCachePoolOp extends FSEditLogOp {
4086    String poolName;
4087
4088    public RemoveCachePoolOp() {
4089      super(OP_REMOVE_CACHE_POOL);
4090    }
4091
4092    static RemoveCachePoolOp getInstance(OpInstanceCache cache) {
4093      return (RemoveCachePoolOp) cache.get(OP_REMOVE_CACHE_POOL);
4094    }
4095
4096    @Override
4097    void resetSubFields() {
4098      poolName = null;
4099    }
4100
4101    public RemoveCachePoolOp setPoolName(String poolName) {
4102      this.poolName = poolName;
4103      return this;
4104    }
4105
4106    @Override
4107    void readFields(DataInputStream in, int logVersion) throws IOException {
4108      poolName = FSImageSerialization.readString(in);
4109      readRpcIds(in, logVersion);
4110    }
4111
4112    @Override
4113    public void writeFields(DataOutputStream out) throws IOException {
4114      FSImageSerialization.writeString(poolName, out);
4115      writeRpcIds(rpcClientId, rpcCallId, out);
4116    }
4117
4118    @Override
4119    protected void toXml(ContentHandler contentHandler) throws SAXException {
4120      XMLUtils.addSaxString(contentHandler, "POOLNAME", poolName);
4121      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
4122    }
4123
4124    @Override
4125    void fromXml(Stanza st) throws InvalidXmlException {
4126      this.poolName = st.getValue("POOLNAME");
4127      readRpcIdsFromXml(st);
4128    }
4129
4130    @Override
4131    public String toString() {
4132      StringBuilder builder = new StringBuilder();
4133      builder.append("RemoveCachePoolOp [");
4134      builder.append("poolName=" + poolName);
4135      appendRpcIdsToString(builder, rpcClientId, rpcCallId);
4136      builder.append("]");
4137      return builder.toString();
4138    }
4139  }
4140  
4141  static class RemoveXAttrOp extends FSEditLogOp {
4142    List<XAttr> xAttrs;
4143    String src;
4144    
4145    private RemoveXAttrOp() {
4146      super(OP_REMOVE_XATTR);
4147    }
4148    
4149    static RemoveXAttrOp getInstance() {
4150      return new RemoveXAttrOp();
4151    }
4152
4153    @Override
4154    void resetSubFields() {
4155      xAttrs = null;
4156      src = null;
4157    }
4158
4159    @Override
4160    void readFields(DataInputStream in, int logVersion) throws IOException {
4161      XAttrEditLogProto p = XAttrEditLogProto.parseDelimitedFrom(in);
4162      src = p.getSrc();
4163      xAttrs = PBHelper.convertXAttrs(p.getXAttrsList());
4164      readRpcIds(in, logVersion);
4165    }
4166
4167    @Override
4168    public void writeFields(DataOutputStream out) throws IOException {
4169      XAttrEditLogProto.Builder b = XAttrEditLogProto.newBuilder();
4170      if (src != null) {
4171        b.setSrc(src);
4172      }
4173      b.addAllXAttrs(PBHelper.convertXAttrProto(xAttrs));
4174      b.build().writeDelimitedTo(out);
4175      // clientId and callId
4176      writeRpcIds(rpcClientId, rpcCallId, out);
4177    }
4178
4179    @Override
4180    protected void toXml(ContentHandler contentHandler) throws SAXException {
4181      XMLUtils.addSaxString(contentHandler, "SRC", src);
4182      appendXAttrsToXml(contentHandler, xAttrs);
4183      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
4184    }
4185
4186    @Override
4187    void fromXml(Stanza st) throws InvalidXmlException {
4188      src = st.getValue("SRC");
4189      xAttrs = readXAttrsFromXml(st);
4190      readRpcIdsFromXml(st);
4191    }
4192  }
4193  
4194  static class SetXAttrOp extends FSEditLogOp {
4195    List<XAttr> xAttrs;
4196    String src;
4197    
4198    private SetXAttrOp() {
4199      super(OP_SET_XATTR);
4200    }
4201    
4202    static SetXAttrOp getInstance() {
4203      return new SetXAttrOp();
4204    }
4205
4206    @Override
4207    void resetSubFields() {
4208      xAttrs = null;
4209      src = null;
4210    }
4211
4212    @Override
4213    void readFields(DataInputStream in, int logVersion) throws IOException {
4214      XAttrEditLogProto p = XAttrEditLogProto.parseDelimitedFrom(in);
4215      src = p.getSrc();
4216      xAttrs = PBHelper.convertXAttrs(p.getXAttrsList());
4217      readRpcIds(in, logVersion);
4218    }
4219
4220    @Override
4221    public void writeFields(DataOutputStream out) throws IOException {
4222      XAttrEditLogProto.Builder b = XAttrEditLogProto.newBuilder();
4223      if (src != null) {
4224        b.setSrc(src);
4225      }
4226      b.addAllXAttrs(PBHelper.convertXAttrProto(xAttrs));
4227      b.build().writeDelimitedTo(out);
4228      // clientId and callId
4229      writeRpcIds(rpcClientId, rpcCallId, out);
4230    }
4231
4232    @Override
4233    protected void toXml(ContentHandler contentHandler) throws SAXException {
4234      XMLUtils.addSaxString(contentHandler, "SRC", src);
4235      appendXAttrsToXml(contentHandler, xAttrs);
4236      appendRpcIdsToXml(contentHandler, rpcClientId, rpcCallId);
4237    }
4238
4239    @Override
4240    void fromXml(Stanza st) throws InvalidXmlException {
4241      src = st.getValue("SRC");
4242      xAttrs = readXAttrsFromXml(st);
4243      readRpcIdsFromXml(st);
4244    }
4245  }
4246
4247  static class SetAclOp extends FSEditLogOp {
4248    List<AclEntry> aclEntries = Lists.newArrayList();
4249    String src;
4250
4251    private SetAclOp() {
4252      super(OP_SET_ACL);
4253    }
4254
4255    static SetAclOp getInstance() {
4256      return new SetAclOp();
4257    }
4258
4259    @Override
4260    void resetSubFields() {
4261      aclEntries = null;
4262      src = null;
4263    }
4264
4265    @Override
4266    void readFields(DataInputStream in, int logVersion) throws IOException {
4267      AclEditLogProto p = AclEditLogProto.parseDelimitedFrom(in);
4268      if (p == null) {
4269        throw new IOException("Failed to read fields from SetAclOp");
4270      }
4271      src = p.getSrc();
4272      aclEntries = PBHelper.convertAclEntry(p.getEntriesList());
4273    }
4274
4275    @Override
4276    public void writeFields(DataOutputStream out) throws IOException {
4277      AclEditLogProto.Builder b = AclEditLogProto.newBuilder();
4278      if (src != null)
4279        b.setSrc(src);
4280      b.addAllEntries(PBHelper.convertAclEntryProto(aclEntries));
4281      b.build().writeDelimitedTo(out);
4282    }
4283
4284    @Override
4285    protected void toXml(ContentHandler contentHandler) throws SAXException {
4286      XMLUtils.addSaxString(contentHandler, "SRC", src);
4287      appendAclEntriesToXml(contentHandler, aclEntries);
4288    }
4289
4290    @Override
4291    void fromXml(Stanza st) throws InvalidXmlException {
4292      src = st.getValue("SRC");
4293      aclEntries = readAclEntriesFromXml(st);
4294      if (aclEntries == null) {
4295        aclEntries = Lists.newArrayList();
4296      }
4297    }
4298  }
4299
4300  static private short readShort(DataInputStream in) throws IOException {
4301    return Short.parseShort(FSImageSerialization.readString(in));
4302  }
4303
4304  static private long readLong(DataInputStream in) throws IOException {
4305    return Long.parseLong(FSImageSerialization.readString(in));
4306  }
4307
4308  /**
4309   * A class to read in blocks stored in the old format. The only two
4310   * fields in the block were blockid and length.
4311   */
4312  static class BlockTwo implements Writable {
4313    long blkid;
4314    long len;
4315
4316    static {                                      // register a ctor
4317      WritableFactories.setFactory
4318        (BlockTwo.class,
4319         new WritableFactory() {
4320           @Override
4321           public Writable newInstance() { return new BlockTwo(); }
4322         });
4323    }
4324
4325
4326    BlockTwo() {
4327      blkid = 0;
4328      len = 0;
4329    }
4330    /////////////////////////////////////
4331    // Writable
4332    /////////////////////////////////////
4333    @Override
4334    public void write(DataOutput out) throws IOException {
4335      out.writeLong(blkid);
4336      out.writeLong(len);
4337    }
4338
4339    @Override
4340    public void readFields(DataInput in) throws IOException {
4341      this.blkid = in.readLong();
4342      this.len = in.readLong();
4343    }
4344  }
4345  /**
4346   * Operation corresponding to upgrade
4347   */
4348  static class RollingUpgradeOp extends FSEditLogOp { // @Idempotent
4349    private final String name;
4350    private long time;
4351
4352    public RollingUpgradeOp(FSEditLogOpCodes code, String name) {
4353      super(code);
4354      this.name = StringUtils.toUpperCase(name);
4355    }
4356
4357    static RollingUpgradeOp getStartInstance(OpInstanceCache cache) {
4358      return (RollingUpgradeOp) cache.get(OP_ROLLING_UPGRADE_START);
4359    }
4360
4361    static RollingUpgradeOp getFinalizeInstance(OpInstanceCache cache) {
4362      return (RollingUpgradeOp) cache.get(OP_ROLLING_UPGRADE_FINALIZE);
4363    }
4364
4365    @Override
4366    void resetSubFields() {
4367      time = 0L;
4368    }
4369
4370    long getTime() {
4371      return time;
4372    }
4373
4374    void setTime(long time) {
4375      this.time = time;
4376    }
4377
4378    @Override
4379    void readFields(DataInputStream in, int logVersion) throws IOException {
4380      time = in.readLong();
4381    }
4382
4383    @Override
4384    public void writeFields(DataOutputStream out) throws IOException {
4385      FSImageSerialization.writeLong(time, out);
4386    }
4387
4388    @Override
4389    protected void toXml(ContentHandler contentHandler) throws SAXException {
4390      XMLUtils.addSaxString(contentHandler, name + "TIME",
4391          Long.toString(time));
4392    }
4393
4394    @Override
4395    void fromXml(Stanza st) throws InvalidXmlException {
4396      this.time = Long.parseLong(st.getValue(name + "TIME"));
4397    }
4398
4399    @Override
4400    public String toString() {
4401      return new StringBuilder().append("RollingUpgradeOp [").append(name)
4402          .append(", time=").append(time).append("]").toString();
4403    }
4404    
4405    static class RollbackException extends IOException {
4406      private static final long serialVersionUID = 1L;
4407    }
4408  }
4409
4410  /** {@literal @Idempotent} for {@link ClientProtocol#setStoragePolicy} */
4411  static class SetStoragePolicyOp extends FSEditLogOp {
4412    String path;
4413    byte policyId;
4414
4415    private SetStoragePolicyOp() {
4416      super(OP_SET_STORAGE_POLICY);
4417    }
4418
4419    static SetStoragePolicyOp getInstance(OpInstanceCache cache) {
4420      return (SetStoragePolicyOp) cache.get(OP_SET_STORAGE_POLICY);
4421    }
4422
4423    @Override
4424    void resetSubFields() {
4425      path = null;
4426      policyId = 0;
4427    }
4428
4429    SetStoragePolicyOp setPath(String path) {
4430      this.path = path;
4431      return this;
4432    }
4433
4434    SetStoragePolicyOp setPolicyId(byte policyId) {
4435      this.policyId = policyId;
4436      return this;
4437    }
4438
4439    @Override
4440    public void writeFields(DataOutputStream out) throws IOException {
4441      FSImageSerialization.writeString(path, out);
4442      out.writeByte(policyId);
4443    }
4444
4445    @Override
4446    void readFields(DataInputStream in, int logVersion)
4447        throws IOException {
4448      this.path = FSImageSerialization.readString(in);
4449      this.policyId = in.readByte();
4450    }
4451
4452    @Override
4453    public String toString() {
4454      StringBuilder builder = new StringBuilder();
4455      builder.append("SetStoragePolicyOp [path=");
4456      builder.append(path);
4457      builder.append(", policyId=");
4458      builder.append(policyId);
4459      builder.append(", opCode=");
4460      builder.append(opCode);
4461      builder.append(", txid=");
4462      builder.append(txid);
4463      builder.append("]");
4464      return builder.toString();
4465    }
4466
4467    @Override
4468    protected void toXml(ContentHandler contentHandler) throws SAXException {
4469      XMLUtils.addSaxString(contentHandler, "PATH", path);
4470      XMLUtils.addSaxString(contentHandler, "POLICYID",
4471          Byte.valueOf(policyId).toString());
4472    }
4473
4474    @Override
4475    void fromXml(Stanza st) throws InvalidXmlException {
4476      this.path = st.getValue("PATH");
4477      this.policyId = Byte.valueOf(st.getValue("POLICYID"));
4478    }
4479  }  
4480
4481  /**
4482   * Class for writing editlog ops
4483   */
4484  public static class Writer {
4485    private final DataOutputBuffer buf;
4486    private final Checksum checksum;
4487
4488    public Writer(DataOutputBuffer out) {
4489      this.buf = out;
4490      this.checksum = DataChecksum.newCrc32();
4491    }
4492
4493    /**
4494     * Write an operation to the output stream
4495     * 
4496     * @param op The operation to write
4497     * @throws IOException if an error occurs during writing.
4498     */
4499    public void writeOp(FSEditLogOp op) throws IOException {
4500      int start = buf.getLength();
4501      // write the op code first to make padding and terminator verification
4502      // work
4503      buf.writeByte(op.opCode.getOpCode());
4504      buf.writeInt(0); // write 0 for the length first
4505      buf.writeLong(op.txid);
4506      op.writeFields(buf);
4507      int end = buf.getLength();
4508      
4509      // write the length back: content of the op + 4 bytes checksum - op_code
4510      int length = end - start - 1;
4511      buf.writeInt(length, start + 1);
4512
4513      checksum.reset();
4514      checksum.update(buf.getData(), start, end-start);
4515      int sum = (int)checksum.getValue();
4516      buf.writeInt(sum);
4517    }
4518  }
4519
4520  /**
4521   * Class for reading editlog ops from a stream
4522   */
4523  public static class Reader {
4524    private final DataInputStream in;
4525    private final StreamLimiter limiter;
4526    private final int logVersion;
4527    private final Checksum checksum;
4528    private final OpInstanceCache cache;
4529    private int maxOpSize;
4530    private final boolean supportEditLogLength;
4531
4532    /**
4533     * Construct the reader
4534     * @param in The stream to read from.
4535     * @param logVersion The version of the data coming from the stream.
4536     */
4537    public Reader(DataInputStream in, StreamLimiter limiter, int logVersion) {
4538      this.logVersion = logVersion;
4539      if (NameNodeLayoutVersion.supports(
4540          LayoutVersion.Feature.EDITS_CHESKUM, logVersion)) {
4541        this.checksum = DataChecksum.newCrc32();
4542      } else {
4543        this.checksum = null;
4544      }
4545      // It is possible that the logVersion is actually a future layoutversion
4546      // during the rolling upgrade (e.g., the NN gets upgraded first). We
4547      // assume future layout will also support length of editlog op.
4548      this.supportEditLogLength = NameNodeLayoutVersion.supports(
4549          NameNodeLayoutVersion.Feature.EDITLOG_LENGTH, logVersion)
4550          || logVersion < NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION;
4551
4552      if (this.checksum != null) {
4553        this.in = new DataInputStream(
4554            new CheckedInputStream(in, this.checksum));
4555      } else {
4556        this.in = in;
4557      }
4558      this.limiter = limiter;
4559      this.cache = new OpInstanceCache();
4560      this.maxOpSize = DFSConfigKeys.DFS_NAMENODE_MAX_OP_SIZE_DEFAULT;
4561    }
4562
4563    public void setMaxOpSize(int maxOpSize) {
4564      this.maxOpSize = maxOpSize;
4565    }
4566
4567    /**
4568     * Read an operation from the input stream.
4569     * 
4570     * Note that the objects returned from this method may be re-used by future
4571     * calls to the same method.
4572     * 
4573     * @param skipBrokenEdits    If true, attempt to skip over damaged parts of
4574     * the input stream, rather than throwing an IOException
4575     * @return the operation read from the stream, or null at the end of the 
4576     *         file
4577     * @throws IOException on error.  This function should only throw an
4578     *         exception when skipBrokenEdits is false.
4579     */
4580    public FSEditLogOp readOp(boolean skipBrokenEdits) throws IOException {
4581      while (true) {
4582        try {
4583          return decodeOp();
4584        } catch (IOException e) {
4585          in.reset();
4586          if (!skipBrokenEdits) {
4587            throw e;
4588          }
4589        } catch (RuntimeException e) {
4590          // FSEditLogOp#decodeOp is not supposed to throw RuntimeException.
4591          // However, we handle it here for recovery mode, just to be more
4592          // robust.
4593          in.reset();
4594          if (!skipBrokenEdits) {
4595            throw e;
4596          }
4597        } catch (Throwable e) {
4598          in.reset();
4599          if (!skipBrokenEdits) {
4600            throw new IOException("got unexpected exception " +
4601                e.getMessage(), e);
4602          }
4603        }
4604        // Move ahead one byte and re-try the decode process.
4605        if (in.skip(1) < 1) {
4606          return null;
4607        }
4608      }
4609    }
4610
4611    private void verifyTerminator() throws IOException {
4612      /** The end of the edit log should contain only 0x00 or 0xff bytes.
4613       * If it contains other bytes, the log itself may be corrupt.
4614       * It is important to check this; if we don't, a stray OP_INVALID byte 
4615       * could make us stop reading the edit log halfway through, and we'd never
4616       * know that we had lost data.
4617       */
4618      byte[] buf = new byte[4096];
4619      limiter.clearLimit();
4620      int numRead = -1, idx = 0;
4621      while (true) {
4622        try {
4623          numRead = -1;
4624          idx = 0;
4625          numRead = in.read(buf);
4626          if (numRead == -1) {
4627            return;
4628          }
4629          while (idx < numRead) {
4630            if ((buf[idx] != (byte)0) && (buf[idx] != (byte)-1)) {
4631              throw new IOException("Read extra bytes after " +
4632                "the terminator!");
4633            }
4634            idx++;
4635          }
4636        } finally {
4637          // After reading each group of bytes, we reposition the mark one
4638          // byte before the next group.  Similarly, if there is an error, we
4639          // want to reposition the mark one byte before the error
4640          if (numRead != -1) { 
4641            in.reset();
4642            IOUtils.skipFully(in, idx);
4643            in.mark(buf.length + 1);
4644            IOUtils.skipFully(in, 1);
4645          }
4646        }
4647      }
4648    }
4649
4650    /**
4651     * Read an opcode from the input stream.
4652     *
4653     * @return   the opcode, or null on EOF.
4654     *
4655     * If an exception is thrown, the stream's mark will be set to the first
4656     * problematic byte.  This usually means the beginning of the opcode.
4657     */
4658    private FSEditLogOp decodeOp() throws IOException {
4659      limiter.setLimit(maxOpSize);
4660      in.mark(maxOpSize);
4661
4662      if (checksum != null) {
4663        checksum.reset();
4664      }
4665
4666      byte opCodeByte;
4667      try {
4668        opCodeByte = in.readByte();
4669      } catch (EOFException eof) {
4670        // EOF at an opcode boundary is expected.
4671        return null;
4672      }
4673
4674      FSEditLogOpCodes opCode = FSEditLogOpCodes.fromByte(opCodeByte);
4675      if (opCode == OP_INVALID) {
4676        verifyTerminator();
4677        return null;
4678      }
4679
4680      FSEditLogOp op = cache.get(opCode);
4681      if (op == null) {
4682        throw new IOException("Read invalid opcode " + opCode);
4683      }
4684
4685      if (supportEditLogLength) {
4686        in.readInt();
4687      }
4688
4689      if (NameNodeLayoutVersion.supports(
4690          LayoutVersion.Feature.STORED_TXIDS, logVersion)) {
4691        // Read the txid
4692        op.setTransactionId(in.readLong());
4693      } else {
4694        op.setTransactionId(HdfsConstants.INVALID_TXID);
4695      }
4696
4697      op.readFields(in, logVersion);
4698
4699      validateChecksum(in, checksum, op.txid);
4700      return op;
4701    }
4702
4703    /**
4704     * Similar with decodeOp(), but instead of doing the real decoding, we skip
4705     * the content of the op if the length of the editlog is supported.
4706     * @return the last txid of the segment, or INVALID_TXID on exception
4707     */
4708    public long scanOp() throws IOException {
4709      if (supportEditLogLength) {
4710        limiter.setLimit(maxOpSize);
4711        in.mark(maxOpSize);
4712
4713        final byte opCodeByte;
4714        try {
4715          opCodeByte = in.readByte(); // op code
4716        } catch (EOFException e) {
4717          return HdfsConstants.INVALID_TXID;
4718        }
4719
4720        FSEditLogOpCodes opCode = FSEditLogOpCodes.fromByte(opCodeByte);
4721        if (opCode == OP_INVALID) {
4722          verifyTerminator();
4723          return HdfsConstants.INVALID_TXID;
4724        }
4725
4726        int length = in.readInt(); // read the length of the op
4727        long txid = in.readLong(); // read the txid
4728
4729        // skip the remaining content
4730        IOUtils.skipFully(in, length - 8); 
4731        // TODO: do we want to verify checksum for JN? For now we don't.
4732        return txid;
4733      } else {
4734        FSEditLogOp op = decodeOp();
4735        return op == null ? HdfsConstants.INVALID_TXID : op.getTransactionId();
4736      }
4737    }
4738
4739    /**
4740     * Validate a transaction's checksum
4741     */
4742    private void validateChecksum(DataInputStream in,
4743                                  Checksum checksum,
4744                                  long txid)
4745        throws IOException {
4746      if (checksum != null) {
4747        int calculatedChecksum = (int)checksum.getValue();
4748        int readChecksum = in.readInt(); // read in checksum
4749        if (readChecksum != calculatedChecksum) {
4750          throw new ChecksumException(
4751              "Transaction is corrupt. Calculated checksum is " +
4752              calculatedChecksum + " but read checksum " + readChecksum, txid);
4753        }
4754      }
4755    }
4756  }
4757
4758  public void outputToXml(ContentHandler contentHandler) throws SAXException {
4759    contentHandler.startElement("", "", "RECORD", new AttributesImpl());
4760    XMLUtils.addSaxString(contentHandler, "OPCODE", opCode.toString());
4761    contentHandler.startElement("", "", "DATA", new AttributesImpl());
4762    XMLUtils.addSaxString(contentHandler, "TXID", "" + txid);
4763    toXml(contentHandler);
4764    contentHandler.endElement("", "", "DATA");
4765    contentHandler.endElement("", "", "RECORD");
4766  }
4767
4768  protected abstract void toXml(ContentHandler contentHandler)
4769      throws SAXException;
4770  
4771  abstract void fromXml(Stanza st) throws InvalidXmlException;
4772  
4773  public void decodeXml(Stanza st) throws InvalidXmlException {
4774    this.txid = Long.parseLong(st.getValue("TXID"));
4775    fromXml(st);
4776  }
4777  
4778  public static void blockToXml(ContentHandler contentHandler, Block block) 
4779      throws SAXException {
4780    contentHandler.startElement("", "", "BLOCK", new AttributesImpl());
4781    XMLUtils.addSaxString(contentHandler, "BLOCK_ID",
4782        Long.toString(block.getBlockId()));
4783    XMLUtils.addSaxString(contentHandler, "NUM_BYTES",
4784        Long.toString(block.getNumBytes()));
4785    XMLUtils.addSaxString(contentHandler, "GENSTAMP",
4786        Long.toString(block.getGenerationStamp()));
4787    contentHandler.endElement("", "", "BLOCK");
4788  }
4789
4790  public static Block blockFromXml(Stanza st)
4791      throws InvalidXmlException {
4792    long blockId = Long.parseLong(st.getValue("BLOCK_ID"));
4793    long numBytes = Long.parseLong(st.getValue("NUM_BYTES"));
4794    long generationStamp = Long.parseLong(st.getValue("GENSTAMP"));
4795    return new Block(blockId, numBytes, generationStamp);
4796  }
4797
4798  public static void delegationTokenToXml(ContentHandler contentHandler,
4799      DelegationTokenIdentifier token) throws SAXException {
4800    contentHandler.startElement("", "", "DELEGATION_TOKEN_IDENTIFIER", new AttributesImpl());
4801    XMLUtils.addSaxString(contentHandler, "KIND", token.getKind().toString());
4802    XMLUtils.addSaxString(contentHandler, "SEQUENCE_NUMBER",
4803        Integer.toString(token.getSequenceNumber()));
4804    XMLUtils.addSaxString(contentHandler, "OWNER",
4805        token.getOwner().toString());
4806    XMLUtils.addSaxString(contentHandler, "RENEWER",
4807        token.getRenewer().toString());
4808    XMLUtils.addSaxString(contentHandler, "REALUSER",
4809        token.getRealUser().toString());
4810    XMLUtils.addSaxString(contentHandler, "ISSUE_DATE",
4811        Long.toString(token.getIssueDate()));
4812    XMLUtils.addSaxString(contentHandler, "MAX_DATE",
4813        Long.toString(token.getMaxDate()));
4814    XMLUtils.addSaxString(contentHandler, "MASTER_KEY_ID",
4815        Integer.toString(token.getMasterKeyId()));
4816    contentHandler.endElement("", "", "DELEGATION_TOKEN_IDENTIFIER");
4817  }
4818
4819  public static DelegationTokenIdentifier delegationTokenFromXml(Stanza st)
4820      throws InvalidXmlException {
4821    String kind = st.getValue("KIND");
4822    if (!kind.equals(DelegationTokenIdentifier.
4823        HDFS_DELEGATION_KIND.toString())) {
4824      throw new InvalidXmlException("can't understand " +
4825        "DelegationTokenIdentifier KIND " + kind);
4826    }
4827    int seqNum = Integer.parseInt(st.getValue("SEQUENCE_NUMBER"));
4828    String owner = st.getValue("OWNER");
4829    String renewer = st.getValue("RENEWER");
4830    String realuser = st.getValue("REALUSER");
4831    long issueDate = Long.parseLong(st.getValue("ISSUE_DATE"));
4832    long maxDate = Long.parseLong(st.getValue("MAX_DATE"));
4833    int masterKeyId = Integer.parseInt(st.getValue("MASTER_KEY_ID"));
4834    DelegationTokenIdentifier token =
4835        new DelegationTokenIdentifier(new Text(owner),
4836            new Text(renewer), new Text(realuser));
4837    token.setSequenceNumber(seqNum);
4838    token.setIssueDate(issueDate);
4839    token.setMaxDate(maxDate);
4840    token.setMasterKeyId(masterKeyId);
4841    return token;
4842  }
4843
4844  public static void delegationKeyToXml(ContentHandler contentHandler,
4845      DelegationKey key) throws SAXException {
4846    contentHandler.startElement("", "", "DELEGATION_KEY", new AttributesImpl());
4847    XMLUtils.addSaxString(contentHandler, "KEY_ID",
4848        Integer.toString(key.getKeyId()));
4849    XMLUtils.addSaxString(contentHandler, "EXPIRY_DATE",
4850        Long.toString(key.getExpiryDate()));
4851    if (key.getEncodedKey() != null) {
4852      XMLUtils.addSaxString(contentHandler, "KEY",
4853          Hex.encodeHexString(key.getEncodedKey()));
4854    }
4855    contentHandler.endElement("", "", "DELEGATION_KEY");
4856  }
4857  
4858  public static DelegationKey delegationKeyFromXml(Stanza st)
4859      throws InvalidXmlException {
4860    int keyId = Integer.parseInt(st.getValue("KEY_ID"));
4861    long expiryDate = Long.parseLong(st.getValue("EXPIRY_DATE"));
4862    byte key[] = null;
4863    try {
4864      key = Hex.decodeHex(st.getValue("KEY").toCharArray());
4865    } catch (DecoderException e) {
4866      throw new InvalidXmlException(e.toString());
4867    } catch (InvalidXmlException e) {
4868    }
4869    return new DelegationKey(keyId, expiryDate, key);
4870  }
4871
4872  public static void permissionStatusToXml(ContentHandler contentHandler,
4873      PermissionStatus perm) throws SAXException {
4874    contentHandler.startElement("", "", "PERMISSION_STATUS", new AttributesImpl());
4875    XMLUtils.addSaxString(contentHandler, "USERNAME", perm.getUserName());
4876    XMLUtils.addSaxString(contentHandler, "GROUPNAME", perm.getGroupName());
4877    fsPermissionToXml(contentHandler, perm.getPermission());
4878    contentHandler.endElement("", "", "PERMISSION_STATUS");
4879  }
4880
4881  public static PermissionStatus permissionStatusFromXml(Stanza st)
4882      throws InvalidXmlException {
4883    Stanza status = st.getChildren("PERMISSION_STATUS").get(0);
4884    String username = status.getValue("USERNAME");
4885    String groupname = status.getValue("GROUPNAME");
4886    FsPermission mode = fsPermissionFromXml(status);
4887    return new PermissionStatus(username, groupname, mode);
4888  }
4889
4890  public static void fsPermissionToXml(ContentHandler contentHandler,
4891      FsPermission mode) throws SAXException {
4892    XMLUtils.addSaxString(contentHandler, "MODE", Short.valueOf(mode.toShort())
4893        .toString());
4894  }
4895
4896  public static FsPermission fsPermissionFromXml(Stanza st)
4897      throws InvalidXmlException {
4898    short mode = Short.valueOf(st.getValue("MODE"));
4899    return new FsPermission(mode);
4900  }
4901
4902  private static void fsActionToXml(ContentHandler contentHandler, FsAction v)
4903      throws SAXException {
4904    XMLUtils.addSaxString(contentHandler, "PERM", v.SYMBOL);
4905  }
4906
4907  private static FsAction fsActionFromXml(Stanza st) throws InvalidXmlException {
4908    FsAction v = FSACTION_SYMBOL_MAP.get(st.getValue("PERM"));
4909    if (v == null)
4910      throw new InvalidXmlException("Invalid value for FsAction");
4911    return v;
4912  }
4913
4914  private static void appendAclEntriesToXml(ContentHandler contentHandler,
4915      List<AclEntry> aclEntries) throws SAXException {
4916    for (AclEntry e : aclEntries) {
4917      contentHandler.startElement("", "", "ENTRY", new AttributesImpl());
4918      XMLUtils.addSaxString(contentHandler, "SCOPE", e.getScope().name());
4919      XMLUtils.addSaxString(contentHandler, "TYPE", e.getType().name());
4920      if (e.getName() != null) {
4921        XMLUtils.addSaxString(contentHandler, "NAME", e.getName());
4922      }
4923      fsActionToXml(contentHandler, e.getPermission());
4924      contentHandler.endElement("", "", "ENTRY");
4925    }
4926  }
4927
4928  private static List<AclEntry> readAclEntriesFromXml(Stanza st) {
4929    List<AclEntry> aclEntries = Lists.newArrayList();
4930    if (!st.hasChildren("ENTRY"))
4931      return null;
4932
4933    List<Stanza> stanzas = st.getChildren("ENTRY");
4934    for (Stanza s : stanzas) {
4935      AclEntry e = new AclEntry.Builder()
4936        .setScope(AclEntryScope.valueOf(s.getValue("SCOPE")))
4937        .setType(AclEntryType.valueOf(s.getValue("TYPE")))
4938        .setName(s.getValueOrNull("NAME"))
4939        .setPermission(fsActionFromXml(s)).build();
4940      aclEntries.add(e);
4941    }
4942    return aclEntries;
4943  }
4944
4945  private static void appendXAttrsToXml(ContentHandler contentHandler,
4946      List<XAttr> xAttrs) throws SAXException {
4947    for (XAttr xAttr: xAttrs) {
4948      contentHandler.startElement("", "", "XATTR", new AttributesImpl());
4949      XMLUtils.addSaxString(contentHandler, "NAMESPACE",
4950          xAttr.getNameSpace().toString());
4951      XMLUtils.addSaxString(contentHandler, "NAME", xAttr.getName());
4952      if (xAttr.getValue() != null) {
4953        try {
4954          XMLUtils.addSaxString(contentHandler, "VALUE",
4955              XAttrCodec.encodeValue(xAttr.getValue(), XAttrCodec.HEX));
4956        } catch (IOException e) {
4957          throw new SAXException(e);
4958        }
4959      }
4960      contentHandler.endElement("", "", "XATTR");
4961    }
4962  }
4963
4964  private static List<XAttr> readXAttrsFromXml(Stanza st)
4965      throws InvalidXmlException {
4966    if (!st.hasChildren("XATTR")) {
4967      return null;
4968    }
4969
4970    List<Stanza> stanzas = st.getChildren("XATTR");
4971    List<XAttr> xattrs = Lists.newArrayListWithCapacity(stanzas.size());
4972    for (Stanza a: stanzas) {
4973      XAttr.Builder builder = new XAttr.Builder();
4974      builder.setNameSpace(XAttr.NameSpace.valueOf(a.getValue("NAMESPACE"))).
4975          setName(a.getValue("NAME"));
4976      String v = a.getValueOrNull("VALUE");
4977      if (v != null) {
4978        try {
4979          builder.setValue(XAttrCodec.decodeValue(v));
4980        } catch (IOException e) {
4981          throw new InvalidXmlException(e.toString());
4982        }
4983      }
4984      xattrs.add(builder.build());
4985    }
4986    return xattrs;
4987  }
4988}